From 7f3b74c052f0473dab30677214e187d676876237 Mon Sep 17 00:00:00 2001 From: Frances Perry Date: Sat, 13 Dec 2014 10:56:14 -0800 Subject: [PATCH 0001/1541] Welcome to Dataflow! --- README | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) create mode 100644 README diff --git a/README b/README new file mode 100644 index 0000000000000..2da8ea5ba93c4 --- /dev/null +++ b/README @@ -0,0 +1,19 @@ +Greetings! Welcome to the (Alpha) Google Cloud Dataflow Java SDK. + +Dataflow provides a simple, powerful model for building both batch and +streaming parallel data processing Pipelines. + +To use the Google Cloud Dataflow SDK, you build a Pipeline which manages a +graph of PTransforms and PCollections that the PTransforms consume and produce. + +You then use a PipelineRunner to specify where and how the pipeline should +execute. Currently there are two runners: + 1. The DirectPipelineRunner runs the pipeline on your local machine. + 2. The [Blocking]DataflowPipelineRunner runs the pipeline on the Dataflow + Service using the Google Cloud Platform. The Dataflow Service is + currently in the Alpha phase of development and access is limited to + whitelisted users. + +For more about both the Dataflow SDK and the Dataflow Service, visit: + http://cloud.google.com/dataflow + From ec739e8bac4ac2b21fe9e3e16f9900cc0a4e26e1 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Sat, 13 Dec 2014 12:20:10 -0800 Subject: [PATCH 0002/1541] Let's get this party started. --- .gitignore | 1 + LICENSE | 202 ++++ checkstyle.xml | 385 ++++++ examples/pom.xml | 223 ++++ .../dataflow/examples/BigQueryTornadoes.java | 149 +++ .../dataflow/examples/DatastoreWordCount.java | 198 ++++ .../google/cloud/dataflow/examples/TfIdf.java | 425 +++++++ .../examples/TopWikipediaSessions.java | 208 ++++ .../cloud/dataflow/examples/WordCount.java | 174 +++ .../examples/BigQueryTornadoesTest.java | 80 ++ .../cloud/dataflow/examples/TfIdfTest.java | 63 + .../examples/TopWikipediaSessionsTest.java | 62 + .../dataflow/examples/WordCountTest.java | 81 ++ pom.xml | 202 ++++ sdk/pom.xml | 315 +++++ .../google/cloud/dataflow/sdk/Pipeline.java | 395 ++++++ .../cloud/dataflow/sdk/PipelineResult.java | 27 + .../dataflow/sdk/coders/AtomicCoder.java | 42 + .../cloud/dataflow/sdk/coders/AvroCoder.java | 202 ++++ .../sdk/coders/BigEndianIntegerCoder.java | 88 ++ .../sdk/coders/BigEndianLongCoder.java | 87 ++ .../dataflow/sdk/coders/ByteArrayCoder.java | 103 ++ .../cloud/dataflow/sdk/coders/Coder.java | 154 +++ .../dataflow/sdk/coders/CoderException.java | 37 + .../dataflow/sdk/coders/CoderRegistry.java | 701 +++++++++++ .../dataflow/sdk/coders/CollectionCoder.java | 63 + .../dataflow/sdk/coders/CustomCoder.java | 83 ++ .../dataflow/sdk/coders/DefaultCoder.java | 68 ++ .../dataflow/sdk/coders/DoubleCoder.java | 92 ++ .../dataflow/sdk/coders/EntityCoder.java | 82 ++ .../dataflow/sdk/coders/InstantCoder.java | 60 + .../dataflow/sdk/coders/IterableCoder.java | 72 ++ .../sdk/coders/IterableLikeCoder.java | 227 ++++ .../cloud/dataflow/sdk/coders/KvCoder.java | 142 +++ .../dataflow/sdk/coders/KvCoderBase.java | 53 + .../cloud/dataflow/sdk/coders/ListCoder.java | 70 ++ .../cloud/dataflow/sdk/coders/MapCoder.java | 149 +++ .../dataflow/sdk/coders/MapCoderBase.java | 52 + .../sdk/coders/SerializableCoder.java | 126 ++ .../cloud/dataflow/sdk/coders/SetCoder.java | 124 ++ .../dataflow/sdk/coders/StandardCoder.java | 143 +++ .../dataflow/sdk/coders/StringUtf8Coder.java | 124 ++ .../sdk/coders/TableRowJsonCoder.java | 80 ++ .../sdk/coders/TextualIntegerCoder.java | 73 ++ .../cloud/dataflow/sdk/coders/URICoder.java | 77 ++ .../dataflow/sdk/coders/VarIntCoder.java | 90 ++ .../dataflow/sdk/coders/VarLongCoder.java | 90 ++ .../cloud/dataflow/sdk/coders/VoidCoder.java | 69 ++ .../dataflow/sdk/coders/package-info.java | 44 + .../google/cloud/dataflow/sdk/io/AvroIO.java | 678 +++++++++++ .../cloud/dataflow/sdk/io/BigQueryIO.java | 937 +++++++++++++++ .../cloud/dataflow/sdk/io/DatastoreIO.java | 603 ++++++++++ .../dataflow/sdk/io/DatastoreIterator.java | 141 +++ .../cloud/dataflow/sdk/io/PubsubIO.java | 331 ++++++ .../dataflow/sdk/io/ShardNameTemplate.java | 75 ++ .../google/cloud/dataflow/sdk/io/TextIO.java | 567 +++++++++ .../cloud/dataflow/sdk/io/package-info.java | 37 + .../sdk/options/ApplicationNameOptions.java | 33 + .../dataflow/sdk/options/BigQueryOptions.java | 29 + .../BlockingDataflowPipelineOptions.java | 46 + .../options/DataflowPipelineDebugOptions.java | 67 ++ .../sdk/options/DataflowPipelineOptions.java | 128 ++ .../DataflowPipelineShuffleOptions.java | 58 + .../DataflowPipelineWorkerPoolOptions.java | 116 ++ .../options/DataflowWorkerHarnessOptions.java | 35 + .../cloud/dataflow/sdk/options/Default.java | 130 ++ .../sdk/options/DefaultValueFactory.java | 38 + .../dataflow/sdk/options/Description.java | 31 + .../sdk/options/DirectPipelineOptions.java | 28 + .../dataflow/sdk/options/GcpOptions.java | 150 +++ .../dataflow/sdk/options/GcsOptions.java | 77 ++ .../dataflow/sdk/options/PipelineOptions.java | 62 + .../sdk/options/PipelineOptionsFactory.java | 862 ++++++++++++++ .../sdk/options/PipelineOptionsValidator.java | 59 + .../sdk/options/ProxyInvocationHandler.java | 390 ++++++ .../sdk/options/StreamingOptions.java | 38 + .../dataflow/sdk/options/Validation.java | 39 + .../dataflow/sdk/options/package-info.java | 25 + .../cloud/dataflow/sdk/package-info.java | 35 + .../BlockingDataflowPipelineRunner.java | 136 +++ .../sdk/runners/DataflowPipeline.java | 49 + .../sdk/runners/DataflowPipelineJob.java | 169 +++ .../sdk/runners/DataflowPipelineRunner.java | 315 +++++ .../runners/DataflowPipelineRunnerHooks.java | 40 + .../runners/DataflowPipelineTranslator.java | 963 +++++++++++++++ .../dataflow/sdk/runners/DirectPipeline.java | 50 + .../sdk/runners/DirectPipelineRunner.java | 844 +++++++++++++ .../dataflow/sdk/runners/PipelineRunner.java | 76 ++ .../sdk/runners/RecordingPipelineVisitor.java | 53 + .../sdk/runners/TransformHierarchy.java | 111 ++ .../sdk/runners/TransformTreeNode.java | 237 ++++ .../runners/dataflow/AvroIOTranslator.java | 113 ++ .../dataflow/BigQueryIOTranslator.java | 200 ++++ .../dataflow/DatastoreIOTranslator.java | 41 + .../runners/dataflow/PubsubIOTranslator.java | 91 ++ .../runners/dataflow/TextIOTranslator.java | 129 ++ .../sdk/runners/dataflow/package-info.java | 20 + .../dataflow/sdk/runners/package-info.java | 33 + .../worker/ApplianceShuffleReader.java | 63 + .../worker/ApplianceShuffleWriter.java | 66 ++ .../runners/worker/AssignWindowsParDoFn.java | 86 ++ .../sdk/runners/worker/AvroByteSink.java | 83 ++ .../sdk/runners/worker/AvroByteSource.java | 95 ++ .../dataflow/sdk/runners/worker/AvroSink.java | 140 +++ .../sdk/runners/worker/AvroSinkFactory.java | 61 + .../sdk/runners/worker/AvroSource.java | 203 ++++ .../sdk/runners/worker/AvroSourceFactory.java | 65 + .../sdk/runners/worker/BigQuerySource.java | 114 ++ .../runners/worker/BigQuerySourceFactory.java | 46 + .../worker/ByteArrayShufflePosition.java | 95 ++ .../worker/ChunkingShuffleBatchReader.java | 97 ++ .../worker/ChunkingShuffleEntryWriter.java | 87 ++ .../sdk/runners/worker/CombineValuesFn.java | 219 ++++ .../worker/CopyableSeekableByteChannel.java | 270 +++++ .../worker/CustomSourceFormatFactory.java | 47 + .../worker/DataflowWorkProgressUpdater.java | 121 ++ .../sdk/runners/worker/DataflowWorker.java | 330 ++++++ .../runners/worker/DataflowWorkerHarness.java | 231 ++++ .../sdk/runners/worker/FileBasedSource.java | 259 ++++ .../worker/GroupAlsoByWindowsParDoFn.java | 119 ++ .../runners/worker/GroupingShuffleSource.java | 368 ++++++ .../worker/GroupingShuffleSourceFactory.java | 62 + .../sdk/runners/worker/InMemorySource.java | 163 +++ .../runners/worker/InMemorySourceFactory.java | 54 + .../worker/LazyMultiSourceIterator.java | 87 ++ .../worker/MapTaskExecutorFactory.java | 413 +++++++ .../sdk/runners/worker/NormalParDoFn.java | 214 ++++ .../sdk/runners/worker/OrderedCode.java | 678 +++++++++++ .../sdk/runners/worker/ParDoFnFactory.java | 115 ++ .../worker/PartitioningShuffleSource.java | 128 ++ .../PartitioningShuffleSourceFactory.java | 50 + .../runners/worker/ShuffleEntryWriter.java | 39 + .../sdk/runners/worker/ShuffleLibrary.java | 44 + .../sdk/runners/worker/ShuffleReader.java | 48 + .../sdk/runners/worker/ShuffleSink.java | 248 ++++ .../runners/worker/ShuffleSinkFactory.java | 55 + .../sdk/runners/worker/ShuffleWriter.java | 37 + .../sdk/runners/worker/SideInputUtils.java | 211 ++++ .../sdk/runners/worker/SinkFactory.java | 94 ++ .../sdk/runners/worker/SourceFactory.java | 113 ++ .../worker/SourceOperationExecutor.java | 72 ++ .../SourceOperationExecutorFactory.java | 31 + .../worker/SourceTranslationUtils.java | 189 +++ .../dataflow/sdk/runners/worker/TextSink.java | 285 +++++ .../sdk/runners/worker/TextSinkFactory.java | 55 + .../sdk/runners/worker/TextSource.java | 383 ++++++ .../sdk/runners/worker/TextSourceFactory.java | 74 ++ .../worker/UngroupedShuffleSource.java | 96 ++ .../worker/UngroupedShuffleSourceFactory.java | 56 + .../DataflowWorkerLoggingFormatter.java | 77 ++ .../DataflowWorkerLoggingInitializer.java | 88 ++ .../sdk/runners/worker/package-info.java | 24 + .../dataflow/sdk/testing/DataflowAssert.java | 374 ++++++ .../sdk/testing/RunnableOnService.java | 29 + .../testing/TestDataflowPipelineOptions.java | 26 + .../testing/TestDataflowPipelineRunner.java | 45 + .../dataflow/sdk/testing/TestPipeline.java | 164 +++ .../sdk/testing/WindowingFnTestUtils.java | 185 +++ .../dataflow/sdk/testing/package-info.java | 21 + .../dataflow/sdk/transforms/Aggregator.java | 64 + .../sdk/transforms/ApproximateQuantiles.java | 723 +++++++++++ .../sdk/transforms/ApproximateUnique.java | 426 +++++++ .../dataflow/sdk/transforms/Combine.java | 1045 ++++++++++++++++ .../cloud/dataflow/sdk/transforms/Count.java | 163 +++ .../cloud/dataflow/sdk/transforms/Create.java | 314 +++++ .../cloud/dataflow/sdk/transforms/DoFn.java | 330 ++++++ .../dataflow/sdk/transforms/DoFnTester.java | 357 ++++++ .../cloud/dataflow/sdk/transforms/First.java | 106 ++ .../dataflow/sdk/transforms/Flatten.java | 206 ++++ .../dataflow/sdk/transforms/GroupByKey.java | 517 ++++++++ .../cloud/dataflow/sdk/transforms/Keys.java | 68 ++ .../cloud/dataflow/sdk/transforms/KvSwap.java | 73 ++ .../cloud/dataflow/sdk/transforms/Max.java | 196 +++ .../cloud/dataflow/sdk/transforms/Mean.java | 143 +++ .../cloud/dataflow/sdk/transforms/Min.java | 196 +++ .../dataflow/sdk/transforms/PTransform.java | 400 +++++++ .../cloud/dataflow/sdk/transforms/ParDo.java | 1054 +++++++++++++++++ .../dataflow/sdk/transforms/Partition.java | 173 +++ .../dataflow/sdk/transforms/RateLimiting.java | 336 ++++++ .../sdk/transforms/RemoveDuplicates.java | 89 ++ .../cloud/dataflow/sdk/transforms/Sample.java | 154 +++ .../transforms/SerializableComparator.java | 28 + .../sdk/transforms/SerializableFunction.java | 31 + .../cloud/dataflow/sdk/transforms/Sum.java | 179 +++ .../cloud/dataflow/sdk/transforms/Top.java | 489 ++++++++ .../cloud/dataflow/sdk/transforms/Values.java | 68 ++ .../cloud/dataflow/sdk/transforms/View.java | 211 ++++ .../dataflow/sdk/transforms/WithKeys.java | 116 ++ .../sdk/transforms/join/CoGbkResult.java | 367 ++++++ .../transforms/join/CoGbkResultSchema.java | 133 +++ .../sdk/transforms/join/CoGroupByKey.java | 208 ++++ .../join/KeyedPCollectionTuple.java | 217 ++++ .../sdk/transforms/join/RawUnionValue.java | 51 + .../sdk/transforms/join/UnionCoder.java | 149 +++ .../sdk/transforms/join/package-info.java | 21 + .../dataflow/sdk/transforms/package-info.java | 43 + .../transforms/windowing/BoundedWindow.java | 37 + .../transforms/windowing/CalendarWindows.java | 300 +++++ .../transforms/windowing/FixedWindows.java | 93 ++ .../transforms/windowing/GlobalWindow.java | 84 ++ .../transforms/windowing/IntervalWindow.java | 257 ++++ .../windowing/InvalidWindowingFn.java | 75 ++ .../MergeOverlappingIntervalWindows.java | 86 ++ .../windowing/NonMergingWindowingFn.java | 31 + .../windowing/PartitioningWindowingFn.java | 42 + .../sdk/transforms/windowing/Sessions.java | 81 ++ .../transforms/windowing/SlidingWindows.java | 131 ++ .../sdk/transforms/windowing/Window.java | 321 +++++ .../sdk/transforms/windowing/WindowingFn.java | 117 ++ .../dataflow/sdk/util/AbstractWindowSet.java | 170 +++ .../dataflow/sdk/util/AggregatorImpl.java | 111 ++ .../dataflow/sdk/util/ApiErrorExtractor.java | 104 ++ .../sdk/util/AppEngineEnvironment.java | 61 + .../dataflow/sdk/util/AssignWindowsDoFn.java | 64 + .../AttemptBoundedExponentialBackOff.java | 82 ++ .../cloud/dataflow/sdk/util/Base64Utils.java | 30 + .../sdk/util/BatchModeExecutionContext.java | 157 +++ .../sdk/util/BigQueryTableInserter.java | 240 ++++ .../sdk/util/BigQueryTableRowIterator.java | 201 ++++ .../dataflow/sdk/util/BufferingWindowSet.java | 193 +++ .../dataflow/sdk/util/CloudCounterUtils.java | 104 ++ .../dataflow/sdk/util/CloudKnownType.java | 138 +++ .../dataflow/sdk/util/CloudMetricUtils.java | 73 ++ .../cloud/dataflow/sdk/util/CloudObject.java | 184 +++ .../dataflow/sdk/util/CloudSourceUtils.java | 80 ++ .../cloud/dataflow/sdk/util/CoderUtils.java | 202 ++++ .../cloud/dataflow/sdk/util/Credentials.java | 244 ++++ .../sdk/util/DataflowReleaseInfo.java | 87 ++ .../sdk/util/DirectModeExecutionContext.java | 68 ++ .../cloud/dataflow/sdk/util/DoFnContext.java | 193 +++ .../dataflow/sdk/util/DoFnProcessContext.java | 136 +++ .../cloud/dataflow/sdk/util/DoFnRunner.java | 147 +++ .../dataflow/sdk/util/ExecutionContext.java | 168 +++ .../sdk/util/FileIOChannelFactory.java | 91 ++ .../dataflow/sdk/util/GCloudCredential.java | 113 ++ .../sdk/util/GcsIOChannelFactory.java | 75 ++ .../cloud/dataflow/sdk/util/GcsUtil.java | 277 +++++ .../sdk/util/GroupAlsoByWindowsDoFn.java | 359 ++++++ .../dataflow/sdk/util/IOChannelFactory.java | 69 ++ .../dataflow/sdk/util/IOChannelUtils.java | 179 +++ .../dataflow/sdk/util/InstanceBuilder.java | 259 ++++ .../cloud/dataflow/sdk/util/MimeTypes.java | 23 + .../dataflow/sdk/util/MonitoringUtil.java | 230 ++++ .../dataflow/sdk/util/OutputReference.java | 42 + .../cloud/dataflow/sdk/util/PTuple.java | 152 +++ .../cloud/dataflow/sdk/util/PackageUtil.java | 307 +++++ .../sdk/util/PartitionBufferingWindowSet.java | 87 ++ .../dataflow/sdk/util/PropertyNames.java | 87 ++ .../sdk/util/RetryHttpRequestInitializer.java | 165 +++ .../dataflow/sdk/util/SerializableUtils.java | 145 +++ .../cloud/dataflow/sdk/util/Serializer.java | 152 +++ .../sdk/util/ShardingWritableByteChannel.java | 118 ++ .../util/StreamingGroupAlsoByWindowsDoFn.java | 133 +++ .../cloud/dataflow/sdk/util/StringUtils.java | 146 +++ .../cloud/dataflow/sdk/util/Structs.java | 345 ++++++ .../dataflow/sdk/util/TestCredential.java | 49 + .../cloud/dataflow/sdk/util/TimeUtil.java | 164 +++ .../dataflow/sdk/util/TimerOrElement.java | 195 +++ .../cloud/dataflow/sdk/util/Transport.java | 141 +++ .../dataflow/sdk/util/UserCodeException.java | 132 +++ .../cloud/dataflow/sdk/util/Values.java | 88 ++ .../cloud/dataflow/sdk/util/VarInt.java | 115 ++ .../cloud/dataflow/sdk/util/WindowUtils.java | 62 + .../dataflow/sdk/util/WindowedValue.java | 368 ++++++ .../dataflow/sdk/util/common/Counter.java | 730 ++++++++++++ .../dataflow/sdk/util/common/CounterSet.java | 152 +++ .../common/ElementByteSizeObservable.java | 41 + .../ElementByteSizeObservableIterable.java | 63 + .../ElementByteSizeObservableIterator.java | 36 + .../util/common/ElementByteSizeObserver.java | 84 ++ .../sdk/util/common/ForwardingReiterator.java | 83 ++ .../dataflow/sdk/util/common/Metric.java | 45 + .../sdk/util/common/PeekingReiterator.java | 98 ++ .../dataflow/sdk/util/common/Reiterable.java | 27 + .../dataflow/sdk/util/common/Reiterator.java | 39 + .../sdk/util/common/package-info.java | 18 + .../worker/BatchingShuffleEntryReader.java | 148 +++ .../worker/CachingShuffleBatchReader.java | 228 ++++ .../common/worker/CustomSourceFormat.java | 61 + .../util/common/worker/FlattenOperation.java | 54 + .../worker/GroupingShuffleEntryIterator.java | 216 ++++ .../worker/KeyGroupedShuffleEntries.java | 35 + .../util/common/worker/MapTaskExecutor.java | 116 ++ .../sdk/util/common/worker/Operation.java | 132 +++ .../util/common/worker/OutputReceiver.java | 207 ++++ .../sdk/util/common/worker/ParDoFn.java | 28 + .../util/common/worker/ParDoOperation.java | 65 + .../worker/PartialGroupByKeyOperation.java | 521 ++++++++ .../util/common/worker/ProgressTracker.java | 38 + .../common/worker/ProgressTrackerGroup.java | 71 ++ .../worker/ProgressTrackingReiterator.java | 57 + .../sdk/util/common/worker/ReadOperation.java | 233 ++++ .../sdk/util/common/worker/Receiver.java | 27 + .../common/worker/ReceivingOperation.java | 45 + .../common/worker/ShuffleBatchReader.java | 61 + .../sdk/util/common/worker/ShuffleEntry.java | 110 ++ .../common/worker/ShuffleEntryReader.java | 50 + .../util/common/worker/ShufflePosition.java | 23 + .../dataflow/sdk/util/common/worker/Sink.java | 47 + .../sdk/util/common/worker/Source.java | 157 +++ .../sdk/util/common/worker/StateSampler.java | 279 +++++ .../sdk/util/common/worker/WorkExecutor.java | 99 ++ .../common/worker/WorkProgressUpdater.java | 239 ++++ .../util/common/worker/WriteOperation.java | 105 ++ .../sdk/util/common/worker/package-info.java | 18 + .../dataflow/sdk/util/gcsfs/GcsPath.java | 617 ++++++++++ .../dataflow/sdk/util/gcsfs/package-info.java | 18 + .../sdk/util/gcsio/ClientRequestHelper.java | 40 + .../gcsio/GoogleCloudStorageExceptions.java | 82 ++ .../gcsio/GoogleCloudStorageReadChannel.java | 538 +++++++++ .../gcsio/GoogleCloudStorageWriteChannel.java | 379 ++++++ ...gingMediaHttpUploaderProgressListener.java | 91 ++ .../sdk/util/gcsio/StorageResourceId.java | 165 +++ .../cloud/dataflow/sdk/util/package-info.java | 18 + .../dataflow/sdk/values/CodedTupleTag.java | 72 ++ .../dataflow/sdk/values/CodedTupleTagMap.java | 59 + .../google/cloud/dataflow/sdk/values/KV.java | 117 ++ .../cloud/dataflow/sdk/values/PBegin.java | 77 ++ .../dataflow/sdk/values/PCollection.java | 240 ++++ .../dataflow/sdk/values/PCollectionList.java | 227 ++++ .../dataflow/sdk/values/PCollectionTuple.java | 252 ++++ .../dataflow/sdk/values/PCollectionView.java | 45 + .../cloud/dataflow/sdk/values/PDone.java | 36 + .../cloud/dataflow/sdk/values/PInput.java | 57 + .../cloud/dataflow/sdk/values/POutput.java | 72 ++ .../dataflow/sdk/values/POutputValueBase.java | 83 ++ .../cloud/dataflow/sdk/values/PValue.java | 37 + .../cloud/dataflow/sdk/values/PValueBase.java | 190 +++ .../dataflow/sdk/values/TimestampedValue.java | 133 +++ .../cloud/dataflow/sdk/values/TupleTag.java | 170 +++ .../dataflow/sdk/values/TupleTagList.java | 146 +++ .../dataflow/sdk/values/TypedPValue.java | 168 +++ .../dataflow/sdk/values/package-info.java | 42 + .../google/cloud/dataflow/sdk/sdk.properties | 5 + .../cloud/dataflow/sdk/PipelineTest.java | 105 ++ .../google/cloud/dataflow/sdk/TestUtils.java | 231 ++++ .../dataflow/sdk/coders/AvroCoderTest.java | 189 +++ .../sdk/coders/ByteArrayCoderTest.java | 70 ++ .../dataflow/sdk/coders/CoderProperties.java | 73 ++ .../sdk/coders/CoderRegistryTest.java | 230 ++++ .../dataflow/sdk/coders/CustomCoderTest.java | 83 ++ .../dataflow/sdk/coders/DefaultCoderTest.java | 93 ++ .../dataflow/sdk/coders/InstantCoderTest.java | 67 ++ .../sdk/coders/IterableCoderTest.java | 46 + .../dataflow/sdk/coders/ListCoderTest.java | 46 + .../dataflow/sdk/coders/MapCoderTest.java | 49 + .../sdk/coders/SerializableCoderTest.java | 182 +++ .../dataflow/sdk/coders/URICoderTest.java | 68 ++ .../cloud/dataflow/sdk/io/AvroIOTest.java | 365 ++++++ .../cloud/dataflow/sdk/io/BigQueryIOTest.java | 307 +++++ .../dataflow/sdk/io/DatastoreIOTest.java | 126 ++ .../cloud/dataflow/sdk/io/TextIOTest.java | 413 +++++++ .../google/cloud/dataflow/sdk/io/user.avsc | 10 + .../options/DataflowPipelineOptionsTest.java | 94 ++ .../options/PipelineOptionsFactoryTest.java | 502 ++++++++ .../sdk/options/PipelineOptionsTest.java | 45 + .../options/PipelineOptionsValidatorTest.java | 86 ++ .../options/ProxyInvocationHandlerTest.java | 625 ++++++++++ .../BlockingDataflowPipelineRunnerTest.java | 137 +++ .../sdk/runners/DataflowPipelineJobTest.java | 66 ++ .../runners/DataflowPipelineRunnerTest.java | 501 ++++++++ .../DataflowPipelineTranslatorTest.java | 582 +++++++++ .../sdk/runners/PipelineRunnerTest.java | 84 ++ .../sdk/runners/TransformTreeTest.java | 179 +++ .../sdk/runners/worker/AvroByteSinkTest.java | 114 ++ .../runners/worker/AvroByteSourceTest.java | 200 ++++ .../runners/worker/AvroSinkFactoryTest.java | 83 ++ .../sdk/runners/worker/AvroSinkTest.java | 104 ++ .../runners/worker/AvroSourceFactoryTest.java | 115 ++ .../sdk/runners/worker/AvroSourceTest.java | 196 +++ .../worker/BigQuerySourceFactoryTest.java | 78 ++ .../runners/worker/BigQuerySourceTest.java | 183 +++ .../runners/worker/CombineValuesFnTest.java | 337 ++++++ .../CopyableSeekableByteChannelTest.java | 152 +++ .../DataflowWorkProgressUpdaterTest.java | 438 +++++++ .../worker/DataflowWorkerHarnessTest.java | 243 ++++ .../runners/worker/DataflowWorkerTest.java | 85 ++ .../worker/GroupingShuffleSourceTest.java | 499 ++++++++ .../worker/InMemorySourceFactoryTest.java | 110 ++ .../runners/worker/InMemorySourceTest.java | 236 ++++ .../worker/MapTaskExecutorFactoryTest.java | 567 +++++++++ .../sdk/runners/worker/NormalParDoFnTest.java | 331 ++++++ .../sdk/runners/worker/OrderedCodeTest.java | 504 ++++++++ .../runners/worker/ParDoFnFactoryTest.java | 125 ++ .../worker/PartitioningShuffleSourceTest.java | 137 +++ .../worker/ShuffleSinkFactoryTest.java | 187 +++ .../sdk/runners/worker/ShuffleSinkTest.java | 236 ++++ .../worker/ShuffleSourceFactoryTest.java | 230 ++++ .../runners/worker/SideInputUtilsTest.java | 145 +++ .../sdk/runners/worker/SinkFactoryTest.java | 119 ++ .../sdk/runners/worker/SourceFactoryTest.java | 124 ++ .../sdk/runners/worker/TestShuffleReader.java | 177 +++ .../runners/worker/TestShuffleReaderTest.java | 139 +++ .../sdk/runners/worker/TestShuffleWriter.java | 69 ++ .../runners/worker/TextSinkFactoryTest.java | 98 ++ .../sdk/runners/worker/TextSinkTest.java | 144 +++ .../runners/worker/TextSourceFactoryTest.java | 98 ++ .../sdk/runners/worker/TextSourceTest.java | 581 +++++++++ .../worker/UngroupedShuffleSourceTest.java | 112 ++ .../DataflowWorkerLoggingFormatterTest.java | 134 +++ .../DataflowWorkerLoggingInitializerTest.java | 109 ++ .../dataflow/sdk/testing/ExpectedLogs.java | 240 ++++ .../sdk/testing/ExpectedLogsTest.java | 102 ++ .../sdk/testing/FastNanoClockAndSleeper.java | 47 + .../testing/FastNanoClockAndSleeperTest.java | 47 + .../sdk/testing/ResetDateTimeProvider.java | 41 + .../testing/ResetDateTimeProviderTest.java | 55 + .../RestoreMappedDiagnosticContext.java | 47 + .../RestoreMappedDiagnosticContextTest.java | 51 + .../sdk/testing/RestoreSystemProperties.java | 51 + .../testing/RestoreSystemPropertiesTest.java | 50 + .../sdk/testing/TestPipelineTest.java | 76 ++ .../transforms/ApproximateQuantilesTest.java | 287 +++++ .../sdk/transforms/ApproximateUniqueTest.java | 302 +++++ .../dataflow/sdk/transforms/CombineTest.java | 527 +++++++++ .../dataflow/sdk/transforms/CountTest.java | 112 ++ .../dataflow/sdk/transforms/CreateTest.java | 189 +++ .../dataflow/sdk/transforms/FirstTest.java | 140 +++ .../dataflow/sdk/transforms/FlattenTest.java | 244 ++++ .../sdk/transforms/GroupByKeyTest.java | 280 +++++ .../dataflow/sdk/transforms/KeysTest.java | 100 ++ .../dataflow/sdk/transforms/KvSwapTest.java | 112 ++ .../dataflow/sdk/transforms/ParDoTest.java | 986 +++++++++++++++ .../sdk/transforms/PartitionTest.java | 141 +++ .../sdk/transforms/RateLimitingTest.java | 225 ++++ .../sdk/transforms/RemoveDuplicatesTest.java | 82 ++ .../dataflow/sdk/transforms/SampleTest.java | 175 +++ .../sdk/transforms/SimpleStatsFnsTest.java | 130 ++ .../dataflow/sdk/transforms/TopTest.java | 244 ++++ .../dataflow/sdk/transforms/ValuesTest.java | 103 ++ .../dataflow/sdk/transforms/ViewTest.java | 159 +++ .../dataflow/sdk/transforms/WithKeysTest.java | 122 ++ .../transforms/join/CoGbkResultCoderTest.java | 55 + .../sdk/transforms/join/CoGroupByKeyTest.java | 348 ++++++ .../sdk/transforms/join/UnionCoderTest.java | 48 + .../windowing/CalendarWindowsTest.java | 260 ++++ .../windowing/FixedWindowsTest.java | 114 ++ .../transforms/windowing/SessionsTest.java | 100 ++ .../windowing/SlidingWindowsTest.java | 127 ++ .../transforms/windowing/WindowingTest.java | 277 +++++ .../dataflow/sdk/util/AggregatorImplTest.java | 194 +++ .../AttemptBoundedExponentialBackOffTest.java | 71 ++ .../dataflow/sdk/util/Base64UtilsTest.java | 53 + .../dataflow/sdk/util/BigQueryUtilTest.java | 306 +++++ .../sdk/util/CloudMetricUtilsTest.java | 66 ++ .../sdk/util/CloudSourceUtilsTest.java | 83 ++ .../dataflow/sdk/util/CoderUtilsTest.java | 158 +++ .../cloud/dataflow/sdk/util/GcsUtilTest.java | 105 ++ .../sdk/util/GroupAlsoByWindowsDoFnTest.java | 231 ++++ .../dataflow/sdk/util/IOChannelUtilsTest.java | 76 ++ .../dataflow/sdk/util/IOFactoryTest.java | 99 ++ .../sdk/util/InstanceBuilderTest.java | 114 ++ .../dataflow/sdk/util/MonitoringUtilTest.java | 90 ++ .../cloud/dataflow/sdk/util/PTupleTest.java | 40 + .../dataflow/sdk/util/PackageUtilTest.java | 342 ++++++ .../util/RetryHttpRequestInitializerTest.java | 234 ++++ .../sdk/util/SerializableUtilsTest.java | 75 ++ .../dataflow/sdk/util/SerializerTest.java | 163 +++ .../StreamingGroupAlsoByWindowsDoFnTest.java | 282 +++++ .../dataflow/sdk/util/StringUtilsTest.java | 88 ++ .../cloud/dataflow/sdk/util/StructsTest.java | 177 +++ .../cloud/dataflow/sdk/util/TimeUtilTest.java | 73 ++ .../cloud/dataflow/sdk/util/VarIntTest.java | 281 +++++ .../dataflow/sdk/util/WindowedValueTest.java | 55 + .../sdk/util/common/CounterSetTest.java | 75 ++ .../dataflow/sdk/util/common/CounterTest.java | 743 ++++++++++++ .../sdk/util/common/CounterTestUtils.java | 123 ++ .../dataflow/sdk/util/common/MetricTest.java | 40 + .../BatchingShuffleEntryReaderTest.java | 138 +++ .../worker/CachingShuffleBatchReaderTest.java | 95 ++ .../util/common/worker/ExecutorTestUtils.java | 238 ++++ .../common/worker/FlattenOperationTest.java | 79 ++ .../common/worker/MapTaskExecutorTest.java | 290 +++++ .../common/worker/OutputReceiverTest.java | 135 +++ .../common/worker/ParDoOperationTest.java | 116 ++ .../PartialGroupByKeyOperationTest.java | 397 +++++++ .../util/common/worker/ReadOperationTest.java | 303 +++++ .../util/common/worker/ShuffleEntryTest.java | 145 +++ .../util/common/worker/StateSamplerTest.java | 139 +++ .../util/common/worker/WorkExecutorTest.java | 58 + .../common/worker/WriteOperationTest.java | 73 ++ .../dataflow/sdk/util/gcsfs/GcsPathTest.java | 334 ++++++ ...MediaHttpUploaderProgressListenerTest.java | 83 ++ .../cloud/dataflow/sdk/values/KVTest.java | 73 ++ .../sdk/values/PCollectionListTest.java | 47 + .../cloud/dataflow/sdk/values/PDoneTest.java | 98 ++ 486 files changed, 80297 insertions(+) create mode 100644 .gitignore create mode 100644 LICENSE create mode 100644 checkstyle.xml create mode 100644 examples/pom.xml create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/BigQueryTornadoes.java create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/DatastoreWordCount.java create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java create mode 100644 examples/src/test/java/com/google/cloud/dataflow/examples/BigQueryTornadoesTest.java create mode 100644 examples/src/test/java/com/google/cloud/dataflow/examples/TfIdfTest.java create mode 100644 examples/src/test/java/com/google/cloud/dataflow/examples/TopWikipediaSessionsTest.java create mode 100644 examples/src/test/java/com/google/cloud/dataflow/examples/WordCountTest.java create mode 100644 pom.xml create mode 100644 sdk/pom.xml create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/PipelineResult.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AtomicCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderException.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DefaultCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoderBase.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoderBase.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/URICoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIterator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ShardNameTemplate.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ApplicationNameOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BigQueryOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BlockingDataflowPipelineOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineShuffleOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerHarnessOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Default.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DefaultValueFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Description.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DirectPipelineOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Validation.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunner.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipeline.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerHooks.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipeline.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/RecordingPipelineVisitor.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformHierarchy.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformTreeNode.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BigQueryIOTranslator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DatastoreIOTranslator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ApplianceShuffleReader.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ApplianceShuffleWriter.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSink.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSource.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSink.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSource.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySource.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ByteArrayShufflePosition.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ChunkingShuffleBatchReader.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ChunkingShuffleEntryWriter.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CopyableSeekableByteChannel.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CustomSourceFormatFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedSource.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySource.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/LazyMultiSourceIterator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/OrderedCode.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSource.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleEntryWriter.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleLibrary.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleReader.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSink.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleWriter.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtils.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutorFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSink.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSource.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSource.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/RunnableOnService.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineOptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineRunner.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowingFnTestUtils.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Aggregator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/First.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Keys.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/KvSwap.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Mean.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/SerializableComparator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/SerializableFunction.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Values.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultSchema.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKey.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/KeyedPCollectionTuple.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/RawUnionValue.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/BoundedWindow.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/IntervalWindow.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowingFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowingFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowingFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AbstractWindowSet.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AggregatorImpl.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ApiErrorExtractor.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AppEngineEnvironment.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AssignWindowsDoFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOff.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Base64Utils.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableInserter.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableRowIterator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudCounterUtils.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudKnownType.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudMetricUtils.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudObject.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtils.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CoderUtils.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowReleaseInfo.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DirectModeExecutionContext.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnProcessContext.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ExecutionContext.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GCloudCredential.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsIOChannelFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/IOChannelFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/IOChannelUtils.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/InstanceBuilder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/MimeTypes.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/MonitoringUtil.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/OutputReference.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PTuple.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SerializableUtils.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Serializer.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ShardingWritableByteChannel.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StringUtils.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Structs.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TestCredential.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimeUtil.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerOrElement.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UserCodeException.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Values.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/VarInt.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowUtils.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Counter.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/CounterSet.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObservable.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObservableIterable.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObservableIterator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObserver.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ForwardingReiterator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Metric.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/PeekingReiterator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Reiterable.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Reiterator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/BatchingShuffleEntryReader.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CachingShuffleBatchReader.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CustomSourceFormat.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperation.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/GroupingShuffleEntryIterator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/KeyGroupedShuffleEntries.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Operation.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiver.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperation.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperation.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ProgressTracker.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ProgressTrackerGroup.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ProgressTrackingReiterator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Receiver.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReceivingOperation.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleBatchReader.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleEntry.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleEntryReader.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShufflePosition.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Sink.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Source.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSampler.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperation.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPath.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/ClientRequestHelper.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageExceptions.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageWriteChannel.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListener.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/StorageResourceId.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/package-info.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTag.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTagMap.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PBegin.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionList.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionView.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PDone.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PInput.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutput.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutputValueBase.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValue.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValueBase.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTagList.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypedPValue.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/values/package-info.java create mode 100644 sdk/src/main/resources/com/google/cloud/dataflow/sdk/sdk.properties create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/PipelineTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/TestUtils.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CustomCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/DefaultCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/InstantCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/IterableCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ListCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/MapCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/SerializableCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/URICoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BigQueryIOTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/io/user.avsc create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidatorTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunnerTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJobTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/TransformTreeTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSinkTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSourceTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CopyableSeekableByteChannelTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/OrderedCodeTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSourceFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtilsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TestShuffleReader.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TestShuffleReaderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TestShuffleWriter.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogs.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/FastNanoClockAndSleeper.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/FastNanoClockAndSleeperTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ResetDateTimeProvider.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ResetDateTimeProviderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContext.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContextTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreSystemProperties.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreSystemPropertiesTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CountTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CreateTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FirstTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FlattenTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KeysTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KvSwapTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicatesTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SimpleStatsFnsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ValuesTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ViewTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/WithKeysTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindowsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindowsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOffTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/Base64UtilsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BigQueryUtilTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CloudMetricUtilsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtilsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CoderUtilsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GcsUtilTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOChannelUtilsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/InstanceBuilderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/MonitoringUtilTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PTupleTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializerTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializableUtilsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializerTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StringUtilsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StructsTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TimeUtilTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/VarIntTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/WindowedValueTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterSetTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTestUtils.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/MetricTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/BatchingShuffleEntryReaderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/CachingShuffleBatchReaderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ExecutorTestUtils.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperationTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiverTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperationTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperationTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleEntryTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutorTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperationTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPathTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListenerTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/values/KVTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PCollectionListTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PDoneTest.java diff --git a/.gitignore b/.gitignore new file mode 100644 index 0000000000000..2f7896d1d1365 --- /dev/null +++ b/.gitignore @@ -0,0 +1 @@ +target/ diff --git a/LICENSE b/LICENSE new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/LICENSE @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/checkstyle.xml b/checkstyle.xml new file mode 100644 index 0000000000000..08df965ae6bb1 --- /dev/null +++ b/checkstyle.xml @@ -0,0 +1,385 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/examples/pom.xml b/examples/pom.xml new file mode 100644 index 0000000000000..fcb52fcdbf8d9 --- /dev/null +++ b/examples/pom.xml @@ -0,0 +1,223 @@ + + + + 4.0.0 + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-parent + manual_build + + + com.google.cloud.dataflow + google-cloud-dataflow-java-examples-all + Google Cloud Dataflow Java Examples - All + Google Cloud Dataflow Java SDK provides a simple, Java-based + interface for processing virtually any size data using Google cloud + resources. This artifact includes all Dataflow Java SDK + examples. + http://cloud.google.com/dataflow + + manual_build + + jar + + + + DataflowPipelineTests + + true + com.google.cloud.dataflow.sdk.testing.RunnableOnService + both + + + + + + + + maven-compiler-plugin + + + + org.apache.maven.plugins + maven-dependency-plugin + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.12 + + ../checkstyle.xml + true + true + true + + + + + check + + + + + + + + org.apache.maven.plugins + maven-source-plugin + 2.4 + + + attach-sources + compile + + jar + + + + attach-test-sources + test-compile + + test-jar + + + + + + + org.apache.felix + maven-bundle-plugin + 2.4.0 + true + + ${project.artifactId}-bundled-${project.version} + + + *;scope=compile|runtime;artifactId=!google-cloud-dataflow-java-sdk-all;inline=true + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + dataflow-examples-compile + compile + + jar + + + + dataflow-examples-test-compile + test-compile + + test-jar + + + + + + + + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-all + ${project.version} + + + + com.google.apis + google-api-services-storage + v1-rev11-1.19.0 + + + + com.google.apis + google-api-services-bigquery + v2-rev167-1.19.0 + + + + com.google.guava + guava-jdk5 + + + + + + com.google.http-client + google-http-client-jackson2 + 1.19.0 + + + + com.fasterxml.jackson.core + jackson-core + 2.4.2 + + + + com.fasterxml.jackson.core + jackson-annotations + 2.4.2 + + + + + org.slf4j + slf4j-api + 1.7.7 + + + + org.slf4j + slf4j-jdk14 + 1.7.7 + + + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-all + ${project.version} + test-jar + test + + + + org.hamcrest + hamcrest-all + 1.3 + test + + + + junit + junit + 4.11 + test + + + diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/BigQueryTornadoes.java b/examples/src/main/java/com/google/cloud/dataflow/examples/BigQueryTornadoes.java new file mode 100644 index 0000000000000..43e94c08633b0 --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/BigQueryTornadoes.java @@ -0,0 +1,149 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import java.util.ArrayList; +import java.util.List; + +/** + * An example that reads the public samples of weather data from BigQuery, counts the number of + * tornadoes that occur in each month, and writes the results to BigQuery. + */ +public class BigQueryTornadoes { + // Default to using a 1000 row subset of the public weather station table publicdata:samples.gsod. + private static final String WEATHER_SAMPLES_TABLE = + "clouddataflow-readonly:samples.weather_stations"; + + /** + * Examines each row in the input table. If a tornado was recorded in that sample, the month in + * which it occurred is output. + */ + static class ExtractTornadoesFn extends DoFn { + @Override + public void processElement(ProcessContext c){ + TableRow row = c.element(); + if ((Boolean) row.get("tornado")) { + c.output(Integer.parseInt((String) row.get("month"))); + } + } + } + + /** + * Prepares the data for writing to BigQuery by building a TableRow object containing an + * integer representation of month and the number of tornadoes that occurred in each month. + */ + static class FormatCountsFn extends DoFn, TableRow> { + @Override + public void processElement(ProcessContext c) { + TableRow row = new TableRow() + .set("month", c.element().getKey().intValue()) + .set("tornado_count", c.element().getValue().longValue()); + c.output(row); + } + } + + /** + * Takes rows from a table and generates a table of counts. + * + * The input schema is described by + * https://developers.google.com/bigquery/docs/dataset-gsod . + * The output contains the total number of tornadoes found in each month in + * the following schema: + *
    + *
  • month: integer
  • + *
  • tornado_count: integer
  • + *
+ */ + static class CountTornadoes + extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection rows) { + + // row... => month... + PCollection tornadoes = rows.apply( + ParDo.of(new ExtractTornadoesFn())); + + // month... => ... + PCollection> tornadoCounts = + tornadoes.apply(Count.perElement()); + + // ... => row... + PCollection results = tornadoCounts.apply( + ParDo.of(new FormatCountsFn())); + + return results; + } + } + + /** + * Options supported by {@link BigQueryTornadoes}. + *

+ * Inherits standard configuration options. + */ + private static interface Options extends PipelineOptions { + @Description("Table to read from, specified as " + + ":.") + @Default.String(WEATHER_SAMPLES_TABLE) + String getInput(); + void setInput(String value); + + @Description("Table to write to, specified as " + + ":.") + @Validation.Required + String getOutput(); + void setOutput(String value); + } + + public static void main(String[] args) { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + + Pipeline p = Pipeline.create(options); + + // Build the table schema for the output table. + List fields = new ArrayList<>(); + fields.add(new TableFieldSchema().setName("month").setType("INTEGER")); + fields.add(new TableFieldSchema().setName("tornado_count").setType("INTEGER")); + TableSchema schema = new TableSchema().setFields(fields); + + p.apply(BigQueryIO.Read.from(options.getInput())) + .apply(new CountTornadoes()) + .apply(BigQueryIO.Write + .to(options.getOutput()) + .withSchema(schema) + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE)); + + p.run(); + } +} diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/DatastoreWordCount.java b/examples/src/main/java/com/google/cloud/dataflow/examples/DatastoreWordCount.java new file mode 100644 index 0000000000000..1e00589281aad --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/DatastoreWordCount.java @@ -0,0 +1,198 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.datastore.DatastoreV1.Entity; +import com.google.api.services.datastore.DatastoreV1.Key; +import com.google.api.services.datastore.DatastoreV1.Property; +import com.google.api.services.datastore.DatastoreV1.Query; +import com.google.api.services.datastore.DatastoreV1.Value; +import com.google.api.services.datastore.client.DatastoreHelper; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.DatastoreIO; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; + +import java.util.Map; + +/** + * A WordCount example using DatastoreIO. + * + *

This example shows how to use DatastoreIO to read from Datastore and + * write the results to Cloud Storage. Note that this example will write + * data to Datastore, which may incur charge for Datastore operations. + * + *

To run this example, users need to set up the environment and use gcloud + * to get credential for Datastore: + *

+ * $ export CLOUDSDK_EXTRA_SCOPES=https://www.googleapis.com/auth/datastore
+ * $ gcloud auth login
+ * 
+ * + *

Note that the environment variable CLOUDSDK_EXTRA_SCOPES must be set + * to the same value when executing a Datastore pipeline, as the local auth + * cache is keyed by the requested scopes. + * + *

To run this pipeline locally, the following options must be provided: + *

{@code
+ *   --project=
+ *   --dataset=
+ *   --output=[ | gs://]
+ * }
+ * + *

To run this example using Dataflow service, you must additionally + * provide either {@literal --stagingLocation} or {@literal --tempLocation}, and + * select one of the Dataflow pipeline runners, eg + * {@literal --runner=BlockingDataflowPipelineRunner}. + */ +public class DatastoreWordCount { + + /** + * A DoFn that gets the content of an entity (one line in a + * Shakespeare play) and converts it to a string. + */ + static class GetContentFn extends DoFn { + @Override + public void processElement(ProcessContext c) { + Map props = DatastoreHelper.getPropertyMap(c.element()); + c.output(DatastoreHelper.getString(props.get("content"))); + } + } + + /** + * A DoFn that creates entity for every line in Shakespeare. + */ + static class CreateEntityFn extends DoFn { + private String kind; + + CreateEntityFn(String kind) { + this.kind = kind; + } + + public Entity makeEntity(String content) { + Entity.Builder entityBuilder = Entity.newBuilder(); + // Create entities with same ancestor Key. + Key ancestorKey = DatastoreHelper.makeKey(kind, "root").build(); + Key key = DatastoreHelper.makeKey(ancestorKey, kind).build(); + entityBuilder.setKey(key); + entityBuilder.addProperty(Property.newBuilder() + .setName("content") + .setValue(Value.newBuilder().setStringValue(content))); + return entityBuilder.build(); + } + + @Override + public void processElement(ProcessContext c) { + c.output(makeEntity(c.element())); + } + } + + /** + * Options supported by {@link DatastoreWordCount}. + *

+ * Inherits standard configuration options. + */ + private static interface Options extends PipelineOptions { + @Description("Path of the file to read from and store to Datastore") + @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") + String getInput(); + void setInput(String value); + + @Description("Path of the file to write to") + @Validation.Required + String getOutput(); + void setOutput(String value); + + @Description("Dataset ID to read from datastore") + @Validation.Required + String getDataset(); + void setDataset(String value); + + @Description("Dataset entity kind") + @Default.String("shakespeare-demo") + String getKind(); + void setKind(String value); + + @Description("Read an existing dataset, do not write first") + boolean isReadOnly(); + void setReadOnly(boolean value); + } + + /** + * An example which creates a pipeline to populate DatastoreIO from a + * text input. Forces use of DirectPipelineRunner for local execution mode. + */ + public static void writeDataToDatastore(Options options) { + // Runs locally via DirectPiplineRunner, as writing is not yet implemented + // for the other runners which is why we just create a PipelineOptions with defaults. + Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); + p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) + .apply(ParDo.of(new CreateEntityFn(options.getKind()))) + .apply(DatastoreIO.Write.to(options.getDataset())); + + p.run(); + } + + /** + * An example which creates a pipeline to do DatastoreIO.Read from Datastore. + */ + public static void readDataFromDatastore(Options options) { + // Build a query: read all entities of the specified kind. + Query.Builder q = Query.newBuilder(); + q.addKindBuilder().setName(options.getKind()); + Query query = q.build(); + + Pipeline p = Pipeline.create(options); + p.apply(DatastoreIO.Read.named("ReadShakespeareFromDatastore") + .from(options.getDataset(), query)) + .apply(ParDo.of(new GetContentFn())) + .apply(new WordCount.CountWords()) + .apply(TextIO.Write.named("WriteLines").to(options.getOutput())); + + p.run(); + } + + /** + * Main function. + * An example to demo how to use DatastoreIO. The runner here is + * customizable, which means users could pass either DirectPipelineRunner + * or DataflowPipelineRunner in PipelineOptions. + */ + public static void main(String args[]) { + // The options are used in two places, for Dataflow service, and + // building DatastoreIO.Read object + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + + if (!options.isReadOnly()) { + // First example: write data to Datastore for reading later. + // Note: this will insert new entries with the given kind. Existing entries + // should be cleared first, or the final counts will contain duplicates. + // The Datastore Admin tool in the AppEngine console can be used to erase + // all entries with a particular kind. + DatastoreWordCount.writeDataToDatastore(options); + } + + // Second example: do parallel read from Datastore. + DatastoreWordCount.readDataFromDatastore(options); + } +} diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java b/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java new file mode 100644 index 0000000000000..a6bd4f27fd61d --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java @@ -0,0 +1,425 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.URICoder; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.GcsOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Flatten; +import com.google.cloud.dataflow.sdk.transforms.Keys; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates; +import com.google.cloud.dataflow.sdk.transforms.Values; +import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.transforms.WithKeys; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; +import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; +import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; +import com.google.cloud.dataflow.sdk.util.GcsUtil; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashSet; +import java.util.Set; + +/** + * An example that computes a basic TF-IDF search table for a directory or GCS prefix. + * + *

Command-line usage for this example: + * + *

+ *     java com.google.cloud.dataflow.examples.TfIdf \
+ *       --runner= \
+ *       --input= \
+ *       --output=
+ * 
+ * + *

For example, to execute this pipeline locally to index a local directory: + * + *

+ *     java com.google.cloud.dataflow.examples.TfIdf \
+ *       --runner=DirectPipelineRunner \
+ *       --input= \
+ *       --output=
+ * 
+ * + *

To execute this pipeline using the Dataflow service + * to index the works of Shakespeare and write the results to a GCS bucket: + * (For execution via the Dataflow service, only GCS locations are supported) + * + *

+ *     java com.google.cloud.dataflow.examples.TfIdf \
+ *       --project= \
+ *       --stagingLocation=gs:// \
+ *       --runner=BlockingDataflowPipelineRunner \
+ *       [--input=gs://] \
+ *       --output=gs://
+ * 
+ * + *

The default input is gs://dataflow-samples/shakespeare/ + */ +public class TfIdf { + /** + * Options supported by {@link TfIdf}. + *

+ * Inherits standard configuration options. + */ + private static interface Options extends PipelineOptions { + @Description("Path to the directory or GCS prefix containing files to read from") + @Default.String("gs://dataflow-samples/shakespeare/") + String getInput(); + void setInput(String value); + + @Description("Prefix of output URI to write to") + @Validation.Required + String getOutput(); + void setOutput(String value); + } + + /** + * Lists documents contained beneath the {@code options.input} prefix/directory. + */ + public static Set listInputDocuments(Options options) + throws URISyntaxException, IOException { + URI baseUri = new URI(options.getInput()); + + // List all documents in the directory or GCS prefix. + URI absoluteUri; + if (baseUri.getScheme() != null) { + absoluteUri = baseUri; + } else { + absoluteUri = new URI( + "file", + baseUri.getAuthority(), + baseUri.getPath(), + baseUri.getQuery(), + baseUri.getFragment()); + } + + Set uris = new HashSet<>(); + if (absoluteUri.getScheme().equals("file")) { + File directory = new File(absoluteUri); + for (String entry : directory.list()) { + File path = new File(directory, entry); + uris.add(path.toURI()); + } + } else if (absoluteUri.getScheme().equals("gs")) { + GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil(); + URI gcsUriGlob = new URI( + absoluteUri.getScheme(), + absoluteUri.getAuthority(), + absoluteUri.getPath() + "*", + absoluteUri.getQuery(), + absoluteUri.getFragment()); + for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) { + uris.add(entry.toUri()); + } + } + + return uris; + } + + /** + * Reads the documents at the provided uris and returns all lines + * from the documents tagged with which document they are from. + */ + public static class ReadDocuments + extends PTransform>> { + + private Iterable uris; + + public ReadDocuments(Iterable uris) { + this.uris = uris; + } + + @Override + public Coder getDefaultOutputCoder() { + return KvCoder.of(URICoder.of(), StringUtf8Coder.of()); + } + + @Override + public PCollection> apply(PInput input) { + Pipeline pipeline = getPipeline(); + + // Create one TextIO.Read transform for each document + // and add its output to a PCollectionList + PCollectionList> urisToLines = + PCollectionList.empty(pipeline); + + // TextIO.Read supports: + // - file: URIs and paths locally + // - gs: URIs on the service + for (final URI uri : uris) { + String uriString; + if (uri.getScheme().equals("file")) { + uriString = new File(uri).getPath(); + } else { + uriString = uri.toString(); + } + + PCollection> oneUriToLines = pipeline + .apply(TextIO.Read.from(uriString) + .named("TextIO.Read(" + uriString + ")")) + .apply(WithKeys.of(uri)); + + urisToLines = urisToLines.and(oneUriToLines); + } + + return urisToLines.apply(Flatten.>create()); + } + } + + /** + * A transform containing a basic TF-IDF pipeline. The input consists of KV objects + * where the key is the document's URI and the value is a piece + * of the document's content. The output is mapping from terms to + * scores for each document URI. + */ + public static class ComputeTfIdf + extends PTransform>, PCollection>>> { + + public ComputeTfIdf() { } + + @Override + public PCollection>> apply( + PCollection> uriToContent) { + + // Compute the total number of documents, and + // prepare this singleton PCollectionView for + // use as a side input. + final PCollectionView totalDocuments = + uriToContent + .apply(Keys.create()) + .apply(RemoveDuplicates.create()) + .apply(Count.globally()) + .apply(View.asSingleton()); + + // Create a collection of pairs mapping a URI to each + // of the words in the document associated with that that URI. + PCollection> uriToWords = uriToContent + .apply(ParDo.named("SplitWords").of( + new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) { + URI uri = c.element().getKey(); + String line = c.element().getValue(); + for (String word : line.split("\\W+")) { + if (!word.isEmpty()) { + c.output(KV.of(uri, word.toLowerCase())); + } + } + } + })); + + // Compute a mapping from each word to the total + // number of documents in which it appears. + PCollection> wordToDocCount = uriToWords + .apply(RemoveDuplicates.>create()) + .apply(Values.create()) + .apply(Count.perElement()); + + // Compute a mapping from each URI to the total + // number of words in the document associated with that URI. + PCollection> uriToWordTotal = uriToWords + .apply(Keys.create()) + .apply(Count.perElement()); + + // Count, for each (URI, word) pair, the number of + // occurrences of that word in the document associated + // with the URI. + PCollection, Long>> uriAndWordToCount = uriToWords + .apply(Count.>perElement()); + + // Adjust the above collection to a mapping from + // (URI, word) pairs to counts into an isomorphic mapping + // from URI to (word, count) pairs, to prepare for a join + // by the URI key. + PCollection>> uriToWordAndCount = uriAndWordToCount + .apply(ParDo.of(new DoFn, Long>, KV>>() { + @Override + public void processElement(ProcessContext c) { + URI uri = c.element().getKey().getKey(); + String word = c.element().getKey().getValue(); + Long occurrences = c.element().getValue(); + c.output(KV.of(uri, KV.of(word, occurrences))); + } + })); + + // Prepare to join the mapping of URI to (word, count) pairs with + // the mapping of URI to total word counts, by associating + // each of the input PCollection> with + // a tuple tag. Each input must have the same key type, URI + // in this case. The type parameter of the tuple tag matches + // the types of the values for each collection. + final TupleTag wordTotalsTag = new TupleTag(); + final TupleTag> wordCountsTag = new TupleTag>(); + KeyedPCollectionTuple coGbkInput = KeyedPCollectionTuple + .of(wordTotalsTag, uriToWordTotal) + .and(wordCountsTag, uriToWordAndCount); + + // Perform a CoGroupByKey (a sort of pre-join) on the prepared + // inputs. This yields a mapping from URI to a CoGbkResult + // (CoGroupByKey Result). The CoGbkResult is a mapping + // from the above tuple tags to the values in each input + // associated with a particular URI. In this case, each + // KV group a URI with the total number of + // words in that document as well as all the (word, count) + // pairs for particular words. + PCollection> uriToWordAndCountAndTotal = coGbkInput + .apply(CoGroupByKey.create().withName("CoGroupByURI")); + + // Compute a mapping from each word to a (URI, term frequency) + // pair for each URI. A word's term frequency for a document + // is simply the number of times that word occurs in the document + // divided by the total number of words in the document. + PCollection>> wordToUriAndTf = uriToWordAndCountAndTotal + .apply(ParDo.of(new DoFn, KV>>() { + @Override + public void processElement(ProcessContext c) { + URI uri = c.element().getKey(); + Long wordTotal = c.element().getValue().getOnly(wordTotalsTag); + + for (KV wordAndCount : c.element().getValue().getAll(wordCountsTag)) { + String word = wordAndCount.getKey(); + Long wordCount = wordAndCount.getValue(); + Double termFrequency = wordCount.doubleValue() / wordTotal.doubleValue(); + c.output(KV.of(word, KV.of(uri, termFrequency))); + } + } + })); + + // Compute a mapping from each word to its document frequency. + // A word's document frequency in a corpus is the number of + // documents in which the word appears divided by the total + // number of documents in the corpus. Note how the total number of + // documents is passed as a side input; the same value is + // presented to each invocation of the DoFn. + PCollection> wordToDf = wordToDocCount + .apply(ParDo + .withSideInputs(totalDocuments) + .of(new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) { + String word = c.element().getKey(); + Long documentCount = c.element().getValue(); + Long documentTotal = c.sideInput(totalDocuments); + Double documentFrequency = documentCount.doubleValue() + / documentTotal.doubleValue(); + + c.output(KV.of(word, documentFrequency)); + } + })); + + // Join the term frequency and document frequency + // collections, each keyed on the word. + final TupleTag> tfTag = new TupleTag>(); + final TupleTag dfTag = new TupleTag(); + PCollection> wordToUriAndTfAndDf = KeyedPCollectionTuple + .of(tfTag, wordToUriAndTf) + .and(dfTag, wordToDf) + .apply(CoGroupByKey.create()); + + // Compute a mapping from each word to a (URI, TF-IDF) score + // for each URI. There are a variety of definitions of TF-IDF + // ("term frequency - inverse document frequency") score; + // here we use a basic version which is the term frequency + // divided by the log of the document frequency. + PCollection>> wordToUriAndTfIdf = wordToUriAndTfAndDf + .apply(ParDo.of(new DoFn, KV>>() { + @Override + public void processElement(ProcessContext c) { + String word = c.element().getKey(); + Double df = c.element().getValue().getOnly(dfTag); + + for (KV uriAndTf : c.element().getValue().getAll(tfTag)) { + URI uri = uriAndTf.getKey(); + Double tf = uriAndTf.getValue(); + Double tfIdf = tf * Math.log(1 / df); + c.output(KV.of(word, KV.of(uri, tfIdf))); + } + } + })); + + return wordToUriAndTfIdf; + } + } + + /** + * A {@link PTransform} to write, in CSV format, a mapping from term and URI + * to score. + */ + public static class WriteTfIdf + extends PTransform>>, PDone> { + + private String output; + + public WriteTfIdf(String output) { + this.output = output; + } + + @Override + public PDone apply(PCollection>> wordToUriAndTfIdf) { + return wordToUriAndTfIdf + .apply(ParDo.of(new DoFn>, String>() { + @Override + public void processElement(ProcessContext c) { + c.output(String.format("%s,\t%s,\t%f", + c.element().getKey(), + c.element().getValue().getKey(), + c.element().getValue().getValue())); + } + })) + .apply(TextIO.Write + .to(output) + .withSuffix(".csv")); + } + } + + public static void main(String[] args) throws Exception { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + Pipeline pipeline = Pipeline.create(options); + + pipeline + .apply(new ReadDocuments(listInputDocuments(options))) + .apply(new ComputeTfIdf()) + .apply(new WriteTfIdf(options.getOutput())); + + pipeline.run(); + } +} diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java b/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java new file mode 100644 index 0000000000000..baa520ea0447f --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java @@ -0,0 +1,208 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SerializableComparator; +import com.google.cloud.dataflow.sdk.transforms.Top; +import com.google.cloud.dataflow.sdk.transforms.windowing.CalendarWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.util.List; + +/** + * Pipeline that reads Wikipedia edit data from BigQuery and computes the user with + * the longest string of edits separated by no more than an hour within each month. + * + *

This pipeline demonstrates how the Windowing API can be used to perform + * various time-based aggregations of data. + * + *

To run this pipeline, the following options must be provided: + *

{@code
+ *   --project=
+ *   --output=gs://
+ *   --stagingLocation=gs://
+ *   --runner=(Blocking)DataflowPipelineRunner
+ * }
+ * + *

To run this example using Dataflow service, you must additionally + * provide either {@literal --stagingLocation} or {@literal --tempLocation}, and + * select one of the Dataflow pipeline runners, eg + * {@literal --runner=BlockingDataflowPipelineRunner}. + */ +public class TopWikipediaSessions { + private static final String EXPORTED_WIKI_TABLE = "gs://dataflow-samples/wikipedia_edits/*.json"; + + /** + * Extracts user and timestamp from a TableRow representing a Wikipedia edit + */ + static class ExtractUserAndTimestamp extends DoFn { + @Override + public void processElement(ProcessContext c) { + TableRow row = c.element(); + int timestamp = (Integer) row.get("timestamp"); + String userName = (String) row.get("contributor_username"); + if (userName != null) { + // Sets the implicit timestamp field to be used in windowing. + c.outputWithTimestamp(userName, new Instant(timestamp * 1000L)); + } + } + } + + /** + * Computes the number of edits in each user session. A session is defined as + * a string of edits where each is separated from the next by less than an hour. + */ + static class ComputeSessions + extends PTransform, PCollection>> { + @Override + public PCollection> apply(PCollection actions) { + return actions + .apply(Window.into(Sessions.withGapDuration(Duration.standardHours(1)))) + + .apply(Count.perElement()); + } + } + + /** + * Computes the longest session ending in each month. + */ + private static class TopPerMonth + extends PTransform>, PCollection>>> { + @Override + public PCollection>> apply(PCollection> sessions) { + return sessions + .apply(Window.>into(CalendarWindows.months(1))) + + .apply(Top.of(1, new SerializableComparator>() { + @Override + public int compare(KV o1, KV o2) { + return Long.compare(o1.getValue(), o2.getValue()); + } + })); + } + } + + static class ComputeTopSessions extends PTransform, PCollection> { + private final double samplingThreshold; + + public ComputeTopSessions(double samplingThreshold) { + this.samplingThreshold = samplingThreshold; + } + + @Override + public PCollection apply(PCollection input) { + return input + .apply(ParDo.of(new ExtractUserAndTimestamp())) + + .apply(ParDo.named("SampleUsers").of( + new DoFn() { + @Override + public void processElement(ProcessContext c) { + if (Math.abs(c.element().hashCode()) <= Integer.MAX_VALUE * samplingThreshold) { + c.output(c.element()); + } + } + })) + + .apply(new ComputeSessions()) + + .apply(ParDo.named("SessionsToStrings").of( + new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) { + c.output(KV.of( + c.element().getKey() + " : " + + c.windows().iterator().next(), c.element().getValue())); + } + })) + + .apply(new TopPerMonth()) + + .apply(ParDo.named("FormatOutput").of( + new DoFn>, String>() { + @Override + public void processElement(ProcessContext c) { + for (KV item : c.element()) { + String session = item.getKey(); + long count = item.getValue(); + c.output( + session + " : " + count + " : " + + ((IntervalWindow) c.windows().iterator().next()).start()); + } + } + })); + } + } + + /** + * Options supported by this class. + * + *

Inherits standard Dataflow configuration options. + */ + private static interface Options extends PipelineOptions { + @Description( + "Input specified as a GCS path containing a BigQuery table exported as json") + @Default.String(EXPORTED_WIKI_TABLE) + String getInput(); + void setInput(String value); + + @Description("File to output results to") + @Validation.Required + String getOutput(); + void setOutput(String value); + } + + public static void main(String[] args) { + Options options = PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(Options.class); + DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); + + Pipeline p = Pipeline.create(dataflowOptions); + + double samplingThreshold = 0.1; + + p.apply(TextIO.Read + .from(options.getInput()) + .withCoder(TableRowJsonCoder.of())) + .apply(new ComputeTopSessions(samplingThreshold)) + .apply(TextIO.Write.named("Write").withoutSharding().to(options.getOutput())); + + p.run(); + } +} diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java b/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java new file mode 100644 index 0000000000000..96893b909bc7a --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java @@ -0,0 +1,174 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.DefaultValueFactory; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +/** + * An example that counts words in Shakespeare. For a detailed walkthrough of this + * example see: + * https://developers.google.com/cloud-dataflow/java-sdk/wordcount-example + * + * To execute this pipeline locally, specify general pipeline configuration: + * --project= + * and example configuration: + * --output=[ | gs://] + * + * To execute this pipeline using the Dataflow service, specify pipeline configuration: + * --project= --stagingLocation=gs:// + * --runner=BlockingDataflowPipelineRunner + * and example configuration: + * --output=gs:// + * + * The input file defaults to gs://dataflow-samples/shakespeare/kinglear.txt and can be + * overridden with --input. + */ +public class WordCount { + + /** A DoFn that tokenizes lines of text into individual words. */ + static class ExtractWordsFn extends DoFn { + private Aggregator emptyLines; + + @Override + public void startBundle(Context c) { + emptyLines = c.createAggregator("emptyLines", new Sum.SumLongFn()); + } + + @Override + public void processElement(ProcessContext c) { + // Split the line into words. + String[] words = c.element().split("[^a-zA-Z']+"); + + // Keep track of the number of lines without any words encountered while tokenizing. + // This aggregator is visible in the monitoring UI when run using DataflowPipelineRunner. + if (words.length == 0) { + emptyLines.addValue(1L); + } + + // Output each word encountered into the output PCollection. + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + /** A DoFn that converts a Word and Count into a printable string. */ + static class FormatCountsFn extends DoFn, String> { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().getKey() + ": " + c.element().getValue()); + } + } + + /** + * A PTransform that converts a PCollection containing lines of text into a PCollection of + * formatted word counts. + *

+ * Although this pipeline fragment could be inlined, bundling it as a PTransform allows for easy + * reuse, modular testing, and an improved monitoring experience. + */ + public static class CountWords extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection lines) { + + // Convert lines of text into individual words. + PCollection words = lines.apply( + ParDo.of(new ExtractWordsFn())); + + // Count the number of times each word occurs. + PCollection> wordCounts = + words.apply(Count.perElement()); + + // Format each word and count into a printable string. + PCollection results = wordCounts.apply( + ParDo.of(new FormatCountsFn())); + + return results; + } + } + + /** + * Options supported by {@link WordCount}. + *

+ * Inherits standard configuration options. + */ + public static interface Options extends PipelineOptions { + @Description("Path of the file to read from") + @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") + String getInput(); + void setInput(String value); + + @Description("Path of the file to write to") + @Default.InstanceFactory(OutputFactory.class) + String getOutput(); + void setOutput(String value); + + /** Returns gs://${STAGING_LOCATION}/"counts.txt" */ + public static class OutputFactory implements DefaultValueFactory { + @Override + public String create(PipelineOptions options) { + DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); + if (dataflowOptions.getStagingLocation() != null) { + return GcsPath.fromUri(dataflowOptions.getStagingLocation()) + .resolve("counts.txt").toString(); + } else { + throw new IllegalArgumentException("Must specify --output or --stagingLocation"); + } + } + } + + /** + * By default (numShards == 0), the system will choose the shard count. + * Most programs will not need this option. + */ + @Description("Number of output shards (0 if the system should choose automatically)") + int getNumShards(); + void setNumShards(int value); + } + + public static void main(String[] args) { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + Pipeline p = Pipeline.create(options); + + p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) + .apply(new CountWords()) + .apply(TextIO.Write.named("WriteCounts") + .to(options.getOutput()) + .withNumShards(options.getNumShards())); + + p.run(); + } +} + diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/BigQueryTornadoesTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/BigQueryTornadoesTest.java new file mode 100644 index 0000000000000..6dafef7036481 --- /dev/null +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/BigQueryTornadoesTest.java @@ -0,0 +1,80 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.dataflow.examples.BigQueryTornadoes.ExtractTornadoesFn; +import com.google.cloud.dataflow.examples.BigQueryTornadoes.FormatCountsFn; +import com.google.cloud.dataflow.sdk.transforms.DoFnTester; +import com.google.cloud.dataflow.sdk.values.KV; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.List; + +/** + * Test case for {@link BigQueryTornadoes}. + */ +@RunWith(JUnit4.class) +public class BigQueryTornadoesTest { + + @Test + public void testExtractTornadoes() throws Exception { + TableRow row = new TableRow() + .set("month", "6") + .set("tornado", true); + DoFnTester extractWordsFn = + DoFnTester.of(new ExtractTornadoesFn()); + Assert.assertThat(extractWordsFn.processBatch(row), + CoreMatchers.hasItems(6)); + } + + @Test + public void testNoTornadoes() throws Exception { + TableRow row = new TableRow() + .set("month", 6) + .set("tornado", false); + DoFnTester extractWordsFn = + DoFnTester.of(new ExtractTornadoesFn()); + Assert.assertTrue(extractWordsFn.processBatch(row).isEmpty()); + } + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testFormatCounts() throws Exception { + DoFnTester, TableRow> formatCountsFn = + DoFnTester.of(new FormatCountsFn()); + KV empty[] = {}; + List results = formatCountsFn.processBatch(empty); + Assert.assertTrue(results.size() == 0); + KV input[] = { KV.of(3, 0L), + KV.of(4, Long.MAX_VALUE), + KV.of(5, Long.MIN_VALUE) }; + results = formatCountsFn.processBatch(input); + Assert.assertEquals(results.size(), 3); + Assert.assertEquals(results.get(0).get("month"), 3); + Assert.assertEquals(results.get(0).get("tornado_count"), 0L); + Assert.assertEquals(results.get(1).get("month"), 4); + Assert.assertEquals(results.get(1).get("tornado_count"), Long.MAX_VALUE); + Assert.assertEquals(results.get(2).get("month"), 5); + Assert.assertEquals(results.get(2).get("tornado_count"), Long.MIN_VALUE); + } +} diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/TfIdfTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/TfIdfTest.java new file mode 100644 index 0000000000000..341fd80c25b2b --- /dev/null +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/TfIdfTest.java @@ -0,0 +1,63 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.Keys; +import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.net.URI; +import java.util.Arrays; + +/** + * Tests of TfIdf + */ +@RunWith(JUnit4.class) +public class TfIdfTest { + + /** Test that the example runs */ + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testTfIdf() throws Exception { + Pipeline pipeline = TestPipeline.create(); + + PCollection>> wordToUriAndTfIdf = pipeline + .apply(Create.of( + KV.of(new URI("x"), "a b c d"), + KV.of(new URI("y"), "a b c"), + KV.of(new URI("z"), "a m n"))) + .apply(new TfIdf.ComputeTfIdf()); + + PCollection words = wordToUriAndTfIdf + .apply(Keys.create()) + .apply(RemoveDuplicates.create()); + + DataflowAssert.that(words).containsInAnyOrder(Arrays.asList("a", "m", "n", "b", "c", "d")); + + pipeline.run(); + } +} diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/TopWikipediaSessionsTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/TopWikipediaSessionsTest.java new file mode 100644 index 0000000000000..ce43ae9930a4b --- /dev/null +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/TopWikipediaSessionsTest.java @@ -0,0 +1,62 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; + +/** Unit tests for {@link TopWikipediaSessions}. */ +@RunWith(JUnit4.class) +public class TopWikipediaSessionsTest { + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testComputeTopUsers() { + Pipeline p = TestPipeline.create(); + + PCollection output = + p.apply(Create.of(Arrays.asList( + new TableRow().set("timestamp", 0).set("contributor_username", "user1"), + new TableRow().set("timestamp", 1).set("contributor_username", "user1"), + new TableRow().set("timestamp", 2).set("contributor_username", "user1"), + new TableRow().set("timestamp", 0).set("contributor_username", "user2"), + new TableRow().set("timestamp", 1).set("contributor_username", "user2"), + new TableRow().set("timestamp", 3601).set("contributor_username", "user2"), + new TableRow().set("timestamp", 3602).set("contributor_username", "user2"), + new TableRow().set("timestamp", 35 * 24 * 3600).set("contributor_username", "user3")))) + .apply(new TopWikipediaSessions.ComputeTopSessions(1.0)); + + DataflowAssert.that(output).containsInAnyOrder(Arrays.asList( + "user1 : [1970-01-01T00:00:00.000Z..1970-01-01T01:00:02.000Z)" + + " : 3 : 1970-01-01T00:00:00.000Z", + "user3 : [1970-02-05T00:00:00.000Z..1970-02-05T01:00:00.000Z)" + + " : 1 : 1970-02-01T00:00:00.000Z")); + + p.run(); + } +} diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/WordCountTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/WordCountTest.java new file mode 100644 index 0000000000000..36efec738ddc5 --- /dev/null +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/WordCountTest.java @@ -0,0 +1,81 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.cloud.dataflow.examples.WordCount.CountWords; +import com.google.cloud.dataflow.examples.WordCount.ExtractWordsFn; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFnTester; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Tests of WordCount. + */ +@RunWith(JUnit4.class) +public class WordCountTest { + + /** Example test that tests a specific DoFn. */ + @Test + public void testExtractWordsFn() { + DoFnTester extractWordsFn = + DoFnTester.of(new ExtractWordsFn()); + + Assert.assertThat(extractWordsFn.processBatch(" some input words "), + CoreMatchers.hasItems("some", "input", "words")); + Assert.assertThat(extractWordsFn.processBatch(" "), + CoreMatchers.hasItems()); + Assert.assertThat(extractWordsFn.processBatch(" some ", " input", " words"), + CoreMatchers.hasItems("some", "input", "words")); + } + + static final String[] WORDS_ARRAY = new String[] { + "hi there", "hi", "hi sue bob", + "hi sue", "", "bob hi"}; + + static final List WORDS = Arrays.asList(WORDS_ARRAY); + + static final String[] COUNTS_ARRAY = new String[] { + "hi: 5", "there: 1", "sue: 2", "bob: 2"}; + + /** Example test that tests a PTransform by using an in-memory input and inspecting the output. */ + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testCountWords() throws Exception { + Pipeline p = TestPipeline.create(); + + PCollection input = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); + + PCollection output = input.apply(new CountWords()); + + DataflowAssert.that(output).containsInAnyOrder(COUNTS_ARRAY); + p.run(); + } +} diff --git a/pom.xml b/pom.xml new file mode 100644 index 0000000000000..fd5b04376e43e --- /dev/null +++ b/pom.xml @@ -0,0 +1,202 @@ + + + + 4.0.0 + + + com.google + google + 5 + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-parent + Google Cloud Dataflow Java SDK - Parent + Google Cloud Dataflow Java SDK provides a simple, Java-based + interface for processing virtually any size data using Google cloud + resources. This artifact includes the parent POM for other Dataflow + artifacts. + http://cloud.google.com/dataflow + 2013 + + manual_build + + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + + Google Inc. + http://www.google.com + + + + + scm:git:git@github.com:GoogleCloudPlatform/DataflowJavaSDK.git + scm:git:git@github.com:GoogleCloudPlatform/DataflowJavaSDK.git + git@github.com:GoogleCloudPlatform/DataflowJavaSDK.git + + + + 3.0.3 + + + + UTF-8 + + + pom + + sdk + examples + + + + + + + maven-compiler-plugin + 3.1 + + 1.7 + 1.7 + -Xlint:all + true + true + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.5 + + + + org.codehaus.mojo + versions-maven-plugin + 2.1 + + + + org.codehaus.mojo + exec-maven-plugin + 1.1 + + + verify + + java + + + + + + + java.util.logging.config.file + logging.properties + + + + + + + org.apache.felix + maven-bundle-plugin + 2.4.0 + + + + + org.jacoco + jacoco-maven-plugin + 0.7.1.201405082137 + + + + prepare-agent + + + file + true + + + + report + prepare-package + + report + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.15 + + ${testParallelValue} + 4 + + ${project.build.directory}/${project.artifactId}-${project.version}.jar + ${project.build.directory}/${project.artifactId}-${project.version}-tests.jar + + ${testGroups} + + ${runIntegrationTestOnService} + ${dataflowProjectName} + + false + + + + org.apache.maven.surefire + surefire-junit47 + 2.7.2 + + + + + + + + + + + org.codehaus.mojo + versions-maven-plugin + 2.1 + + + + dependency-updates-report + plugin-updates-report + + + + + + + diff --git a/sdk/pom.xml b/sdk/pom.xml new file mode 100644 index 0000000000000..93a8f277a8370 --- /dev/null +++ b/sdk/pom.xml @@ -0,0 +1,315 @@ + + + + 4.0.0 + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-parent + manual_build + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-all + Google Cloud Dataflow Java SDK - All + Google Cloud Dataflow Java SDK provides a simple, Java-based + interface for processing virtually any size data using Google cloud + resources. This artifact includes entire Dataflow Java SDK. + http://cloud.google.com/dataflow + + manual_build + + jar + + + ${maven.build.timestamp} + yyyy-MM-dd HH:mm + com.google.cloud.dataflow + false + none + + + + + + + DataflowPipelineTests + + true + com.google.cloud.dataflow.sdk.testing.RunnableOnService + both + + + + + + + + src/main/resources + true + + + + + + maven-compiler-plugin + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.12 + + ../checkstyle.xml + true + true + false + true + + + + + check + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + dataflow-sdk-compile + compile + + jar + + + + dataflow-sdk-test-compile + test-compile + + test-jar + + + + + + + + org.apache.maven.plugins + maven-source-plugin + 2.4 + + + attach-sources + compile + + jar + + + + attach-test-sources + test-compile + + test-jar + + + + + + + org.apache.felix + maven-bundle-plugin + true + + ${project.artifactId}-bundled-${project.version} + + + !${dataflow}.sdk.runners.worker.*, + !${dataflow}.sdk.streaming.*, + !${dataflow}.sdk.util.gcsio, + ${dataflow}.* + + true + *;scope=compile|runtime;inline=true + + + + + + + org.jacoco + jacoco-maven-plugin + + + + + org.apache.avro + avro-maven-plugin + 1.7.7 + + + schemas + generate-test-sources + + schema + + + ${project.basedir}/src/test/ + ${project.build.directory}/generated-test-sources/java + + + + + + + + + + com.google.apis + google-api-services-dataflow + v1beta3-rev1-1.19.0 + + + + com.google.guava + guava-jdk5 + + + + + + com.google.apis + google-api-services-bigquery + v2-rev167-1.19.0 + + + + com.google.apis + google-api-services-compute + v1-rev34-1.19.0 + + + + com.google.apis + google-api-services-pubsub + v1beta1-rev9-1.19.0 + + + + com.google.apis + google-api-services-storage + v1-rev11-1.19.0 + + + + com.google.http-client + google-http-client-jackson2 + 1.19.0 + + + + com.google.oauth-client + google-oauth-client-java6 + 1.19.0 + + + + com.google.apis + google-api-services-datastore-protobuf + v1beta2-rev1-2.1.0 + + + + com.google.guava + guava + 18.0 + + + + com.fasterxml.jackson.core + jackson-core + 2.4.2 + + + + com.fasterxml.jackson.core + jackson-annotations + 2.4.2 + + + + com.fasterxml.jackson.core + jackson-databind + 2.4.2 + + + + + org.slf4j + slf4j-api + 1.7.7 + + + + org.slf4j + slf4j-jdk14 + 1.7.7 + + + + org.apache.avro + avro + 1.7.7 + + + + joda-time + joda-time + 2.4 + + + + + org.hamcrest + hamcrest-all + 1.3 + test + + + + junit + junit + 4.11 + test + + + + org.mockito + mockito-all + 1.9.5 + test + + + diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java new file mode 100644 index 0000000000000..ec67fd7aabc3f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java @@ -0,0 +1,395 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk; + +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; +import com.google.cloud.dataflow.sdk.runners.TransformHierarchy; +import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.util.UserCodeException; +import com.google.cloud.dataflow.sdk.values.PBegin; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.POutput; +import com.google.cloud.dataflow.sdk.values.PValue; +import com.google.common.base.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * A Pipeline manages a DAG of PTransforms, and the PCollections + * that the PTransforms consume and produce. + * + *

After a {@code Pipeline} has been constructed, it can be executed, + * using a default or an explicit {@link PipelineRunner}. + * + *

Multiple {@code Pipeline}s can be constructed and executed independently + * and concurrently. + * + *

Each {@code Pipeline} is self-contained and isolated from any other + * {@code Pipeline}. The {@link PValues} that are inputs and outputs of each of a + * {@code Pipeline}'s {@link PTransform}s are also owned by that {@code Pipeline}. + * A {@code PValue} owned by one {@code Pipeline} can be read only by {@code PTransform}s + * also owned by that {@code Pipeline}. + * + *

Here's a typical example of use: + *

 {@code
+ * // Start by defining the options for the pipeline.
+ * PipelineOptions options = PipelineOptionsFactory.create();
+ * // Then create the pipeline.
+ * Pipeline p = Pipeline.create(options);
+ *
+ * // A root PTransform, like TextIO.Read or Create, gets added
+ * // to the Pipeline by being applied:
+ * PCollection lines =
+ *     p.apply(TextIO.Read.from("gs://bucket/dir/file*.txt"));
+ *
+ * // A Pipeline can have multiple root transforms:
+ * PCollection moreLines =
+ *     p.apply(TextIO.Read.from("gs://bucket/other/dir/file*.txt"));
+ * PCollection yetMoreLines =
+ *     p.apply(Create.of("yet", "more", "lines")).setCoder(StringUtf8Coder.of());
+ *
+ * // Further PTransforms can be applied, in an arbitrary (acyclic) graph.
+ * // Subsequent PTransforms (and intermediate PCollections etc.) are
+ * // implicitly part of the same Pipeline.
+ * PCollection allLines =
+ *     PCollectionList.of(lines).and(moreLines).and(yetMoreLines)
+ *     .apply(new Flatten());
+ * PCollection> wordCounts =
+ *     allLines
+ *     .apply(ParDo.of(new ExtractWords()))
+ *     .apply(new Count());
+ * PCollection formattedWordCounts =
+ *     wordCounts.apply(ParDo.of(new FormatCounts()));
+ * formattedWordCounts.apply(TextIO.Write.to("gs://bucket/dir/counts.txt"));
+ *
+ * // PTransforms aren't executed when they're applied, rather they're
+ * // just added to the Pipeline.  Once the whole Pipeline of PTransforms
+ * // is constructed, the Pipeline's PTransforms can be run using a
+ * // PipelineRunner.  The default PipelineRunner executes the Pipeline
+ * // directly, sequentially, in this one process, which is useful for
+ * // unit tests and simple experiments:
+ * p.run();
+ *
+ * } 
+ */ +public class Pipeline { + private static final Logger LOG = LoggerFactory.getLogger(Pipeline.class); + + ///////////////////////////////////////////////////////////////////////////// + // Public operations. + + /** + * Constructs a pipeline from the provided options. + * + * @return The newly created pipeline. + */ + public static Pipeline create(PipelineOptions options) { + Pipeline pipeline = new Pipeline(PipelineRunner.fromOptions(options), options); + LOG.debug("Creating {}", pipeline); + return pipeline; + } + + /** + * Returns a {@link PBegin} owned by this Pipeline. This is useful + * as the input of a root PTransform such as {@code TextIO.Read} or + * {@link com.google.cloud.dataflow.sdk.transforms.Create}. + */ + public PBegin begin() { + return PBegin.in(this); + } + + /** + * Starts using this pipeline with a root PTransform such as + * {@code TextIO.Read} or + * {@link com.google.cloud.dataflow.sdk.transforms.Create}. + * + *

+ * Alias for {@code begin().apply(root)}. + */ + public Output apply( + PTransform root) { + return begin().apply(root); + } + + /** + * Runs the Pipeline. + */ + public PipelineResult run() { + LOG.debug("Running {} via {}", this, runner); + try { + return runner.run(this); + } catch (UserCodeException e) { + // This serves to replace the stack with one that ends here and + // is caused by the caught UserCodeException, thereby splicing + // out all the stack frames in between the PipelineRunner itself + // and where the worker calls into the user's code. + throw new RuntimeException(e.getCause()); + } + } + + + ///////////////////////////////////////////////////////////////////////////// + // Below here are operations that aren't normally called by users. + + /** + * Returns the {@link CoderRegistry} that this Pipeline uses. + */ + public CoderRegistry getCoderRegistry() { + if (coderRegistry == null) { + coderRegistry = new CoderRegistry(); + coderRegistry.registerStandardCoders(); + } + return coderRegistry; + } + + /** + * Sets the {@link CoderRegistry} that this Pipeline uses. + */ + public void setCoderRegistry(CoderRegistry coderRegistry) { + this.coderRegistry = coderRegistry; + } + + /** + * A PipelineVisitor can be passed into + * {@link Pipeline#traverseTopologically} to be called for each of the + * transforms and values in the Pipeline. + */ + public interface PipelineVisitor { + public void enterCompositeTransform(TransformTreeNode node); + public void leaveCompositeTransform(TransformTreeNode node); + public void visitTransform(TransformTreeNode node); + public void visitValue(PValue value, TransformTreeNode producer); + } + + /** + * Invokes the PipelineVisitor's + * {@link PipelineVisitor#visitTransform} and + * {@link PipelineVisitor#visitValue} operations on each of this + * Pipeline's PTransforms and PValues, in forward + * topological order. + * + *

Traversal of the pipeline causes PTransform and PValue instances to + * be marked as finished, at which point they may no longer be modified. + * + *

Typically invoked by {@link PipelineRunner} subclasses. + */ + public void traverseTopologically(PipelineVisitor visitor) { + Set visitedValues = new HashSet<>(); + // Visit all the transforms, which should implicitly visit all the values. + transforms.visit(visitor, visitedValues); + if (!visitedValues.containsAll(values)) { + throw new RuntimeException( + "internal error: should have visited all the values " + + "after visiting all the transforms"); + } + } + + /** + * Applies the given PTransform to the given Input, + * and returns its Output. + * + *

Called by PInput subclasses in their {@code apply} methods. + */ + public static + Output applyTransform(Input input, + PTransform transform) { + return input.getPipeline().applyInternal(input, transform); + } + + ///////////////////////////////////////////////////////////////////////////// + // Below here are internal operations, never called by users. + + private final PipelineRunner runner; + private final PipelineOptions options; + private final TransformHierarchy transforms = new TransformHierarchy(); + private Collection values = new ArrayList<>(); + private Set usedFullNames = new HashSet<>(); + private CoderRegistry coderRegistry; + + @Deprecated + protected Pipeline(PipelineRunner runner) { + this(runner, PipelineOptionsFactory.create()); + } + + protected Pipeline(PipelineRunner runner, PipelineOptions options) { + this.runner = runner; + this.options = options; + } + + @Override + public String toString() { return "Pipeline#" + hashCode(); } + + /** + * Applies a transformation to the given input. + * + * @see Pipeline#apply + */ + private + Output applyInternal(Input input, + PTransform transform) { + input.finishSpecifying(); + + TransformTreeNode parent = transforms.getCurrent(); + String namePrefix = parent.getFullName(); + String fullName = uniquifyInternal(namePrefix, transform.getName()); + TransformTreeNode child = new TransformTreeNode(parent, transform, fullName, input); + parent.addComposite(child); + + transforms.addInput(child, input); + + transform.setPipeline(this); + LOG.debug("Adding {} to {}", transform, this); + try { + transforms.pushNode(child); + Output output = runner.apply(transform, input); + transforms.setOutput(child, output); + + // recordAsOutput is a NOOP if already called; + output.recordAsOutput(this, child.getTransform()); + verifyOutputState(output, child); + return output; + } finally { + transforms.popNode(); + } + } + + /** + * Returns all producing transforms for the {@link PValue}s contained + * in {@code output}. + */ + private List getProducingTransforms(POutput output) { + List producingTransforms = new ArrayList<>(); + for (PValue value : output.expand()) { + PTransform transform = value.getProducingTransformInternal(); + if (transform != null) { + producingTransforms.add(transform); + } + } + return producingTransforms; + } + + /** + * Verifies that the output of a PTransform is correctly defined. + * + *

A non-composite transform must have all + * of its outputs registered as produced by the transform. + */ + private void verifyOutputState(POutput output, TransformTreeNode node) { + if (!node.isCompositeNode()) { + PTransform thisTransform = node.getTransform(); + List producingTransforms = getProducingTransforms(output); + for (PTransform producingTransform : producingTransforms) { + if (thisTransform != producingTransform) { + throw new IllegalArgumentException("Output of non-composite transform " + + thisTransform + " is registered as being produced by" + + " a different transform: " + producingTransform); + } + } + } + } + + /** + * Returns the configured pipeline runner. + */ + public PipelineRunner getRunner() { + return runner; + } + + /** + * Returns the configured pipeline options. + */ + public PipelineOptions getOptions() { + return options; + } + + /** + * Returns the output associated with a transform. + * + * @throws IllegalStateException if the transform has not been applied to the pipeline. + */ + public POutput getOutput(PTransform transform) { + TransformTreeNode node = transforms.getNode(transform); + Preconditions.checkState(node != null, + "Unknown transform: " + transform); + return node.getOutput(); + } + + /** + * Returns the input associated with a transform. + * + * @throws IllegalStateException if the transform has not been applied to the pipeline. + */ + public PInput getInput(PTransform transform) { + TransformTreeNode node = transforms.getNode(transform); + Preconditions.checkState(node != null, + "Unknown transform: " + transform); + return node.getInput(); + } + + /** + * Returns the fully qualified name of a transform. + * + * @throws IllegalStateException if the transform has not been applied to the pipeline. + */ + public String getFullName(PTransform transform) { + TransformTreeNode node = transforms.getNode(transform); + Preconditions.checkState(node != null, + "Unknown transform: " + transform); + return node.getFullName(); + } + + /** + * Returns a unique name for a transform with the given prefix (from + * enclosing transforms) and initial name. + * + *

For internal use only. + */ + private String uniquifyInternal(String namePrefix, String origName) { + String name = origName; + int suffixNum = 2; + while (true) { + String candidate = namePrefix.isEmpty() ? name : namePrefix + "/" + name; + if (usedFullNames.add(candidate)) { + return candidate; + } + // A duplicate! Retry. + name = origName + suffixNum++; + } + } + + /** + * Adds the given PValue to this Pipeline. + * + *

For internal use only. + */ + public void addValueInternal(PValue value) { + this.values.add(value); + value.setPipelineInternal(this); + LOG.debug("Adding {} to {}", value, this); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/PipelineResult.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/PipelineResult.java new file mode 100644 index 0000000000000..7ab3845724f29 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/PipelineResult.java @@ -0,0 +1,27 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk; + +/** + * Result of {@link com.google.cloud.dataflow.sdk.Pipeline#run()}. + */ +public interface PipelineResult { + + // TODO: method to ask if pipeline is running / finished. + // TODO: method to retrieve error messages. + +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AtomicCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AtomicCoder.java new file mode 100644 index 0000000000000..6d032371207f1 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AtomicCoder.java @@ -0,0 +1,42 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import java.util.Collections; +import java.util.List; + +/** + * An AtomicCoder is one that has no component Coders or other state. + * All instances of its class are equal. + * + * @param the type of the values being transcoded + */ +public abstract class AtomicCoder extends StandardCoder { + protected AtomicCoder() {} + + @Override + public List> getCoderArguments() { return null; } + + /** + * Returns a list of values contained in the provided example + * value, one per type parameter. If there are no type parameters, + * returns the empty list. + */ + public static List getInstanceComponents(T exampleValue) { + return Collections.emptyList(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java new file mode 100644 index 0000000000000..5ea631a970a79 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java @@ -0,0 +1,202 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.cloud.dataflow.sdk.util.CloudObject; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.reflect.ReflectData; +import org.apache.avro.reflect.ReflectDatumReader; +import org.apache.avro.reflect.ReflectDatumWriter; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; + +/** + * An encoder using Avro binary format. + *

+ * The Avro schema is generated using reflection on the element type, using + * Avro's + * org.apache.avro.reflect.ReflectData, + * and encoded as part of the {@code Coder} instance. + *

+ * For complete details about schema generation and how it can be controlled please see + * the + * org.apache.avro.reflect package. + * Only concrete classes with a no-argument constructor can be mapped to Avro records. + * All inherited fields that are not static or transient are used. Fields are not permitted to be + * null unless annotated by + * + * org.apache.avro.reflect.Nullable or a + * + * org.apache.avro.reflect.Union containing null. + *

+ * To use, specify the {@code Coder} type on a PCollection: + *

+ * {@code
+ * PCollection records =
+ *     input.apply(...)
+ *          .setCoder(AvroCoder.of(MyCustomElement.class);
+ * }
+ * 
+ *

+ * or annotate the element class using {@code @DefaultCoder}. + *


+ * {@literal @}DefaultCoder(AvroCoder.class)
+ * public class MyCustomElement {
+ *   ...
+ * }
+ * 
+ * + * @param the type of elements handled by this coder + */ +public class AvroCoder extends StandardCoder { + + /** + * Returns an {@code AvroCoder} instance for the provided element type. + * @param the element type + */ + public static AvroCoder of(Class type) { + return new AvroCoder<>(type, ReflectData.get().getSchema(type)); + } + + /** + * Returns an {@code AvroCoder} instance for the Avro schema. The implicit + * type is GenericRecord. + */ + public static AvroCoder of(Schema schema) { + return new AvroCoder<>(GenericRecord.class, schema); + } + + /** + * Returns an {@code AvroCoder} instance for the provided element type + * using the provided Avro schema. + * + *

If the type argument is GenericRecord, the schema may be arbitrary. + * Otherwise, the schema must correspond to the type provided. + * + * @param the element type + */ + public static AvroCoder of(Class type, Schema schema) { + return new AvroCoder<>(type, schema); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @JsonCreator + public static AvroCoder of( + @JsonProperty("type") String classType, + @JsonProperty("schema") String schema) throws ClassNotFoundException { + Schema.Parser parser = new Schema.Parser(); + return new AvroCoder(Class.forName(classType), parser.parse(schema)); + } + + private final Class type; + private final Schema schema; + private final DatumWriter writer; + private final DatumReader reader; + private final EncoderFactory encoderFactory = new EncoderFactory(); + private final DecoderFactory decoderFactory = new DecoderFactory(); + + protected AvroCoder(Class type, Schema schema) { + this.type = type; + this.schema = schema; + this.reader = createDatumReader(); + this.writer = createDatumWriter(); + } + + @Override + public void encode(T value, OutputStream outStream, Context context) + throws IOException { + BinaryEncoder encoder = encoderFactory.directBinaryEncoder(outStream, null); + writer.write(value, encoder); + encoder.flush(); + } + + @Override + public T decode(InputStream inStream, Context context) throws IOException { + BinaryDecoder decoder = decoderFactory.directBinaryDecoder(inStream, null); + return reader.read(null, decoder); + } + + @Override + public List> getCoderArguments() { + return null; + } + + @Override + public CloudObject asCloudObject() { + CloudObject result = super.asCloudObject(); + addString(result, "type", type.getName()); + addString(result, "schema", schema.toString()); + return result; + } + + /** + * Depends upon the structure being serialized. + */ + @Override + public boolean isDeterministic() { + return false; + } + + /** + * Returns a new DatumReader that can be used to read from + * an Avro file directly. + */ + public DatumReader createDatumReader() { + if (type.equals(GenericRecord.class)) { + return new GenericDatumReader<>(schema); + } else { + return new ReflectDatumReader<>(schema); + } + } + + /** + * Returns a new DatumWriter that can be used to write to + * an Avro file directly. + */ + public DatumWriter createDatumWriter() { + if (type.equals(GenericRecord.class)) { + return new GenericDatumWriter<>(schema); + } else { + return new ReflectDatumWriter<>(schema); + } + } + + /** + * Returns the schema used by this coder. + */ + public Schema getSchema() { + return schema; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoder.java new file mode 100644 index 0000000000000..6af2d6f5ac4ea --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoder.java @@ -0,0 +1,88 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UTFDataFormatException; + +/** + * A BigEndianIntegerCoder encodes Integers in 4 bytes, big-endian. + */ +public class BigEndianIntegerCoder extends AtomicCoder { + @JsonCreator + public static BigEndianIntegerCoder of() { + return INSTANCE; + } + + + ///////////////////////////////////////////////////////////////////////////// + + private static final BigEndianIntegerCoder INSTANCE = + new BigEndianIntegerCoder(); + + private BigEndianIntegerCoder() {} + + @Override + public void encode(Integer value, OutputStream outStream, Context context) + throws IOException, CoderException { + if (value == null) { + throw new CoderException("cannot encode a null Integer"); + } + new DataOutputStream(outStream).writeInt(value); + } + + @Override + public Integer decode(InputStream inStream, Context context) + throws IOException, CoderException { + try { + return new DataInputStream(inStream).readInt(); + } catch (EOFException | UTFDataFormatException exn) { + // These exceptions correspond to decoding problems, so change + // what kind of exception they're branded as. + throw new CoderException(exn); + } + } + + @Override + public boolean isDeterministic() { + return true; + } + + /** + * Returns true since registerByteSizeObserver() runs in constant time. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(Integer value, Context context) { + return true; + } + + @Override + protected long getEncodedElementByteSize(Integer value, Context context) + throws Exception { + if (value == null) { + throw new CoderException("cannot encode a null Integer"); + } + return 4; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoder.java new file mode 100644 index 0000000000000..43ee9cab34be5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoder.java @@ -0,0 +1,87 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UTFDataFormatException; + +/** + * A BigEndianLongCoder encodes Longs in 8 bytes, big-endian. + */ +public class BigEndianLongCoder extends AtomicCoder { + @JsonCreator + public static BigEndianLongCoder of() { + return INSTANCE; + } + + + ///////////////////////////////////////////////////////////////////////////// + + private static final BigEndianLongCoder INSTANCE = new BigEndianLongCoder(); + + private BigEndianLongCoder() {} + + @Override + public void encode(Long value, OutputStream outStream, Context context) + throws IOException, CoderException { + if (value == null) { + throw new CoderException("cannot encode a null Long"); + } + new DataOutputStream(outStream).writeLong(value); + } + + @Override + public Long decode(InputStream inStream, Context context) + throws IOException, CoderException { + try { + return new DataInputStream(inStream).readLong(); + } catch (EOFException | UTFDataFormatException exn) { + // These exceptions correspond to decoding problems, so change + // what kind of exception they're branded as. + throw new CoderException(exn); + } + } + + @Override + public boolean isDeterministic() { + return true; + } + + /** + * Returns true since registerByteSizeObserver() runs in constant time. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(Long value, Context context) { + return true; + } + + @Override + protected long getEncodedElementByteSize(Long value, Context context) + throws Exception { + if (value == null) { + throw new CoderException("cannot encode a null Long"); + } + return 8; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java new file mode 100644 index 0000000000000..c750d932dd066 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java @@ -0,0 +1,103 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.util.VarInt; +import com.google.common.io.ByteStreams; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** + * A ByteArrayCoder encodes byte[] objects. + * + * If in a nested context, prefixes the encoded array with a VarInt encoding + * of the length. + */ +public class ByteArrayCoder extends AtomicCoder { + @JsonCreator + public static ByteArrayCoder of() { + return INSTANCE; + } + + + ///////////////////////////////////////////////////////////////////////////// + + private static final ByteArrayCoder INSTANCE = new ByteArrayCoder(); + + private ByteArrayCoder() {} + + @Override + public void encode(byte[] value, OutputStream outStream, Context context) + throws IOException, CoderException { + if (value == null) { + throw new CoderException("cannot encode a null byte[]"); + } + if (!context.isWholeStream) { + VarInt.encode(value.length, outStream); + } + outStream.write(value); + } + + @Override + public byte[] decode(InputStream inStream, Context context) + throws IOException, CoderException { + if (context.isWholeStream) { + ByteArrayOutputStream outStream = new ByteArrayOutputStream(); + ByteStreams.copy(inStream, outStream); + return outStream.toByteArray(); + } else { + int length = VarInt.decodeInt(inStream); + if (length < 0) { + throw new IOException("invalid length " + length); + } + byte[] value = new byte[length]; + ByteStreams.readFully(inStream, value); + return value; + } + } + + @Override + public boolean isDeterministic() { + return true; + } + + /** + * Returns true since registerByteSizeObserver() runs in constant time. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(byte[] value, Context context) { + return true; + } + + @Override + protected long getEncodedElementByteSize(byte[] value, Context context) + throws Exception { + if (value == null) { + throw new CoderException("cannot encode a null byte[]"); + } + long size = 0; + if (!context.isWholeStream) { + size += VarInt.getLength(value.length); + } + return size + value.length; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java new file mode 100644 index 0000000000000..3760cb82003bb --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java @@ -0,0 +1,154 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.List; + +/** + * A {@code Coder} defines how to encode and decode values of type {@code T} into byte streams. + * + *

All methods of a {@code Coder} are required to be thread safe. + * + *

{@code Coder}s are serialized during job creation and deserialized + * before use, via JSON serialization. + * + *

See {@link SerializableCoder} for an example of a {@code Coder} that adds + * a custom field to the {@code Coder} serialization. It provides a + * constructor annotated with {@link + * com.fasterxml.jackson.annotation.JsonCreator}, which is a factory method + * used when deserializing a {@code Coder} instance. + * + *

See {@link KvCoder} for an example of a nested {@code Coder} type. + * + * @param the type of the values being transcoded + */ +public interface Coder extends Serializable { + /** The context in which encoding or decoding is being done. */ + public static class Context { + /** + * The outer context. The value being encoded or decoded takes + * up the remainder of the whole record/stream contents. + */ + public static final Context OUTER = new Context(true); + + /** + * The nested context. The value being encoded or decoded is + * (potentially) a part of a larger record/stream contents, and + * may have other parts encoded or decoded after it. + */ + public static final Context NESTED = new Context(false); + + /** + * Whether the encoded or decoded value fills the remainder of the + * output or input (resp.) record/stream contents. If so, then + * the size of the decoded value can be determined from the + * remaining size of the record/stream contents, and so explicit + * lengths aren't required. + */ + public final boolean isWholeStream; + + public Context(boolean isWholeStream) { + this.isWholeStream = isWholeStream; + } + + public Context nested() { + return NESTED; + } + } + + /** + * Encodes the given value of type {@code T} onto the given output stream + * in the given context. + * + * @throws IOException if writing to the {@code OutputStream} fails + * for some reason + * @throws CoderException if the value could not be encoded for some reason + */ + public void encode(T value, OutputStream outStream, Context context) + throws CoderException, IOException; + + /** + * Decodes a value of type {@code T} from the given input stream in + * the given context. Returns the decoded value. + * + * @throws IOException if reading from the {@code InputStream} fails + * for some reason + * @throws CoderException if the value could not be decoded for some reason + */ + public T decode(InputStream inStream, Context context) + throws CoderException, IOException; + + /** + * If this is a {@code Coder} for a parameterized type, returns the + * list of {@code Coder}s being used for each of the parameters, or + * returns {@code null} if this cannot be done or this is not a + * parameterized type. + */ + public List> getCoderArguments(); + + /** + * Returns the {@link CloudObject} that represents this {@code Coder}. + */ + public CloudObject asCloudObject(); + + /** + * Returns true if the coding is deterministic. + * + *

In order for a {@code Coder} to be considered deterministic, + * the following must be true: + *

    + *
  • two values which compare as equal (via {@code Object.equals()} + * or {@code Comparable.compareTo()}, if supported), have the same + * encoding. + *
  • the {@code Coder} always produces a canonical encoding, which is the + * same for an instance of an object even if produced on different + * computers at different times. + *
+ */ + public boolean isDeterministic(); + + /** + * Returns whether {@link #registerByteSizeObserver} cheap enough to + * call for every element, that is, if this {@code Coder} can + * calculate the byte size of the element to be coded in roughly + * constant time (or lazily). + * + *

Not intended to be called by user code, but instead by + * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} + * implementations. + */ + public boolean isRegisterByteSizeObserverCheap(T value, Context context); + + /** + * Notifies the {@code ElementByteSizeObserver} about the byte size + * of the encoded value using this {@code Coder}. + * + *

Not intended to be called by user code, but instead by + * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} + * implementations. + */ + public void registerByteSizeObserver( + T value, ElementByteSizeObserver observer, Context context) + throws Exception; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderException.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderException.java new file mode 100644 index 0000000000000..1bbc3fa176b7e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderException.java @@ -0,0 +1,37 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import java.io.IOException; + +/** + * A CoderException is thrown if there is a problem encoding or + * decoding a value. + */ +public class CoderException extends IOException { + public CoderException(String message) { + super(message); + } + + public CoderException(String message, Throwable cause) { + super(message, cause); + } + + public CoderException(Throwable cause) { + super(cause); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java new file mode 100644 index 0000000000000..670b4e3e320af --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java @@ -0,0 +1,701 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.util.InstanceBuilder; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.TimestampedValue; +import com.google.common.reflect.TypeToken; + +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; +import java.lang.reflect.TypeVariable; +import java.lang.reflect.WildcardType; +import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * A CoderRegistry allows registering the default Coder to use for a Java class, + * and looking up and instantiating the default Coder for a Java type. + * + *

{@code CoderRegistry} uses the following mechanisms to determine a + * default {@link Coder} for a Java class, in order of precedence: + *

    + *
  • Registration: coders can be registered explicitly via + * {@link #registerCoder}. Built-in types are registered via + * {@link #registerStandardCoders()}. + *
  • Annotations: {@link DefaultCoder} can be used to annotate a type with + * the default {@code Coder} type. + *
  • Inheritance: {@link Serializable} objects are given a default + * {@code Coder} of {@link SerializableCoder}. + *
+ */ +public class CoderRegistry { + private static final Logger LOG = LoggerFactory.getLogger(CoderRegistry.class); + + /** A factory for default Coders for values of a particular class. */ + public abstract static class CoderFactory { + /** + * Returns the default Coder to use for values of a particular type, + * given the Coders for each of the type's generic parameter types. + * May return null if no default Coder can be created. + */ + public abstract Coder create( + List> typeArgumentCoders); + + /** + * Returns a list of objects contained in {@code value}, one per + * type argument, or {@code null} if none can be determined. + */ + public abstract List getInstanceComponents(Object value); + } + + /** A factory that always returns the coder with which it is instantiated. */ + public class ConstantCoderFactory extends CoderFactory { + private Coder coder; + + public ConstantCoderFactory(Coder coder) { + this.coder = coder; + } + + @Override + public Coder create(List> typeArgumentCoders) { + return this.coder; + } + + @Override + public List getInstanceComponents(Object value) { + return Collections.emptyList(); + } + } + + public CoderRegistry() {} + + /** + * Registers standard Coders with this CoderRegistry. + */ + public void registerStandardCoders() { + registerCoder(Double.class, DoubleCoder.class); + registerCoder(Instant.class, InstantCoder.class); + registerCoder(Integer.class, VarIntCoder.class); + registerCoder(Iterable.class, IterableCoder.class); + registerCoder(KV.class, KvCoder.class); + registerCoder(List.class, ListCoder.class); + registerCoder(Long.class, VarLongCoder.class); + registerCoder(String.class, StringUtf8Coder.class); + registerCoder(TableRow.class, TableRowJsonCoder.class); + registerCoder(Void.class, VoidCoder.class); + registerCoder(byte[].class, ByteArrayCoder.class); + registerCoder(URI.class, URICoder.class); + registerCoder(TimestampedValue.class, TimestampedValue.TimestampedValueCoder.class); + } + + /** + * Registers {@code coderClazz} as the default {@code Coder} + * class to handle encoding and decoding instances of {@code clazz} + * of type {@code T}. + * + *

{@code coderClazz} should have a static factory method with the + * following signature: + * + *

 {@code
+   * public static Coder of(Coder argCoder1, Coder argCoder2, ...)
+   * } 
+ * + *

This method will be called to create instances of {@code Coder} + * for values of type {@code T}, passing Coders for each of the generic type + * parameters of {@code T}. If {@code T} takes no generic type parameters, + * then the {@code of()} factory method should have no arguments. + * + *

If {@code T} is a parameterized type, then it should additionally + * have a method with the following signature: + * + *

 {@code
+   * public static List getInstanceComponents(T exampleValue);
+   * } 
+   *
+   * 

This method will be called to decompose a value during the coder + * inference process, to automatically choose coders for the components + */ + public void registerCoder(Class clazz, + Class coderClazz) { + int numTypeParameters = clazz.getTypeParameters().length; + + // Find the static factory method of coderClazz named 'of' with + // the appropriate number of type parameters. + + Class[] factoryMethodArgTypes = new Class[numTypeParameters]; + Arrays.fill(factoryMethodArgTypes, Coder.class); + + Method factoryMethod; + try { + factoryMethod = + coderClazz.getDeclaredMethod("of", factoryMethodArgTypes); + } catch (NoSuchMethodException | SecurityException exn) { + throw new IllegalArgumentException( + "cannot register Coder " + coderClazz + ": " + + "does not have an accessible method named 'of' with " + + numTypeParameters + " arguments of Coder type", + exn); + } + if (!Modifier.isStatic(factoryMethod.getModifiers())) { + throw new IllegalArgumentException( + "cannot register Coder " + coderClazz + ": " + + "method named 'of' with " + numTypeParameters + + " arguments of Coder type is not static"); + } + if (!coderClazz.isAssignableFrom(factoryMethod.getReturnType())) { + throw new IllegalArgumentException( + "cannot register Coder " + coderClazz + ": " + + "method named 'of' with " + numTypeParameters + + " arguments of Coder type does not return a " + coderClazz); + } + try { + if (!factoryMethod.isAccessible()) { + factoryMethod.setAccessible(true); + } + } catch (SecurityException exn) { + throw new IllegalArgumentException( + "cannot register Coder " + coderClazz + ": " + + "method named 'of' with " + numTypeParameters + + " arguments of Coder type is not accessible", + exn); + } + + // Find the static method to decompose values when inferring a coder, + // if there are type parameters for which we also need an example + // value + Method getComponentsMethod = null; + if (clazz.getTypeParameters().length > 0) { + try { + getComponentsMethod = coderClazz.getDeclaredMethod( + "getInstanceComponents", + clazz); + } catch (NoSuchMethodException | SecurityException exn) { + LOG.warn("cannot find getInstanceComponents for class {}. This may limit the ability to" + + " infer a Coder for values of this type.", coderClazz, exn); + } + } + + registerCoder(clazz, defaultCoderFactory(coderClazz, factoryMethod, getComponentsMethod)); + } + + public void registerCoder(Class rawClazz, + CoderFactory coderFactory) { + if (coderFactoryMap.put(rawClazz, coderFactory) != null) { + throw new IllegalArgumentException( + "cannot register multiple default Coder factories for " + rawClazz); + } + } + + public void registerCoder(Class rawClazz, Coder coder) { + CoderFactory factory = new ConstantCoderFactory(coder); + registerCoder(rawClazz, factory); + } + + /** + * Returns the Coder to use by default for values of the given type, + * or null if there is no default Coder. + */ + public Coder getDefaultCoder(TypeToken typeToken) { + return getDefaultCoder(typeToken, Collections.>emptyMap()); + } + + /** + * Returns the Coder to use by default for values of the given type, + * where the given context type uses the given context coder, + * or null if there is no default Coder. + */ + public Coder getDefaultCoder(TypeToken typeToken, + TypeToken contextTypeToken, + Coder contextCoder) { + return getDefaultCoder(typeToken, + createTypeBindings(contextTypeToken, contextCoder)); + } + + /** + * Returns the Coder to use on elements produced by this function, given + * the coder used for its input elements. + */ + public Coder getDefaultOutputCoder( + SerializableFunction fn, Coder inputCoder) { + return getDefaultCoder( + fn.getClass(), SerializableFunction.class, inputCoder); + } + + /** + * Returns the Coder to use for the last type parameter specialization + * of the subclass given Coders to use for all other type parameters + * specializations (if any). + */ + public Coder getDefaultCoder( + Class subClass, + Class baseClass, + Coder... knownCoders) { + Coder[] allCoders = new Coder[knownCoders.length + 1]; + // Last entry intentionally left null. + System.arraycopy(knownCoders, 0, allCoders, 0, knownCoders.length); + allCoders = getDefaultCoders(subClass, baseClass, allCoders); + @SuppressWarnings("unchecked") // trusted + Coder coder = (Coder) allCoders[knownCoders.length]; + return coder; + } + + /** + * Returns the Coder to use for the specified type parameter specialization + * of the subclass, given Coders to use for all other type parameters + * (if any). + */ + @SuppressWarnings("unchecked") + public Coder getDefaultCoder( + Class subClass, + Class baseClass, + Map> knownCoders, + String paramName) { + // TODO: Don't infer unneeded params. + return (Coder) getDefaultCoders(subClass, baseClass, knownCoders) + .get(paramName); + } + + /** + * Returns the Coder to use for the provided example value, if it can + * be determined, otherwise returns {@code null}. If more than one + * default coder matches, this will raise an exception. + */ + public Coder getDefaultCoder(T exampleValue) { + Class clazz = exampleValue.getClass(); + + if (clazz.getTypeParameters().length == 0) { + // Trust that getDefaultCoder returns a valid + // Coder for non-generic clazz. + @SuppressWarnings("unchecked") + Coder coder = (Coder) getDefaultCoder(clazz); + return coder; + } else { + CoderFactory factory = getDefaultCoderFactory(clazz); + if (factory == null) { + return null; + } + + List components = factory.getInstanceComponents(exampleValue); + if (components == null) { + return null; + } + + // componentcoders = components.map(this.getDefaultCoder) + List> componentCoders = new ArrayList<>(); + for (Object component : components) { + Coder componentCoder = getDefaultCoder(component); + if (componentCoder == null) { + return null; + } else { + componentCoders.add(componentCoder); + } + } + + // Trust that factory.create maps from valid component coders + // to a valid Coder. + @SuppressWarnings("unchecked") + Coder coder = (Coder) factory.create(componentCoders); + return coder; + } + } + + + /** + * Returns a Map from each of baseClass's type parameters to the Coder to + * use by default for it, in the context of subClass's specialization of + * baseClass. + * + *

For example, if baseClass is Map.class and subClass extends + * {@code Map} then this will return the registered Coders + * to use for String and Integer as a {"K": stringCoder, "V": intCoder} Map. + * The knownCoders parameter can be used to provide known coders for any of + * the parameters which will be used to infer the others. + * + * @param subClass the concrete type whose specializations are being inferred + * @param baseClass the base type, a parameterized class + * @param knownCoders a map corresponding to the set of known coders indexed + * by parameter name + */ + public Map> getDefaultCoders( + Class subClass, + Class baseClass, + Map> knownCoders) { + TypeVariable>[] typeParams = baseClass.getTypeParameters(); + Coder[] knownCodersArray = new Coder[typeParams.length]; + for (int i = 0; i < typeParams.length; i++) { + knownCodersArray[i] = knownCoders.get(typeParams[i].getName()); + } + Coder[] resultArray = getDefaultCoders( + subClass, baseClass, knownCodersArray); + Map> result = new HashMap<>(); + for (int i = 0; i < typeParams.length; i++) { + result.put(typeParams[i].getName(), resultArray[i]); + } + return result; + } + + /** + * Returns an array listing, for each of baseClass's type parameters, the + * Coder to use by default for it, in the context of subClass's specialization + * of baseClass. + * + *

For example, if baseClass is Map.class and subClass extends + * {@code Map} then this will return the registered Coders + * to use for String and Integer in that order. The knownCoders parameter + * can be used to provide known coders for any of the parameters which will + * be used to infer the others. + * + *

If a type cannot be inferred, null is returned. + * + * @param subClass the concrete type whose specializations are being inferred + * @param baseClass the base type, a parameterized class + * @param knownCoders an array corresponding to the set of base class + * type parameters. Each entry is can be either a Coder (in which + * case it will be used for inference) or null (in which case it + * will be inferred). May be null to indicate the entire set of + * parameters should be inferred. + * @throws IllegalArgumentException if baseClass doesn't have type parameters + * or if the length of knownCoders is not equal to the number of type + * parameters + */ + public Coder[] getDefaultCoders( + Class subClass, + Class baseClass, + Coder[] knownCoders) { + Type type = TypeToken.of(subClass).getSupertype(baseClass).getType(); + if (!(type instanceof ParameterizedType)) { + throw new IllegalArgumentException(type + " is not a ParameterizedType"); + } + ParameterizedType parameterizedType = (ParameterizedType) type; + Type[] typeArgs = parameterizedType.getActualTypeArguments(); + if (knownCoders == null) { + knownCoders = new Coder[typeArgs.length]; + } else if (typeArgs.length != knownCoders.length) { + throw new IllegalArgumentException( + "Class " + baseClass + " has " + typeArgs.length + " parameters, " + + "but " + knownCoders.length + " coders are requested."); + } + Map> context = new HashMap<>(); + for (int i = 0; i < knownCoders.length; i++) { + if (knownCoders[i] != null) { + if (!isCompatible(knownCoders[i], typeArgs[i])) { + throw new IllegalArgumentException( + "Cannot encode elements of type " + typeArgs[i] + + " with " + knownCoders[i]); + } + context.put(typeArgs[i], knownCoders[i]); + } + } + Coder[] result = new Coder[typeArgs.length]; + for (int i = 0; i < knownCoders.length; i++) { + if (knownCoders[i] != null) { + result[i] = knownCoders[i]; + } else { + result[i] = getDefaultCoder(typeArgs[i], context); + } + } + return result; + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Returns whether the given coder can possibly encode elements + * of the given type. + */ + static boolean isCompatible(Coder coder, Type type) { + Type coderType = + ((ParameterizedType) + TypeToken.of(coder.getClass()).getSupertype(Coder.class).getType()) + .getActualTypeArguments()[0]; + if (type instanceof TypeVariable) { + return true; // Can't rule it out. + } + Class coderClass = TypeToken.of(coderType).getRawType(); + if (!coderClass.isAssignableFrom(TypeToken.of(type).getRawType())) { + return false; + } + if (coderType instanceof ParameterizedType + && !isNullOrEmpty(coder.getCoderArguments())) { + @SuppressWarnings("unchecked") + Type[] typeArguments = + ((ParameterizedType) + TypeToken.of(type).getSupertype((Class) coderClass).getType()) + .getActualTypeArguments(); + List> typeArgumentCoders = coder.getCoderArguments(); + assert typeArguments.length == typeArgumentCoders.size(); + for (int i = 0; i < typeArguments.length; i++) { + if (!isCompatible( + typeArgumentCoders.get(i), + TypeToken.of(type).resolveType(typeArguments[i]).getType())) { + return false; + } + } + } + return true; // For all we can tell. + } + + private static boolean isNullOrEmpty(Collection c) { + return c == null || c.size() == 0; + } + + /** + * The map of classes to the CoderFactories to use to create their + * default Coders. + */ + Map, CoderFactory> coderFactoryMap = new HashMap<>(); + + /** + * Returns a CoderFactory that invokes the given static factory method + * to create the Coder. + */ + static CoderFactory defaultCoderFactory( + final Class coderClazz, + final Method coderFactoryMethod, + final Method getComponentsMethod) { + + return new CoderFactory() { + @Override + public Coder create(List> typeArgumentCoders) { + try { + return (Coder) coderFactoryMethod.invoke( + null /* static */, typeArgumentCoders.toArray()); + } catch (IllegalAccessException | + IllegalArgumentException | + InvocationTargetException | + NullPointerException | + ExceptionInInitializerError exn) { + throw new IllegalStateException( + "error when invoking Coder factory method " + coderFactoryMethod, + exn); + } + } + + @Override + public List getInstanceComponents(Object value) { + if (getComponentsMethod == null) { + throw new IllegalStateException( + "no suitable static getInstanceComponents method available for " + + "Coder " + coderClazz); + } + + try { + @SuppressWarnings("unchecked") + List result = (List) (getComponentsMethod.invoke( + null /* static */, value)); + return result; + } catch (IllegalAccessException + | IllegalArgumentException + | InvocationTargetException + | NullPointerException + | ExceptionInInitializerError exn) { + throw new IllegalStateException( + "error when invoking Coder getComponents method " + getComponentsMethod, + exn); + } + } + }; + } + + static CoderFactory defaultCoderFactory(Class coderClazz, final Method coderFactoryMethod) { + return defaultCoderFactory(coderClazz, coderFactoryMethod, null); + } + + /** + * Returns the CoderFactory to use to create default Coders for + * instances of the given class, or null if there is no default + * CoderFactory registered. + */ + CoderFactory getDefaultCoderFactory(Class clazz) { + CoderFactory coderFactory = coderFactoryMap.get(clazz); + if (coderFactory == null) { + LOG.debug("No Coder registered for {}", clazz); + } + return coderFactory; + } + + /** + * Returns the Coder to use by default for values of the given type, + * in a context where the given types use the given coders, + * or null if there is no default Coder. + */ + Coder getDefaultCoder(TypeToken typeToken, + Map> typeCoderBindings) { + Coder defaultCoder = getDefaultCoder(typeToken.getType(), + typeCoderBindings); + LOG.debug("Default Coder for {}: {}", typeToken, defaultCoder); + @SuppressWarnings("unchecked") + Coder result = (Coder) defaultCoder; + return result; + } + + /** + * Returns the Coder to use by default for values of the given type, + * in a context where the given types use the given coders, + * or null if there is no default Coder. + */ + Coder getDefaultCoder(Type type, Map> typeCoderBindings) { + Coder coder = typeCoderBindings.get(type); + if (coder != null) { + return coder; + } + if (type instanceof Class) { + return getDefaultCoder((Class) type); + } else if (type instanceof ParameterizedType) { + return this.getDefaultCoder((ParameterizedType) type, + typeCoderBindings); + } else if (type instanceof TypeVariable + || type instanceof WildcardType) { + // No default coder for an unknown generic type. + LOG.debug("No Coder for unknown generic type {}", type); + return null; + } else { + throw new RuntimeException( + "internal error: unexpected kind of Type: " + type); + } + } + + /** + * Returns the Coder to use by default for values of the given + * class, or null if there is no default Coder. + */ + Coder getDefaultCoder(Class clazz) { + CoderFactory coderFactory = getDefaultCoderFactory(clazz); + if (coderFactory != null) { + LOG.debug("Default Coder for {} found by factory", clazz); + return coderFactory.create(Collections.>emptyList()); + } + + DefaultCoder defaultAnnotation = clazz.getAnnotation( + DefaultCoder.class); + if (defaultAnnotation != null) { + LOG.debug("Default Coder for {} found by DefaultCoder annotation", clazz); + return InstanceBuilder.ofType(Coder.class) + .fromClass(defaultAnnotation.value()) + .fromFactoryMethod("of") + .withArg(Class.class, clazz) + .build(); + } + + // Interface-based defaults. + if (Serializable.class.isAssignableFrom(clazz)) { + @SuppressWarnings("unchecked") + Class serializableClazz = + (Class) clazz; + LOG.debug("Default Coder for {}: SerializableCoder", serializableClazz); + return SerializableCoder.of(serializableClazz); + } + + LOG.debug("No default Coder for {}", clazz); + return null; + } + + /** + * Returns the Coder to use by default for values of the given + * parameterized type, in a context where the given types use the + * given coders, or null if there is no default Coder. + */ + Coder getDefaultCoder( + ParameterizedType type, + Map> typeCoderBindings) { + Class rawClazz = (Class) type.getRawType(); + CoderFactory coderFactory = getDefaultCoderFactory(rawClazz); + if (coderFactory == null) { + return null; + } + List> typeArgumentCoders = new ArrayList<>(); + for (Type typeArgument : type.getActualTypeArguments()) { + Coder typeArgumentCoder = getDefaultCoder(typeArgument, + typeCoderBindings); + if (typeArgumentCoder == null) { + return null; + } + typeArgumentCoders.add(typeArgumentCoder); + } + return coderFactory.create(typeArgumentCoders); + } + + /** + * Returns a Map where each of the type variables embedded in the + * given type are mapped to the corresponding Coders in the given + * coder. + */ + Map> createTypeBindings(TypeToken typeToken, + Coder coder) { + Map> typeCoderBindings = new HashMap<>(); + fillTypeBindings(typeToken.getType(), coder, typeCoderBindings); + return typeCoderBindings; + } + + /** + * Adds to the given map bindings from each of the type variables + * embedded in the given type to the corresponding Coders in the + * given coder. + */ + void fillTypeBindings(Type type, + Coder coder, + Map> typeCoderBindings) { + if (type instanceof TypeVariable) { + LOG.debug("Binding type {} to Coder {}", type, coder); + typeCoderBindings.put(type, coder); + } else if (type instanceof ParameterizedType) { + fillTypeBindings((ParameterizedType) type, + coder, + typeCoderBindings); + } + } + + /** + * Adds to the given map bindings from each of the type variables + * embedded in the given parameterized type to the corresponding + * Coders in the given coder. + */ + void fillTypeBindings(ParameterizedType type, + Coder coder, + Map> typeCoderBindings) { + Type[] typeArguments = type.getActualTypeArguments(); + List> coderArguments = coder.getCoderArguments(); + if (coderArguments == null + || typeArguments.length != coderArguments.size()) { + return; + } + for (int i = 0; i < typeArguments.length; i++) { + fillTypeBindings(typeArguments[i], + coderArguments.get(i), + typeCoderBindings); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java new file mode 100644 index 0000000000000..546695dfefe80 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java @@ -0,0 +1,63 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.util.PropertyNames; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Collection; +import java.util.List; + +/** + * A CollectionCoder encodes Collections. + * + * @param the type of the elements of the Collections being transcoded + */ +public class CollectionCoder extends IterableLikeCoder> { + + public static CollectionCoder of(Coder elemCoder) { + return new CollectionCoder<>(elemCoder); + } + + ///////////////////////////////////////////////////////////////////////////// + // Internal operations below here. + + @JsonCreator + public static CollectionCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List components) { + Preconditions.checkArgument(components.size() == 1, + "Expecting 1 component, got " + components.size()); + return of((Coder) components.get(0)); + } + + /** + * Returns the first element in this collection if it is non-empty, + * otherwise returns {@code null}. + */ + public static List getInstanceComponents( + Collection exampleValue) { + return getInstanceComponentsHelper(exampleValue); + } + + CollectionCoder(Coder elemCoder) { + super(elemCoder); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java new file mode 100644 index 0000000000000..6b31297a1071c --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java @@ -0,0 +1,83 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.util.StringUtils; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.Serializable; + +/** + * An abstract base class for writing Coders that encodes itself via java + * serialization. Subclasses only need to implement the {@link Coder#encode} + * and {@link Coder#decode} methods. + * + *

+ * Not to be confused with {@link SerializableCoder} that encodes serializables. + * + * @param the type of elements handled by this coder + */ +public abstract class CustomCoder extends AtomicCoder + implements Serializable { + + @JsonCreator + public static CustomCoder of( + // N.B. typeId is a required parameter here, since a field named "@type" + // is presented to the deserializer as an input. + // + // If this method did not consume the field, Jackson2 would observe an + // unconsumed field and a returned value of a derived type. So Jackson2 + // would attempt to update the returned value with the unconsumed field + // data, The standard JsonDeserializer does not implement a mechanism for + // updating constructed values, so it would throw an exception, causing + // deserialization to fail. + @JsonProperty(value = "@type", required = false) String typeId, + @JsonProperty("type") String type, + @JsonProperty("serialized_coder") String serializedCoder) { + return (CustomCoder) SerializableUtils.deserializeFromByteArray( + StringUtils.jsonStringToByteArray(serializedCoder), + type); + } + + @Override + public CloudObject asCloudObject() { + // N.B. We use the CustomCoder class, not the derived class, since during + // deserialization we will be using the CustomCoder's static factory method + // to construct an instance of the derived class. + CloudObject result = CloudObject.forClass(CustomCoder.class); + addString(result, "type", getClass().getName()); + addString(result, "serialized_coder", + StringUtils.byteArrayToJsonString( + SerializableUtils.serializeToByteArray(this))); + return result; + } + + @Override + public boolean isDeterministic() { + return false; + } + + // This coder inherits isRegisterByteSizeObserverCheap, + // getEncodedElementByteSize and registerByteSizeObserver + // from StandardCoder. Override if we can do better. +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DefaultCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DefaultCoder.java new file mode 100644 index 0000000000000..6c6f4197c5a86 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DefaultCoder.java @@ -0,0 +1,68 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Specifies a default {@link Coder} class to handle encoding and decoding + * instances of the annotated class. + * + *

The specified {@code Coder} must implement a function with the following + * signature: + *

{@code
+ * public static Coder of(Class clazz) {...}
+ * }
+ * + *

For example, to configure the use of Java serialization as the default + * for a class, annotate the class to use + * {@link com.google.cloud.dataflow.sdk.coders.SerializableCoder} as follows:the + * + *


+ * {@literal @}DefaultCoder(SerializableCoder.class)
+ * public class MyCustomDataType {
+ *   // ...
+ * }
+ * 
+ * + *

Similarly, to configure the use of + * {@link com.google.cloud.dataflow.sdk.coders.AvroCoder} as the default: + *


+ * {@literal @}DefaultCoder(AvroCoder.class)
+ * public class MyCustomDataType {
+ *   public MyCustomDataType() {}   // Avro requires an empty constructor.
+ *   // ...
+ * }
+ * 
+ * + *

Coders specified explicitly via + * {@link com.google.cloud.dataflow.sdk.values.PCollection#setCoder(Coder) + * PCollection.setCoder} + * take precedence, followed by Coders registered at runtime via + * {@link CoderRegistry#registerCoder}. + */ +@Documented +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.TYPE) +@SuppressWarnings("rawtypes") +public @interface DefaultCoder { + Class value(); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java new file mode 100644 index 0000000000000..6b531ad0dc454 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java @@ -0,0 +1,92 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UTFDataFormatException; + +/** + * A DoubleCoder encodes Doubles in 8 bytes. + */ +public class DoubleCoder extends AtomicCoder { + @JsonCreator + public static DoubleCoder of() { + return INSTANCE; + } + + + ///////////////////////////////////////////////////////////////////////////// + + private static final DoubleCoder INSTANCE = new DoubleCoder(); + + private DoubleCoder() {} + + @Override + public void encode(Double value, OutputStream outStream, Context context) + throws IOException, CoderException { + if (value == null) { + throw new CoderException("cannot encode a null Double"); + } + new DataOutputStream(outStream).writeDouble(value); + } + + @Override + public Double decode(InputStream inStream, Context context) + throws IOException, CoderException { + try { + return new DataInputStream(inStream).readDouble(); + } catch (EOFException | UTFDataFormatException exn) { + // These exceptions correspond to decoding problems, so change + // what kind of exception they're branded as. + throw new CoderException(exn); + } + } + + /** + * Floating-point operations are not guaranteed to be deterministic, even + * if the storage format might be, so floating point representations are not + * recommended for use in operations which require deterministic inputs. + */ + @Override + public boolean isDeterministic() { + return false; + } + + /** + * Returns true since registerByteSizeObserver() runs in constant time. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(Double value, Context context) { + return true; + } + + @Override + protected long getEncodedElementByteSize(Double value, Context context) + throws Exception { + if (value == null) { + throw new CoderException("cannot encode a null Double"); + } + return 8; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java new file mode 100644 index 0000000000000..988a04c03160c --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java @@ -0,0 +1,82 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.api.services.datastore.DatastoreV1.Entity; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** + * An EntityCoder encodes/decodes Datastore Entity objects. + */ +public class EntityCoder extends AtomicCoder { + + @JsonCreator + public static EntityCoder of() { + return INSTANCE; + } + + /***************************/ + + private static final EntityCoder INSTANCE = new EntityCoder(); + + private EntityCoder() {} + + @Override + public void encode(Entity value, OutputStream outStream, Context context) + throws IOException, CoderException { + if (value == null) { + throw new CoderException("cannot encode a null Entity"); + } + + // Since Entity implements com.google.protobuf.MessageLite, + // we could directly use writeTo to write to a OutputStream object + outStream.write(java.nio.ByteBuffer.allocate(4).putInt(value.getSerializedSize()).array()); + value.writeTo(outStream); + outStream.flush(); + } + + @Override + public Entity decode(InputStream inStream, Context context) + throws IOException { + byte[] entitySize = new byte[4]; + inStream.read(entitySize, 0, 4); + int size = java.nio.ByteBuffer.wrap(entitySize).getInt(); + byte[] data = new byte[size]; + inStream.read(data, 0, size); + return Entity.parseFrom(data); + } + + @Override + protected long getEncodedElementByteSize(Entity value, Context context) + throws Exception { + return value.getSerializedSize(); + } + + /** + * A datastore kind can hold arbitrary Object instances, + * which makes the encoding non-deterministic. + */ + @Override + public boolean isDeterministic() { + return false; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java new file mode 100644 index 0000000000000..3190124391703 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java @@ -0,0 +1,60 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import org.joda.time.Instant; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** + * A InstantCoder encodes joda Instant. + */ +public class InstantCoder extends AtomicCoder { + @JsonCreator + public static InstantCoder of() { + return INSTANCE; + } + + ///////////////////////////////////////////////////////////////////////////// + + private static final InstantCoder INSTANCE = new InstantCoder(); + + private InstantCoder() {} + + @Override + public void encode(Instant value, OutputStream outStream, Context context) + throws CoderException, IOException { + // Shift the millis by Long.MIN_VALUE so that negative values sort before positive + // values when encoded. The overflow is well-defined: + // http://docs.oracle.com/javase/specs/jls/se7/html/jls-15.html#jls-15.18.2 + BigEndianLongCoder.of().encode(value.getMillis() - Long.MIN_VALUE, outStream, context); + } + + @Override + public Instant decode(InputStream inStream, Context context) + throws CoderException, IOException { + return new Instant(BigEndianLongCoder.of().decode(inStream, context) + Long.MIN_VALUE); + } + + @Override + public boolean isDeterministic() { + return true; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java new file mode 100644 index 0000000000000..801dd2042cfdd --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java @@ -0,0 +1,72 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.PropertyNames; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +/** + * An IterableCoder encodes Iterables. + * + * @param the type of the elements of the Iterables being transcoded + */ +public class IterableCoder extends IterableLikeCoder> { + + public static IterableCoder of(Coder elemCoder) { + return new IterableCoder<>(elemCoder); + } + + ///////////////////////////////////////////////////////////////////////////// + // Internal operations below here. + + @JsonCreator + public static IterableCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> components) { + Preconditions.checkArgument(components.size() == 1, + "Expecting 1 component, got " + components.size()); + return of(components.get(0)); + } + + /** + * Returns the first element in this iterable if it is non-empty, + * otherwise returns {@code null}. + */ + public static List getInstanceComponents( + Iterable exampleValue) { + return getInstanceComponentsHelper(exampleValue); + } + + IterableCoder(Coder elemCoder) { + super(elemCoder); + } + + @Override + public CloudObject asCloudObject() { + CloudObject result = super.asCloudObject(); + addBoolean(result, PropertyNames.IS_STREAM_LIKE, true); + return result; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java new file mode 100644 index 0000000000000..e6ecdbe26bb97 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java @@ -0,0 +1,227 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObservableIterable; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Observable; +import java.util.Observer; + +/** + * The base class of Coders for Iterable subclasses. + * + * @param the type of the elements of the Iterables being transcoded + * @param the type of the Iterables being transcoded + */ +public abstract class IterableLikeCoder> + extends StandardCoder { + + public Coder getElemCoder() { return elemCoder; } + + ///////////////////////////////////////////////////////////////////////////// + // Internal operations below here. + + final Coder elemCoder; + + /** + * Returns the first element in this iterable-like if it is non-empty, + * otherwise returns {@code null}. + */ + protected static > + List getInstanceComponentsHelper( + IT exampleValue) { + for (T value : exampleValue) { + return Arrays.asList(value); + } + return null; + } + + protected IterableLikeCoder(Coder elemCoder) { + this.elemCoder = elemCoder; + } + + @Override + public void encode(IT iterable, OutputStream outStream, Context context) + throws IOException, CoderException { + if (iterable == null) { + throw new CoderException("cannot encode a null Iterable"); + } + Context nestedContext = context.nested(); + DataOutputStream dataOutStream = new DataOutputStream(outStream); + if (iterable instanceof Collection) { + // We can know the size of the Iterable. Use an encoding with a + // leading size field, followed by that many elements. + Collection collection = (Collection) iterable; + dataOutStream.writeInt(collection.size()); + for (T elem : collection) { + elemCoder.encode(elem, dataOutStream, nestedContext); + } + } else { + // We don't know the size without traversing it. So use a + // "hasNext" sentinel before each element. + // TODO: Don't use the sentinel if context.isWholeStream. + dataOutStream.writeInt(-1); + for (T elem : iterable) { + dataOutStream.writeBoolean(true); + elemCoder.encode(elem, dataOutStream, nestedContext); + } + dataOutStream.writeBoolean(false); + } + // Make sure all our output gets pushed to the underlying outStream. + dataOutStream.flush(); + } + + @Override + public IT decode(InputStream inStream, Context context) + throws IOException, CoderException { + Context nestedContext = context.nested(); + DataInputStream dataInStream = new DataInputStream(inStream); + int size = dataInStream.readInt(); + if (size >= 0) { + List elements = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + elements.add(elemCoder.decode(dataInStream, nestedContext)); + } + return (IT) elements; + } else { + // We don't know the size a priori. Check if we're done with + // each element. + List elements = new ArrayList<>(); + while (dataInStream.readBoolean()) { + elements.add(elemCoder.decode(dataInStream, nestedContext)); + } + return (IT) elements; + } + } + + @Override + public List> getCoderArguments() { + return Arrays.asList(elemCoder); + } + + /** + * Encoding is not deterministic for the general Iterable case, as it depends + * upon the type of iterable. This may allow two objects to compare as equal + * while the encoding differs. + */ + @Override + public boolean isDeterministic() { + return false; + } + + /** + * Returns whether iterable can use lazy counting, since that + * requires minimal extra computation. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(IT iterable, Context context) { + return iterable instanceof ElementByteSizeObservableIterable; + } + + /** + * Notifies ElementByteSizeObserver about the byte size of the + * encoded value using this coder. + */ + @Override + public void registerByteSizeObserver( + IT iterable, ElementByteSizeObserver observer, Context context) + throws Exception { + if (iterable == null) { + throw new CoderException("cannot encode a null Iterable"); + } + Context nestedContext = context.nested(); + + if (iterable instanceof ElementByteSizeObservableIterable) { + observer.setLazy(); + ElementByteSizeObservableIterable observableIT = + (ElementByteSizeObservableIterable) iterable; + observableIT.addObserver( + new IteratorObserver(observer, iterable instanceof Collection)); + } else { + if (iterable instanceof Collection) { + // We can know the size of the Iterable. Use an encoding with a + // leading size field, followed by that many elements. + Collection collection = (Collection) iterable; + observer.update(4L); + for (T elem : collection) { + elemCoder.registerByteSizeObserver(elem, observer, nestedContext); + } + } else { + // We don't know the size without traversing it. So use a + // "hasNext" sentinel before each element. + // TODO: Don't use the sentinel if context.isWholeStream. + observer.update(4L); + for (T elem : iterable) { + observer.update(1L); + elemCoder.registerByteSizeObserver(elem, observer, nestedContext); + } + observer.update(1L); + } + } + } + + /** + * An observer that gets notified when an observable iterator + * returns a new value. This observer just notifies an outerObserver + * about this event. Additionally, the outerObserver is notified + * about additional separators that are transparently added by this + * coder. + */ + private class IteratorObserver implements Observer { + private final ElementByteSizeObserver outerObserver; + private final boolean countable; + + public IteratorObserver(ElementByteSizeObserver outerObserver, + boolean countable) { + this.outerObserver = outerObserver; + this.countable = countable; + + if (countable) { + // Additional 4 bytes are due to size. + outerObserver.update(4L); + } else { + // Additional 5 bytes are due to size = -1 (4 bytes) and + // hasNext = false (1 byte). + outerObserver.update(5L); + } + } + + @Override + public void update(Observable obs, Object obj) { + if (!(obj instanceof Long)) { + throw new AssertionError("unexpected parameter object"); + } + + if (countable) { + outerObserver.update(obs, obj); + } else { + // Additional 1 byte is due to hasNext = true flag. + outerObserver.update(obs, 1 + (long) obj); + } + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java new file mode 100644 index 0000000000000..000d6ca75807a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java @@ -0,0 +1,142 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; +import com.google.cloud.dataflow.sdk.values.KV; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.List; + +/** + * A KvCoder encodes KVs. + * + * @param the type of the keys of the KVs being transcoded + * @param the type of the values of the KVs being transcoded + */ +public class KvCoder extends KvCoderBase> { + + public static KvCoder of(Coder keyCoder, + Coder valueCoder) { + return new KvCoder<>(keyCoder, valueCoder); + } + + @JsonCreator + public static KvCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> components) { + Preconditions.checkArgument(components.size() == 2, + "Expecting 2 components, got " + components.size()); + return of(components.get(0), components.get(1)); + } + + public static List getInstanceComponents( + KV exampleValue) { + return Arrays.asList( + exampleValue.getKey(), + exampleValue.getValue()); + } + + public Coder getKeyCoder() { return keyCoder; } + public Coder getValueCoder() { return valueCoder; } + + ///////////////////////////////////////////////////////////////////////////// + + Coder keyCoder; + Coder valueCoder; + + KvCoder(Coder keyCoder, Coder valueCoder) { + this.keyCoder = keyCoder; + this.valueCoder = valueCoder; + } + + @Override + public void encode(KV kv, OutputStream outStream, Context context) + throws IOException, CoderException { + if (kv == null) { + throw new CoderException("cannot encode a null KV"); + } + Context nestedContext = context.nested(); + keyCoder.encode(kv.getKey(), outStream, nestedContext); + valueCoder.encode(kv.getValue(), outStream, nestedContext); + } + + @Override + public KV decode(InputStream inStream, Context context) + throws IOException, CoderException { + Context nestedContext = context.nested(); + K key = keyCoder.decode(inStream, nestedContext); + V value = valueCoder.decode(inStream, nestedContext); + return KV.of(key, value); + } + + @Override + public List> getCoderArguments() { + return Arrays.asList(keyCoder, valueCoder); + } + + @Override + public boolean isDeterministic() { + return getKeyCoder().isDeterministic() && getValueCoder().isDeterministic(); + } + + @Override + public CloudObject asCloudObject() { + CloudObject result = super.asCloudObject(); + addBoolean(result, PropertyNames.IS_PAIR_LIKE, true); + return result; + } + + /** + * Returns whether both keyCoder and valueCoder are considered not expensive. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(KV kv, Context context) { + return keyCoder.isRegisterByteSizeObserverCheap(kv.getKey(), + context.nested()) + && valueCoder.isRegisterByteSizeObserverCheap(kv.getValue(), + context.nested()); + } + + /** + * Notifies ElementByteSizeObserver about the byte size of the + * encoded value using this coder. + */ + @Override + public void registerByteSizeObserver( + KV kv, ElementByteSizeObserver observer, Context context) + throws Exception { + if (kv == null) { + throw new CoderException("cannot encode a null KV"); + } + keyCoder.registerByteSizeObserver( + kv.getKey(), observer, context.nested()); + valueCoder.registerByteSizeObserver( + kv.getValue(), observer, context.nested()); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoderBase.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoderBase.java new file mode 100644 index 0000000000000..b959e1c3c576b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoderBase.java @@ -0,0 +1,53 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.util.PropertyNames; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +/** + * A abstract base class for KvCoder. Works around a Jackson2 bug tickled when building + * {@link KvCoder} directly (as of this writing, Jackson2 walks off the end of + * an array when it tries to deserialize a class with multiple generic type + * parameters). This class should be removed when possible. + * @param the type of values being transcoded + */ +public abstract class KvCoderBase extends StandardCoder { + @JsonCreator + public static KvCoderBase of( + // N.B. typeId is a required parameter here, since a field named "@type" + // is presented to the deserializer as an input. + // + // If this method did not consume the field, Jackson2 would observe an + // unconsumed field and a returned value of a derived type. So Jackson2 + // would attempt to update the returned value with the unconsumed field + // data. The standard JsonDeserializer does not implement a mechanism for + // updating constructed values, so it would throw an exception, causing + // deserialization to fail. + @JsonProperty(value = "@type", required = false) String typeId, + @JsonProperty(value = PropertyNames.IS_PAIR_LIKE, required = false) boolean isPairLike, + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> components) { + return KvCoder.of(components); + } + + protected KvCoderBase() {} +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java new file mode 100644 index 0000000000000..ab9d8147aa1f1 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java @@ -0,0 +1,70 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.util.PropertyNames; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +/** + * A ListCoder encodes Lists. + * + * @param the type of the elements of the Lists being transcoded + */ +public class ListCoder extends IterableLikeCoder> { + + public static ListCoder of(Coder elemCoder) { + return new ListCoder<>(elemCoder); + } + + ///////////////////////////////////////////////////////////////////////////// + // Internal operations below here. + + @JsonCreator + public static ListCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> components) { + Preconditions.checkArgument(components.size() == 1, + "Expecting 1 component, got " + components.size()); + return of((Coder) components.get(0)); + } + + /** + * Returns the first element in this list if it is non-empty, + * otherwise returns {@code null}. + */ + public static List getInstanceComponents(List exampleValue) { + return getInstanceComponentsHelper(exampleValue); + } + + ListCoder(Coder elemCoder) { + super(elemCoder); + } + + /** + * List sizes are always known, so ListIterable may be deterministic while + * the general IterableLikeCoder is not. + */ + @Override + public boolean isDeterministic() { + return getElemCoder().isDeterministic(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java new file mode 100644 index 0000000000000..fa3fc58950150 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java @@ -0,0 +1,149 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +/** + * A MapCoder encodes Maps. + * + * @param the type of the keys of the KVs being transcoded + * @param the type of the values of the KVs being transcoded + */ +public class MapCoder extends MapCoderBase> { + + /** + * Produces a MapCoder with the given keyCoder and valueCoder. + */ + public static MapCoder of( + Coder keyCoder, + Coder valueCoder) { + return new MapCoder<>(keyCoder, valueCoder); + } + + @JsonCreator + public static MapCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> components) { + Preconditions.checkArgument(components.size() == 2, + "Expecting 2 components, got " + components.size()); + return of((Coder) components.get(0), (Coder) components.get(1)); + } + + /** + * Returns the key and value for an arbitrary element of this map, + * if it is non-empty, otherwise returns {@code null}. + */ + public static List getInstanceComponents( + Map exampleValue) { + for (Map.Entry entry : exampleValue.entrySet()) { + return Arrays.asList(entry.getKey(), entry.getValue()); + } + return null; + } + + public Coder getKeyCoder() { return keyCoder; } + public Coder getValueCoder() { return valueCoder; } + + ///////////////////////////////////////////////////////////////////////////// + + Coder keyCoder; + Coder valueCoder; + + MapCoder(Coder keyCoder, Coder valueCoder) { + this.keyCoder = keyCoder; + this.valueCoder = valueCoder; + } + + @Override + public void encode( + Map map, + OutputStream outStream, + Context context) + throws IOException, CoderException { + DataOutputStream dataOutStream = new DataOutputStream(outStream); + dataOutStream.writeInt(map.size()); + for (Entry entry : map.entrySet()) { + keyCoder.encode(entry.getKey(), outStream, context.nested()); + valueCoder.encode(entry.getValue(), outStream, context.nested()); + } + dataOutStream.flush(); + } + + @Override + public Map decode(InputStream inStream, Context context) + throws IOException, CoderException { + DataInputStream dataInStream = new DataInputStream(inStream); + int size = dataInStream.readInt(); + Map retval = new HashMap<>(); + for (int i = 0; i < size; ++i) { + K key = keyCoder.decode(inStream, context.nested()); + V value = valueCoder.decode(inStream, context.nested()); + retval.put(key, value); + } + return retval; + } + + @Override + public List> getCoderArguments() { + return Arrays.asList(keyCoder, valueCoder); + } + + /** + * Not all maps have a deterministic encoding. + * + *

For example, HashMap comparison does not depend on element order, so + * two HashMap instances may be equal but produce different encodings. + */ + @Override + public boolean isDeterministic() { + return false; + } + + /** + * Notifies ElementByteSizeObserver about the byte size of the + * encoded value using this coder. + */ + @Override + public void registerByteSizeObserver( + Map map, ElementByteSizeObserver observer, Context context) + throws Exception { + observer.update(4L); + for (Entry entry : map.entrySet()) { + keyCoder.registerByteSizeObserver( + entry.getKey(), observer, context.nested()); + valueCoder.registerByteSizeObserver( + entry.getValue(), observer, context.nested()); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoderBase.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoderBase.java new file mode 100644 index 0000000000000..e896e0d36dc14 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoderBase.java @@ -0,0 +1,52 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.util.PropertyNames; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +/** + * A abstract base class for MapCoder. Works around a Jackson2 bug tickled when building + * {@link MapCoder} directly (as of this writing, Jackson2 walks off the end of + * an array when it tries to deserialize a class with multiple generic type + * parameters). This should be removed in favor of a better workaround. + * @param the type of values being transcoded + */ +public abstract class MapCoderBase extends StandardCoder { + @JsonCreator + public static MapCoderBase of( + // N.B. typeId is a required parameter here, since a field named "@type" + // is presented to the deserializer as an input. + // + // If this method did not consume the field, Jackson2 would observe an + // unconsumed field and a returned value of a derived type. So Jackson2 + // would attempt to update the returned value with the unconsumed field + // data, The standard JsonDeserializer does not implement a mechanism for + // updating constructed values, so it would throw an exception, causing + // deserialization to fail. + @JsonProperty(value = "@type", required = false) String typeId, + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> components) { + return MapCoder.of(components); + } + + protected MapCoderBase() {} +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java new file mode 100644 index 0000000000000..c078e6629a2b7 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java @@ -0,0 +1,126 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.util.CloudObject; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * An encoder of {@link java.io.Serializable} objects. + * + * To use, specify the coder type on a PCollection. + *

+ * {@code
+ *   PCollection records =
+ *       foo.apply(...).setCoder(SerializableCoder.of(MyRecord.class));
+ * }
+ * 
+ * + *

SerializableCoder does not guarantee a deterministic encoding, as Java + * Serialization may produce different binary encodings for two equivalent + * objects. + * + * @param the type of elements handled by this coder + */ +public class SerializableCoder + extends AtomicCoder { + /** + * Returns a {@code SerializableCoder} instance for the provided element type. + * @param the element type + */ + public static SerializableCoder of(Class type) { + return new SerializableCoder<>(type); + } + + @JsonCreator + public static SerializableCoder of(@JsonProperty("type") String classType) + throws ClassNotFoundException { + Class clazz = Class.forName(classType); + if (!Serializable.class.isAssignableFrom(clazz)) { + throw new ClassNotFoundException( + "Class " + classType + " does not implement Serializable"); + } + return of((Class) clazz); + } + + private final Class type; + + protected SerializableCoder(Class type) { + this.type = type; + } + + public Class getRecordType() { + return type; + } + + @Override + public void encode(T value, OutputStream outStream, Context context) + throws IOException, CoderException { + if (value == null) { + throw new CoderException("cannot encode a null record"); + } + try (ObjectOutputStream oos = new ObjectOutputStream(outStream)) { + oos.writeObject(value); + } catch (IOException exn) { + throw new CoderException("unable to serialize record " + value, exn); + } + } + + @Override + public T decode(InputStream inStream, Context context) + throws IOException, CoderException { + try (ObjectInputStream ois = new ObjectInputStream(inStream)) { + return type.cast(ois.readObject()); + } catch (ClassNotFoundException e) { + throw new CoderException("unable to deserialize record", e); + } + } + + @Override + public CloudObject asCloudObject() { + CloudObject result = super.asCloudObject(); + result.put("type", type.getName()); + return result; + } + + @Override + public boolean isDeterministic() { + return false; + } + + @Override + public boolean equals(Object other) { + if (getClass() != other.getClass()) { + return false; + } + return type == ((SerializableCoder) other).type; + } + + // This coder inherits isRegisterByteSizeObserverCheap, + // getEncodedElementByteSize and registerByteSizeObserver + // from StandardCoder. Looks like we cannot do much better + // in this case. +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java new file mode 100644 index 0000000000000..1a234c7b40ed8 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java @@ -0,0 +1,124 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * A SetCoder encodes Sets. + * + * @param the type of the elements of the set + */ +public class SetCoder extends StandardCoder> { + + /** + * Produces a SetCoder with the given elementCoder. + */ + public static SetCoder of(Coder elementCoder) { + return new SetCoder<>(elementCoder); + } + + @JsonCreator + public static SetCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List components) { + Preconditions.checkArgument(components.size() == 1, + "Expecting 1 component, got " + components.size()); + return of((Coder) components.get(0)); + } + + public Coder getElementCoder() { return elementCoder; } + + ///////////////////////////////////////////////////////////////////////////// + + Coder elementCoder; + + SetCoder(Coder elementCoder) { + this.elementCoder = elementCoder; + } + + @Override + public void encode( + Set set, + OutputStream outStream, + Context context) + throws IOException, CoderException { + DataOutputStream dataOutStream = new DataOutputStream(outStream); + dataOutStream.writeInt(set.size()); + for (T element : set) { + elementCoder.encode(element, outStream, context.nested()); + } + dataOutStream.flush(); + } + + @Override + public Set decode(InputStream inStream, Context context) + throws IOException, CoderException { + DataInputStream dataInStream = new DataInputStream(inStream); + int size = dataInStream.readInt(); + Set retval = new HashSet(); + for (int i = 0; i < size; ++i) { + T element = elementCoder.decode(inStream, context.nested()); + retval.add(element); + } + return retval; + } + + @Override + public List> getCoderArguments() { + return Arrays.>asList(elementCoder); + } + + /** + * Not all sets have a deterministic encoding. + * + *

For example, HashSet comparison does not depend on element order, so + * two HashSet instances may be equal but produce different encodings. + */ + @Override + public boolean isDeterministic() { + return false; + } + + /** + * Notifies ElementByteSizeObserver about the byte size of the encoded value using this coder. + */ + @Override + public void registerByteSizeObserver( + Set set, ElementByteSizeObserver observer, Context context) + throws Exception { + observer.update(4L); + for (T element : set) { + elementCoder.registerByteSizeObserver(element, observer, context.nested()); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java new file mode 100644 index 0000000000000..7a35fdcafbf27 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java @@ -0,0 +1,143 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static com.google.cloud.dataflow.sdk.util.Structs.addList; + +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; + +import java.io.ByteArrayOutputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * A StandardCoder is one that defines equality, hashing, and printing + * via the class name and recursively using {@link #getComponents}. + * + * @param the type of the values being transcoded + */ +public abstract class StandardCoder implements Coder { + + protected StandardCoder() {} + + /** + * Returns the list of {@code Coder}s that are components of this + * {@code Coder}. Returns an empty list if this is an {@link AtomicCoder} (or + * other {@code Coder} with no components). + */ + public List> getComponents() { + List> coderArguments = getCoderArguments(); + if (coderArguments == null) { + return Collections.emptyList(); + } else { + return coderArguments; + } + } + + @Override + public boolean equals(Object o) { + if (this.getClass() != o.getClass()) { + return false; + } + StandardCoder that = (StandardCoder) o; + return this.getComponents().equals(that.getComponents()); + } + + @Override + public int hashCode() { + return getClass().hashCode() * 31 + getComponents().hashCode(); + } + + @Override + public String toString() { + String s = getClass().getName(); + s = s.substring(s.lastIndexOf('.') + 1); + List> componentCoders = getComponents(); + if (!componentCoders.isEmpty()) { + s += "("; + boolean first = true; + for (Coder componentCoder : componentCoders) { + if (first) { + first = false; + } else { + s += ", "; + } + s += componentCoder.toString(); + } + s += ")"; + } + return s; + } + + @Override + public CloudObject asCloudObject() { + CloudObject result = CloudObject.forClass(getClass()); + + List> components = getComponents(); + if (!components.isEmpty()) { + List cloudComponents = new ArrayList<>(components.size()); + for (Coder coder : components) { + cloudComponents.add(coder.asCloudObject()); + } + addList(result, PropertyNames.COMPONENT_ENCODINGS, cloudComponents); + } + + return result; + } + + /** + * StandardCoder requires elements to be fully encoded and copied + * into a byte stream to determine the byte size of the element, + * which is considered expensive. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(T value, Context context) { + return false; + } + + /** + * Returns the size in bytes of the encoded value using this + * coder. Derived classes override this method if byte size can be + * computed with less computation or copying. + */ + protected long getEncodedElementByteSize(T value, Context context) + throws Exception { + try { + ByteArrayOutputStream os = new ByteArrayOutputStream(); + encode(value, os, context); + return os.size(); + } catch (Exception exn) { + throw new IllegalArgumentException( + "Unable to encode element " + value + " with coder " + this, exn); + } + } + + /** + * Notifies ElementByteSizeObserver about the byte size of the + * encoded value using this coder. Calls + * getEncodedElementByteSize() and notifies ElementByteSizeObserver. + */ + @Override + public void registerByteSizeObserver( + T value, ElementByteSizeObserver observer, Context context) + throws Exception { + observer.update(getEncodedElementByteSize(value, context)); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java new file mode 100644 index 0000000000000..17995c31b65be --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java @@ -0,0 +1,124 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.util.VarInt; +import com.google.common.io.ByteStreams; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UTFDataFormatException; +import java.nio.charset.Charset; + +/** + * A StringUtf8Coder encodes Java Strings in UTF-8 encoding. + * If in a nested context, prefixes the string with a VarInt length field. + */ +public class StringUtf8Coder extends AtomicCoder { + @JsonCreator + public static StringUtf8Coder of() { + return INSTANCE; + } + + + ///////////////////////////////////////////////////////////////////////////// + + private static final StringUtf8Coder INSTANCE = new StringUtf8Coder(); + + private static class Singletons { + private static final Charset UTF8 = Charset.forName("UTF-8"); + } + + // Writes a string with VarInt size prefix, supporting large strings. + private static void writeString(String value, DataOutputStream dos) + throws IOException { + byte[] bytes = value.getBytes(Singletons.UTF8); + VarInt.encode(bytes.length, dos); + dos.write(bytes); + } + + // Reads a string with VarInt size prefix, supporting large strings. + private static String readString(DataInputStream dis) throws IOException { + int len = VarInt.decodeInt(dis); + if (len < 0) { + throw new CoderException("Invalid encoded string length: " + len); + } + byte[] bytes = new byte[len]; + dis.readFully(bytes); + return new String(bytes, Singletons.UTF8); + } + + private StringUtf8Coder() {} + + @Override + public void encode(String value, OutputStream outStream, Context context) + throws IOException { + if (value == null) { + throw new CoderException("cannot encode a null String"); + } + if (context.isWholeStream) { + outStream.write(value.getBytes(Singletons.UTF8)); + } else { + writeString(value, new DataOutputStream(outStream)); + } + } + + @Override + public String decode(InputStream inStream, Context context) + throws IOException { + if (context.isWholeStream) { + ByteArrayOutputStream outStream = new ByteArrayOutputStream(); + ByteStreams.copy(inStream, outStream); + // ByteArrayOutputStream.toString provides no Charset overloads. + return outStream.toString("UTF-8"); + } else { + try { + return readString(new DataInputStream(inStream)); + } catch (EOFException | UTFDataFormatException exn) { + // These exceptions correspond to decoding problems, so change + // what kind of exception they're branded as. + throw new CoderException(exn); + } + } + } + + @Override + public boolean isDeterministic() { + return true; + } + + protected long getEncodedElementByteSize(String value, Context context) + throws Exception { + if (value == null) { + throw new CoderException("cannot encode a null String"); + } + if (context.isWholeStream) { + return value.getBytes(Singletons.UTF8).length; + } else { + DataOutputStream stream = new DataOutputStream(new ByteArrayOutputStream()); + writeString(value, stream); + return stream.size(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java new file mode 100644 index 0000000000000..e49dfbb9c01c8 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java @@ -0,0 +1,80 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.api.services.bigquery.model.TableRow; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** + * A TableRowJsonCoder encodes BigQuery TableRow objects. + */ +public class TableRowJsonCoder extends AtomicCoder { + + @JsonCreator + public static TableRowJsonCoder of() { + return INSTANCE; + } + + @Override + public void encode(TableRow value, OutputStream outStream, Context context) + throws IOException { + String strValue = MAPPER.writeValueAsString(value); + StringUtf8Coder.of().encode(strValue, outStream, context); + } + + @Override + public TableRow decode(InputStream inStream, Context context) + throws IOException { + String strValue = StringUtf8Coder.of().decode(inStream, context); + return MAPPER.readValue(strValue, TableRow.class); + } + + @Override + protected long getEncodedElementByteSize(TableRow value, Context context) + throws Exception { + String strValue = MAPPER.writeValueAsString(value); + return StringUtf8Coder.of().getEncodedElementByteSize(strValue, context); + } + + ///////////////////////////////////////////////////////////////////////////// + + // FAIL_ON_EMPTY_BEANS is disabled in order to handle null values in + // TableRow. + private static final ObjectMapper MAPPER = + new ObjectMapper().disable(SerializationFeature.FAIL_ON_EMPTY_BEANS); + + private static final TableRowJsonCoder INSTANCE = new TableRowJsonCoder(); + + private TableRowJsonCoder() { + } + + /** + * TableCell can hold arbitrary Object instances, which makes the encoding + * non-deterministic. + */ + @Override + public boolean isDeterministic() { + return false; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java new file mode 100644 index 0000000000000..93d080b7f01cd --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java @@ -0,0 +1,73 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** + * A TextualIntegerCoder encodes Integers as text. + */ +public class TextualIntegerCoder extends AtomicCoder { + @JsonCreator + public static TextualIntegerCoder of() { + return new TextualIntegerCoder(); + } + + + ///////////////////////////////////////////////////////////////////////////// + + private TextualIntegerCoder() {} + + @Override + public void encode(Integer value, OutputStream outStream, Context context) + throws IOException, CoderException { + if (value == null) { + throw new CoderException("cannot encode a null Integer"); + } + String textualValue = value.toString(); + StringUtf8Coder.of().encode(textualValue, outStream, context); + } + + @Override + public Integer decode(InputStream inStream, Context context) + throws IOException, CoderException { + String textualValue = StringUtf8Coder.of().decode(inStream, context); + try { + return Integer.valueOf(textualValue); + } catch (NumberFormatException exn) { + throw new CoderException("error when decoding a textual integer", exn); + } + } + + @Override + public boolean isDeterministic() { + return true; + } + + protected long getEncodedElementByteSize(Integer value, Context context) + throws Exception { + if (value == null) { + throw new CoderException("cannot encode a null Integer"); + } + String textualValue = value.toString(); + return StringUtf8Coder.of().getEncodedElementByteSize(textualValue, context); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/URICoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/URICoder.java new file mode 100644 index 0000000000000..ed5ae45c53e77 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/URICoder.java @@ -0,0 +1,77 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.URI; +import java.net.URISyntaxException; + +/** + * A {@code URICoder} encodes/decodes {@link URI}s by conversion to/from {@link String}, delegating + * encoding/decoding of the string to {@link StringUtf8Coder}. + */ +public class URICoder extends AtomicCoder { + + @JsonCreator + public static URICoder of() { + return INSTANCE; + } + + private static final URICoder INSTANCE = new URICoder(); + private static final StringUtf8Coder STRING_CODER = StringUtf8Coder.of(); + + private URICoder() {} + + ///////////////////////////////////////////////////////////////////////////// + + @Override + public void encode(URI value, OutputStream outStream, Context context) + throws IOException { + if (value == null) { + throw new CoderException("cannot encode a null URI"); + } + STRING_CODER.encode(value.toString(), outStream, context); + } + + @Override + public URI decode(InputStream inStream, Context context) + throws IOException { + try { + return new URI(STRING_CODER.decode(inStream, context)); + } catch (URISyntaxException exn) { + throw new CoderException(exn); + } + } + + @Override + public boolean isDeterministic() { + return STRING_CODER.isDeterministic(); + } + + @Override + protected long getEncodedElementByteSize(URI value, Context context) + throws Exception { + if (value == null) { + throw new CoderException("cannot encode a null URI"); + } + return STRING_CODER.getEncodedElementByteSize(value.toString(), context); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java new file mode 100644 index 0000000000000..eff03fb737324 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java @@ -0,0 +1,90 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.util.VarInt; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UTFDataFormatException; + +/** + * A VarIntCoder encodes Integers using between 1 and 5 bytes. Negative + * numbers always take 5 bytes, so BigEndianIntegerCoder may be preferable for + * ints that are known to often be large or negative. + */ +public class VarIntCoder extends AtomicCoder { + @JsonCreator + public static VarIntCoder of() { + return INSTANCE; + } + + + ///////////////////////////////////////////////////////////////////////////// + + private static final VarIntCoder INSTANCE = + new VarIntCoder(); + + private VarIntCoder() {} + + @Override + public void encode(Integer value, OutputStream outStream, Context context) + throws IOException, CoderException { + if (value == null) { + throw new CoderException("cannot encode a null Integer"); + } + VarInt.encode(value.intValue(), outStream); + } + + @Override + public Integer decode(InputStream inStream, Context context) + throws IOException, CoderException { + try { + return VarInt.decodeInt(inStream); + } catch (EOFException | UTFDataFormatException exn) { + // These exceptions correspond to decoding problems, so change + // what kind of exception they're branded as. + throw new CoderException(exn); + } + } + + @Override + public boolean isDeterministic() { + return true; + } + + /** + * Returns true since registerByteSizeObserver() runs in constant time. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(Integer value, Context context) { + return true; + } + + @Override + protected long getEncodedElementByteSize(Integer value, Context context) + throws Exception { + if (value == null) { + throw new CoderException("cannot encode a null Integer"); + } + return VarInt.getLength(value.longValue()); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java new file mode 100644 index 0000000000000..74f9b6092288f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java @@ -0,0 +1,90 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.util.VarInt; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UTFDataFormatException; + +/** + * A VarLongCoder encodes longs using between 1 and 10 bytes. Negative + * numbers always take 10 bytes, so BigEndianLongCoder may be preferable for + * longs that are known to often be large or negative. + */ +public class VarLongCoder extends AtomicCoder { + @JsonCreator + public static VarLongCoder of() { + return INSTANCE; + } + + + ///////////////////////////////////////////////////////////////////////////// + + private static final VarLongCoder INSTANCE = + new VarLongCoder(); + + private VarLongCoder() {} + + @Override + public void encode(Long value, OutputStream outStream, Context context) + throws IOException, CoderException { + if (value == null) { + throw new CoderException("cannot encode a null Long"); + } + VarInt.encode(value.longValue(), outStream); + } + + @Override + public Long decode(InputStream inStream, Context context) + throws IOException, CoderException { + try { + return VarInt.decodeLong(inStream); + } catch (EOFException | UTFDataFormatException exn) { + // These exceptions correspond to decoding problems, so change + // what kind of exception they're branded as. + throw new CoderException(exn); + } + } + + @Override + public boolean isDeterministic() { + return true; + } + + /** + * Returns true since registerByteSizeObserver() runs in constant time. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(Long value, Context context) { + return true; + } + + @Override + protected long getEncodedElementByteSize(Long value, Context context) + throws Exception { + if (value == null) { + throw new CoderException("cannot encode a null Long"); + } + return VarInt.getLength(value.longValue()); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java new file mode 100644 index 0000000000000..fc9a1e0958b24 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.InputStream; +import java.io.OutputStream; + +/** + * A VoidCoder encodes Voids. Uses zero bytes per Void. + */ +public class VoidCoder extends AtomicCoder { + @JsonCreator + public static VoidCoder of() { + return INSTANCE; + } + + + ///////////////////////////////////////////////////////////////////////////// + + private static final VoidCoder INSTANCE = new VoidCoder(); + + private VoidCoder() {} + + @Override + public void encode(Void value, OutputStream outStream, Context context) { + // Nothing to write! + } + + @Override + public Void decode(InputStream inStream, Context context) { + // Nothing to read! + return null; + } + + @Override + public boolean isDeterministic() { + return true; + } + + /** + * Returns true since registerByteSizeObserver() runs in constant time. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(Void value, Context context) { + return true; + } + + @Override + protected long getEncodedElementByteSize(Void value, Context context) + throws Exception { + return 0; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/package-info.java new file mode 100644 index 0000000000000..ea305e776bc9a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/package-info.java @@ -0,0 +1,44 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Defines {@link com.google.cloud.dataflow.sdk.coders.Coder}s + * to specify how data is encoded to and decoded from byte strings. + * + *

During execution of a Pipeline, elements in a + * {@link com.google.cloud.dataflow.sdk.values.PCollection} + * may need to be encoded into byte strings. + * This happens both at the beginning and end of a pipeline when data is read from and written to + * persistent storage and also during execution of a pipeline when elements are communicated between + * machines. + * + *

Exactly when PCollection elements are encoded during execution depends on which + * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} is being used and how that runner + * chooses to execute the pipeline. As such, Dataflow requires that all PCollections have an + * appropriate Coder in case it becomes necessary. In many cases, the Coder can be inferred from + * the available Java type + * information and the Pipeline's {@link com.google.cloud.dataflow.sdk.coders.CoderRegistry}. It + * can be specified per PCollection via + * {@link com.google.cloud.dataflow.sdk.values.PCollection#setCoder(Coder)} or per type using the + * {@link com.google.cloud.dataflow.sdk.coders.DefaultCoder} annotation. + * + *

This package provides a number of coders for common types like {@code Integer}, + * {@code String}, and {@code List}, as well as coders like + * {@link com.google.cloud.dataflow.sdk.coders.AvroCoder} that can be used to encode many custom + * types. + * + */ +package com.google.cloud.dataflow.sdk.coders; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java new file mode 100644 index 0000000000000..7a9e6ea3d394c --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java @@ -0,0 +1,678 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import static com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.ValueWithMetadata; +import static com.google.cloud.dataflow.sdk.util.CloudSourceUtils.readElemsFromSource; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.worker.AvroSink; +import com.google.cloud.dataflow.sdk.runners.worker.AvroSource; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.cloud.dataflow.sdk.values.PInput; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.reflect.ReflectData; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Pattern; + +import javax.annotation.Nullable; + +/** + * Transforms for reading and writing Avro files. + * + *

To read a {@link PCollection} from one or more Avro files, use + * {@link AvroIO.Read}, specifying {@link AvroIO.Read#from} to specify + * the path of the file(s) to read from (e.g., a local filename or + * filename pattern if running locally, or a Google Cloud Storage + * filename or filename pattern of the form + * {@code "gs:///"}), and optionally + * {@link AvroIO.Read#named} to specify the name of the pipeline step. + * + *

It is required to specify {@link AvroIO.Read#withSchema}. To + * read specific records, such as Avro-generated classes, provide an + * Avro-generated class type. To read GenericRecords, provide either + * an org.apache.avro.Schema or a schema in a JSON-encoded string form. + * An exception will be thrown if a record doesn't match the specified + * schema. + * + *

For example: + *

 {@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection records =
+ *     p.apply(AvroIO.Read.from("/path/to/file.avro")
+ *                        .withSchema(AvroAutoGenClass.class));
+ *
+ * // A Read from a GCS file (runs locally and via the Google Cloud
+ * // Dataflow service):
+ * Schema schema = new Schema.Parser().parse(new File(
+ *     "gs://my_bucket/path/to/schema.avsc"));
+ * PCollection records =
+ *     p.apply(AvroIO.Read.named("ReadFromAvro")
+ *                        .from("gs://my_bucket/path/to/records-*.avro")
+ *                        .withSchema(schema));
+ * } 
+ * + *

To write a {@link PCollection} to one or more Avro files, use + * {@link AvroIO.Write}, specifying {@link AvroIO.Write#to} to specify + * the path of the file to write to (e.g., a local filename or sharded + * filename pattern if running locally, or a Google Cloud Storage + * filename or sharded filename pattern of the form + * {@code "gs:///"}), and optionally + * {@link AvroIO.Write#named} to specify the name of the pipeline step. + * + *

It is required to specify {@link AvroIO.Write#withSchema}. To + * write specific records, such as Avro-generated classes, provide an + * Avro-generated class type. To write GenericRecords, provide either + * an org.apache.avro.Schema or a schema in a JSON-encoded string form. + * An exception will be thrown if a record doesn't match the specified + * schema. + * + *

For example: + *

 {@code
+ * // A simple Write to a local file (only runs locally):
+ * PCollection records = ...;
+ * records.apply(AvroIO.Write.to("/path/to/file.avro")
+ *                           .withSchema(AvroAutoGenClass.class));
+ *
+ * // A Write to a sharded GCS file (runs locally and via the Google Cloud
+ * // Dataflow service):
+ * Schema schema = new Schema.Parser().parse(new File(
+ *     "gs://my_bucket/path/to/schema.avsc"));
+ * PCollection records = ...;
+ * records.apply(AvroIO.Write.named("WriteToAvro")
+ *                           .to("gs://my_bucket/path/to/numbers")
+ *                           .withSchema(schema)
+ *                           .withSuffix(".avro"));
+ * } 
+ */ +public class AvroIO { + + /** + * A root PTransform that reads from an Avro file (or multiple Avro + * files matching a pattern) and returns a PCollection containing + * the decoding of each record. + */ + public static class Read { + + /** + * Returns an AvroIO.Read PTransform with the given step name. + */ + public static Bound named(String name) { + return new Bound<>(GenericRecord.class).named(name); + } + + /** + * Returns an AvroIO.Read PTransform that reads from the file(s) + * with the given name or pattern. This can be a local filename + * or filename pattern (if running locally), or a Google Cloud + * Storage filename or filename pattern of the form + * {@code "gs:///"}) (if running locally or via + * the Google Cloud Dataflow service). Standard + * Java Filesystem glob patterns ("*", "?", "[..]") are supported. + */ + public static Bound from(String filepattern) { + return new Bound<>(GenericRecord.class).from(filepattern); + } + + /** + * Returns an AvroIO.Read PTransform that reads Avro file(s) + * containing records whose type is the specified Avro-generated class. + * + * @param the type of the decoded elements, and the elements + * of the resulting PCollection + */ + public static Bound withSchema(Class type) { + return new Bound<>(type).withSchema(type); + } + + /** + * Returns an AvroIO.Read PTransform that reads Avro file(s) + * containing records of the specified schema. + */ + public static Bound withSchema(Schema schema) { + return new Bound<>(GenericRecord.class).withSchema(schema); + } + + /** + * Returns an AvroIO.Read PTransform that reads Avro file(s) + * containing records of the specified schema in a JSON-encoded + * string form. + */ + public static Bound withSchema(String schema) { + return withSchema((new Schema.Parser()).parse(schema)); + } + + /** + * A PTransform that reads from an Avro file (or multiple Avro + * files matching a pattern) and returns a bounded PCollection containing + * the decoding of each record. + * + * @param the type of each of the elements of the resulting + * PCollection + */ + public static class Bound + extends PTransform> { + private static final long serialVersionUID = 0; + + /** The filepattern to read from. */ + @Nullable final String filepattern; + /** The class type of the records. */ + final Class type; + /** The schema of the input file. */ + @Nullable final Schema schema; + + Bound(Class type) { + this(null, null, type, null); + } + + Bound(String name, String filepattern, Class type, Schema schema) { + super(name); + this.filepattern = filepattern; + this.type = type; + this.schema = schema; + } + + /** + * Returns a new AvroIO.Read PTransform that's like this one but + * with the given step name. Does not modify this object. + */ + public Bound named(String name) { + return new Bound<>(name, filepattern, type, schema); + } + + /** + * Returns a new AvroIO.Read PTransform that's like this one but + * that reads from the file(s) with the given name or pattern. + * (See {@link AvroIO.Read#from} for a description of + * filepatterns.) Does not modify this object. + */ + public Bound from(String filepattern) { + return new Bound<>(name, filepattern, type, schema); + } + + /** + * Returns a new AvroIO.Read PTransform that's like this one but + * that reads Avro file(s) containing records whose type is the + * specified Avro-generated class. Does not modify this object. + * + * @param the type of the decoded elements, and the elements of + * the resulting PCollection + */ + public Bound withSchema(Class type) { + return new Bound<>(name, filepattern, type, ReflectData.get().getSchema(type)); + } + + /** + * Returns a new AvroIO.Read PTransform that's like this one but + * that reads Avro file(s) containing records of the specified schema. + * Does not modify this object. + */ + public Bound withSchema(Schema schema) { + return new Bound<>(name, filepattern, GenericRecord.class, schema); + } + + /** + * Returns a new AvroIO.Read PTransform that's like this one but + * that reads Avro file(s) containing records of the specified schema + * in a JSON-encoded string form. Does not modify this object. + */ + public Bound withSchema(String schema) { + return withSchema((new Schema.Parser()).parse(schema)); + } + + @Override + public PCollection apply(PInput input) { + if (filepattern == null) { + throw new IllegalStateException( + "need to set the filepattern of an AvroIO.Read transform"); + } + if (schema == null) { + throw new IllegalStateException( + "need to set the schema of an AvroIO.Read transform"); + } + + // Force the output's Coder to be what the read is using, and + // unchangeable later, to ensure that we read the input in the + // format specified by the Read transform. + return PCollection.createPrimitiveOutputInternal(new GlobalWindow()) + .setCoder(getDefaultOutputCoder()); + } + + @Override + protected Coder getDefaultOutputCoder() { + return AvroCoder.of(type, schema); + } + + @Override + protected String getKindString() { return "AvroIO.Read"; } + + public String getFilepattern() { + return filepattern; + } + + public Schema getSchema() { + return schema; + } + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + Bound.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + Bound transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateReadHelper(transform, context); + } + }); + } + } + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * A root PTransform that writes a PCollection to an Avro file (or + * multiple Avro files matching a sharding pattern). + */ + public static class Write { + + /** + * Returns an AvroIO.Write PTransform with the given step name. + */ + public static Bound named(String name) { + return new Bound<>(GenericRecord.class).named(name); + } + + /** + * Returns an AvroIO.Write PTransform that writes to the file(s) + * with the given prefix. This can be a local filename + * (if running locally), or a Google Cloud Storage filename of + * the form {@code "gs:///"}) + * (if running locally or via the Google Cloud Dataflow service). + * + *

The files written will begin with this prefix, followed by + * a shard identifier (see {@link Bound#withNumShards}, and end + * in a common extension, if given by {@link Bound#withSuffix}. + */ + public static Bound to(String prefix) { + return new Bound<>(GenericRecord.class).to(prefix); + } + + /** + * Returns an AvroIO.Write PTransform that writes to the file(s) with the + * given filename suffix. + */ + public static Bound withSuffix(String filenameSuffix) { + return new Bound<>(GenericRecord.class).withSuffix(filenameSuffix); + } + + /** + * Returns an AvroIO.Write PTransform that uses the provided shard count. + * + *

Constraining the number of shards is likely to reduce + * the performance of a pipeline. Setting this value is not recommended + * unless you require a specific number of output files. + * + * @param numShards the number of shards to use, or 0 to let the system + * decide. + */ + public static Bound withNumShards(int numShards) { + return new Bound<>(GenericRecord.class).withNumShards(numShards); + } + + /** + * Returns an AvroIO.Write PTransform that uses the given shard name + * template. + * + * See {@link ShardNameTemplate} for a description of shard templates. + */ + public static Bound withShardNameTemplate(String shardTemplate) { + return new Bound<>(GenericRecord.class).withShardNameTemplate(shardTemplate); + } + + /** + * Returns an AvroIO.Write PTransform that forces a single file as + * output. + * + *

Constraining the number of shards is likely to reduce + * the performance of a pipeline. Setting this value is not recommended + * unless you require a specific number of output files. + */ + public static Bound withoutSharding() { + return new Bound<>(GenericRecord.class).withoutSharding(); + } + + /** + * Returns an AvroIO.Write PTransform that writes Avro file(s) + * containing records whose type is the specified Avro-generated class. + * + * @param the type of the elements of the input PCollection + */ + public static Bound withSchema(Class type) { + return new Bound<>(type).withSchema(type); + } + + /** + * Returns an AvroIO.Write PTransform that writes Avro file(s) + * containing records of the specified schema. + */ + public static Bound withSchema(Schema schema) { + return new Bound<>(GenericRecord.class).withSchema(schema); + } + + /** + * Returns an AvroIO.Write PTransform that writes Avro file(s) + * containing records of the specified schema in a JSON-encoded + * string form. + */ + public static Bound withSchema(String schema) { + return withSchema((new Schema.Parser()).parse(schema)); + } + + /** + * A PTransform that writes a bounded PCollection to an Avro file (or + * multiple Avro files matching a sharding pattern). + * + * @param the type of each of the elements of the input PCollection + */ + public static class Bound + extends PTransform, PDone> { + private static final long serialVersionUID = 0; + + /** The filename to write to. */ + @Nullable final String filenamePrefix; + /** Suffix to use for each filename. */ + final String filenameSuffix; + /** Requested number of shards. 0 for automatic. */ + final int numShards; + /** Shard template string. */ + final String shardTemplate; + /** The class type of the records. */ + final Class type; + /** The schema of the output file. */ + @Nullable final Schema schema; + + Bound(Class type) { + this(null, null, "", 0, ShardNameTemplate.INDEX_OF_MAX, type, null); + } + + Bound(String name, String filenamePrefix, String filenameSuffix, + int numShards, String shardTemplate, + Class type, Schema schema) { + super(name); + this.filenamePrefix = filenamePrefix; + this.filenameSuffix = filenameSuffix; + this.numShards = numShards; + this.shardTemplate = shardTemplate; + this.type = type; + this.schema = schema; + } + + /** + * Returns a new AvroIO.Write PTransform that's like this one but + * with the given step name. Does not modify this object. + */ + public Bound named(String name) { + return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, + type, schema); + } + + /** + * Returns a new AvroIO.Write PTransform that's like this one but + * that writes to the file(s) with the given filename prefix. + * + *

See {@link Write#to(String) Write.to(String)} for more information. + * + *

Does not modify this object. + */ + public Bound to(String filenamePrefix) { + validateOutputComponent(filenamePrefix); + return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, + type, schema); + } + + /** + * Returns a new AvroIO.Write PTransform that's like this one but + * that writes to the file(s) with the given filename suffix. + * + *

Does not modify this object. + * + * @see ShardNameTemplate + */ + public Bound withSuffix(String filenameSuffix) { + validateOutputComponent(filenameSuffix); + return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, + type, schema); + } + + /** + * Returns a new AvroIO.Write PTransform that's like this one but + * that uses the provided shard count. + * + *

Constraining the number of shards is likely to reduce + * the performance of a pipeline. Setting this value is not recommended + * unless you require a specific number of output files. + * + *

Does not modify this object. + * + * @param numShards the number of shards to use, or 0 to let the system + * decide. + * @see ShardNameTemplate + */ + public Bound withNumShards(int numShards) { + Preconditions.checkArgument(numShards >= 0); + return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, + type, schema); + } + + /** + * Returns a new AvroIO.Write PTransform that's like this one but + * that uses the given shard name template. + * + *

Does not modify this object. + * + * @see ShardNameTemplate + */ + public Bound withShardNameTemplate(String shardTemplate) { + return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, + type, schema); + } + + /** + * Returns a new AvroIO.Write PTransform that's like this one but + * that forces a single file as output. + * + *

This is a shortcut for + * {@code .withNumShards(1).withShardNameTemplate("")} + * + *

Does not modify this object. + */ + public Bound withoutSharding() { + return new Bound<>(name, filenamePrefix, filenameSuffix, 1, "", type, schema); + } + + /** + * Returns a new AvroIO.Write PTransform that's like this one but + * that writes to Avro file(s) containing records whose type is the + * specified Avro-generated class. Does not modify this object. + * + * @param the type of the elements of the input PCollection + */ + public Bound withSchema(Class type) { + return new Bound<>(name, filenamePrefix, filenameSuffix, + numShards, shardTemplate, + type, ReflectData.get().getSchema(type)); + } + + /** + * Returns a new AvroIO.Write PTransform that's like this one but + * that writes to Avro file(s) containing records of the specified + * schema. Does not modify this object. + */ + public Bound withSchema(Schema schema) { + return new Bound<>(name, filenamePrefix, filenameSuffix, + numShards, shardTemplate, + GenericRecord.class, schema); + } + + /** + * Returns a new AvroIO.Write PTransform that's like this one but + * that writes to Avro file(s) containing records of the specified + * schema in a JSON-encoded string form. Does not modify this object. + */ + public Bound withSchema(String schema) { + return withSchema((new Schema.Parser()).parse(schema)); + } + + @Override + public PDone apply(PCollection input) { + if (filenamePrefix == null) { + throw new IllegalStateException( + "need to set the filename prefix of an AvroIO.Write transform"); + } + if (schema == null) { + throw new IllegalStateException( + "need to set the schema of an AvroIO.Write transform"); + } + + return new PDone(); + } + + /** + * Returns the current shard name template string. + */ + public String getShardNameTemplate() { + return shardTemplate; + } + + @Override + protected Coder getDefaultOutputCoder() { + return VoidCoder.of(); + } + + @Override + protected String getKindString() { return "AvroIO.Write"; } + + public String getFilenamePrefix() { + return filenamePrefix; + } + + public String getShardTemplate() { + return shardTemplate; + } + + public int getNumShards() { + return numShards; + } + + public String getFilenameSuffix() { + return filenameSuffix; + } + + public Class getType() { + return type; + } + + public Schema getSchema() { + return schema; + } + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + Bound.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + Bound transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateWriteHelper(transform, context); + } + }); + } + } + } + + // Pattern which matches old-style shard output patterns, which are now + // disallowed. + private static final Pattern SHARD_OUTPUT_PATTERN = + Pattern.compile("@([0-9]+|\\*)"); + + private static void validateOutputComponent(String partialFilePattern) { + Preconditions.checkArgument( + !SHARD_OUTPUT_PATTERN.matcher(partialFilePattern).find(), + "Output name components are not allowed to contain @* or @N patterns: " + + partialFilePattern); + } + + ///////////////////////////////////////////////////////////////////////////// + + private static void evaluateReadHelper( + Read.Bound transform, + DirectPipelineRunner.EvaluationContext context) { + AvroSource source = new AvroSource<>( + transform.filepattern, null, null, WindowedValue.getValueOnlyCoder( + transform.getDefaultOutputCoder())); + List> elems = readElemsFromSource(source); + List> output = new ArrayList<>(); + for (WindowedValue elem : elems) { + output.add(ValueWithMetadata.of(elem)); + } + context.setPCollectionValuesWithMetadata(transform.getOutput(), output); + } + + private static void evaluateWriteHelper( + Write.Bound transform, + DirectPipelineRunner.EvaluationContext context) { + List> elems = context.getPCollectionWindowedValues(transform.getInput()); + int numShards = transform.numShards; + if (numShards < 1) { + // System gets to choose. For direct mode, choose 1. + numShards = 1; + } + AvroSink writer = new AvroSink<>(transform.filenamePrefix, transform.shardTemplate, + transform.filenameSuffix, numShards, + WindowedValue.getValueOnlyCoder( + AvroCoder.of(transform.type, transform.schema))); + try (Sink.SinkWriter> sink = writer.writer()) { + for (WindowedValue elem : elems) { + sink.add(elem); + } + } catch (IOException exn) { + throw new RuntimeException( + "unable to write to output file \"" + transform.filenamePrefix + "\"", + exn); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java new file mode 100644 index 0000000000000..2fffe4de2c45b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java @@ -0,0 +1,937 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import com.google.api.client.json.JsonFactory; +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.options.BigQueryOptions; +import com.google.cloud.dataflow.sdk.options.GcpOptions; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.worker.BigQuerySource; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.util.BigQueryTableInserter; +import com.google.cloud.dataflow.sdk.util.CloudSourceUtils; +import com.google.cloud.dataflow.sdk.util.Transport; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.cloud.dataflow.sdk.values.PInput; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Transformations for reading and writing + * BigQuery tables. + *

Table References

+ * A fully-qualified BigQuery table name consists of three components: + *
    + *
  • {@code projectId}: the Cloud project id (defaults to + * {@link GcpOptions#getProject()}). + *
  • {@code datasetId}: the BigQuery dataset id, unique within a project. + *
  • {@code tableId}: a table id, unique within a dataset. + *
+ *

+ * BigQuery table references are stored as a {@link TableReference}, which comes + * from the BigQuery Java Client API. + * Tables can be referred to as Strings, with or without the {@code projectId}. + * A helper function is provided ({@link BigQueryIO#parseTableSpec(String)}), + * which parses the following string forms into a {@link TableReference}: + *

    + *
  • [{@code project_id}]:[{@code dataset_id}].[{@code table_id}] + *
  • [{@code dataset_id}].[{@code table_id}] + *
+ *

Reading

+ * To read from a BigQuery table, apply a {@link BigQueryIO.Read} transformation. + * This produces a {@code PCollection} as output: + *
{@code
+ * PCollection shakespeare = pipeline.apply(
+ *     BigQueryIO.Read
+ *         .named("Read")
+ *         .from("clouddataflow-readonly:samples.weather_stations");
+ * }
+ *

Writing

+ * To write to a BigQuery table, apply a {@link BigQueryIO.Write} transformation. + * This consumes a {@code PCollection} as input. + *

+ *

{@code
+ * PCollection quotes = ...
+ *
+ * List fields = new ArrayList<>();
+ * fields.add(new TableFieldSchema().setName("source").setType("STRING"));
+ * fields.add(new TableFieldSchema().setName("quote").setType("STRING"));
+ * TableSchema schema = new TableSchema().setFields(fields);
+ *
+ * quotes.apply(BigQueryIO.Write
+ *     .named("Write")
+ *     .to("my-project:output.output_table")
+ *     .withSchema(schema)
+ *     .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
+ * }
+ *

+ * See {@link BigQueryIO.Write} for details on how to specify if a write should + * append to an existing table, replace the table, or verify that the table is + * empty. + * + * @see TableRow + */ +public class BigQueryIO { + private static final Logger LOG = LoggerFactory.getLogger(BigQueryIO.class); + + /** + * Singleton instance of the JSON factory used to read and write JSON + * formatted rows. + */ + private static final JsonFactory JSON_FACTORY = Transport.getJsonFactory(); + + /** + * Project IDs must contain 6-63 lowercase letters, digits, or dashes. + * IDs must start with a letter and may not end with a dash. + * This regex isn't exact - this allows for patterns that would be rejected by + * the service, but this is sufficient for basic parsing of table references. + */ + private static final String PROJECT_ID_REGEXP = + "[a-z][-a-z0-9:.]{4,61}[a-z0-9]"; + + /** + * Regular expression which matches Dataset IDs. + */ + private static final String DATASET_REGEXP = "[-\\w.]{1,1024}"; + + /** + * Regular expression which matches Table IDs. + */ + private static final String TABLE_REGEXP = "[-\\w$@]{1,1024}"; + + /** + * Matches table specifications in the form + * "[project_id]:[dataset_id].[table_id]" or "[dataset_id].[table_id]". + */ + private static final String DATASET_TABLE_REGEXP = String.format( + "((?%s):)?(?%s)\\.(?%s)", + PROJECT_ID_REGEXP, DATASET_REGEXP, TABLE_REGEXP); + + private static final Pattern TABLE_SPEC = + Pattern.compile(DATASET_TABLE_REGEXP); + + /** + * Parse a table specification in the form + * "[project_id]:[dataset_id].[table_id]" or "[dataset_id].[table_id]". + *

+ * If the project id is omitted, the default project id is used. + */ + public static TableReference parseTableSpec(String tableSpec) { + Matcher match = TABLE_SPEC.matcher(tableSpec); + if (!match.matches()) { + throw new IllegalArgumentException( + "Table reference is not in [project_id]:[dataset_id].[table_id] " + + "format: " + tableSpec); + } + + TableReference ref = new TableReference(); + ref.setProjectId(match.group("PROJECT")); + + return ref + .setDatasetId(match.group("DATASET")) + .setTableId(match.group("TABLE")); + } + + /** + * Returns a canonical string representation of the TableReference. + */ + public static String toTableSpec(TableReference ref) { + StringBuilder sb = new StringBuilder(); + if (ref.getProjectId() != null) { + sb.append(ref.getProjectId()); + sb.append(":"); + } + + sb.append(ref.getDatasetId()) + .append('.') + .append(ref.getTableId()); + return sb.toString(); + } + + /** + * A PTransform that reads from a BigQuery table and returns a + * {@code PCollection} containing each of the rows of the table. + *

+ * Each TableRow record contains values indexed by column name. Here is a + * sample processing function which processes a "line" column from rows: + *


+   * static class ExtractWordsFn extends DoFn{@literal } {
+   *   {@literal @}Override
+   *   public void processElement(ProcessContext c) {
+   *     // Get the "line" field of the TableRow object, split it into words, and emit them.
+   *     TableRow row = c.element();
+   *     String[] words = row.get("line").toString().split("[^a-zA-Z']+");
+   *     for (String word : words) {
+   *       if (!word.isEmpty()) {
+   *         c.output(word);
+   *       }
+   *     }
+   *   }
+   * }
+   * 
+ */ + public static class Read { + public static Bound named(String name) { + return new Bound().named(name); + } + + /** + * Reads a BigQuery table specified as + * "[project_id]:[dataset_id].[table_id]" or "[dataset_id].[table_id]" for + * tables within the current project. + */ + public static Bound from(String tableSpec) { + return new Bound().from(tableSpec); + } + + /** + * Reads a BigQuery table specified as a TableReference object. + */ + public static Bound from(TableReference table) { + return new Bound().from(table); + } + + /** + * Disables BigQuery table validation which is enabled by default. + */ + public static Bound withoutValidation() { + return new Bound().withoutValidation(); + } + + /** + * A PTransform that reads from a BigQuery table and returns a bounded + * {@code PCollection}. + */ + public static class Bound + extends PTransform> { + TableReference table; + final boolean validate; + + Bound() { + this.validate = true; + } + + Bound(String name, TableReference reference, boolean validate) { + super(name); + this.table = reference; + this.validate = validate; + } + + /** + * Sets the name associated with this transformation. + */ + public Bound named(String name) { + return new Bound(name, table, validate); + } + + /** + * Sets the table specification. + *

+ * Refer to {@link #parseTableSpec(String)} for the specification format. + */ + public Bound from(String tableSpec) { + return from(parseTableSpec(tableSpec)); + } + + /** + * Sets the table specification. + */ + public Bound from(TableReference table) { + return new Bound(name, table, validate); + } + + /** + * Disable table validation. + */ + public Bound withoutValidation() { + return new Bound(name, table, false); + } + + @Override + public PCollection apply(PInput input) { + if (table == null) { + throw new IllegalStateException( + "must set the table reference of a BigQueryIO.Read transform"); + } + return PCollection.createPrimitiveOutputInternal( + new GlobalWindow()) + // Force the output's Coder to be what the read is using, and + // unchangeable later, to ensure that we read the input in the + // format specified by the Read transform. + .setCoder(TableRowJsonCoder.of()); + } + + @Override + protected Coder getDefaultOutputCoder() { + return TableRowJsonCoder.of(); + } + + @Override + protected String getKindString() { return "BigQueryIO.Read"; } + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + Bound.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + Bound transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateReadHelper(transform, context); + } + }); + } + + /** + * Returns the table to write. + */ + public TableReference getTable() { + return table; + } + + /** + * Returns true if table validation is enabled. + */ + public boolean getValidate() { + return validate; + } + } + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * A PTransform that writes a {@code PCollection} containing rows + * to a BigQuery table. + *

+ * By default, tables will be created if they do not exist, which + * corresponds to a {@code CreateDisposition.CREATE_IF_NEEDED} disposition + * which matches the default of BigQuery's Jobs API. A schema must be + * provided (via {@link Write#withSchema}), or else the transform may fail + * at runtime with an {@link java.lang.IllegalArgumentException}. + *

+ * By default, writes require an empty table, which corresponds to + * a {@code WriteDisposition.WRITE_EMPTY} disposition which matches the + * default of BigQuery's Jobs API. + *

+ * Here is a sample transform which produces TableRow values containing + * "word" and "count" columns: + *


+   * static class FormatCountsFn extends DoFnP{@literal , TableRow>} {
+   *   {@literal @}Override
+   *   public void processElement(ProcessContext c) {
+   *     TableRow row = new TableRow()
+   *         .set("word", c.element().getKey())
+   *         .set("count", c.element().getValue().intValue());
+   *     c.output(row);
+   *   }
+   * }
+   * 
+ */ + public static class Write { + + /** + * An enumeration type for the BigQuery create disposition strings publicly + * documented as {@code CREATE_NEVER}, and {@code CREATE_IF_NEEDED}. + */ + public enum CreateDisposition { + /** + * Specifics that tables should not be created. + *

+ * If the output table does not exist, the write fails. + */ + CREATE_NEVER, + + /** + * Specifies that tables should be created if needed. This is the default + * behavior. + *

+ * Requires that a table schema is provided via {@link Write#withSchema}. + * This precondition is checked before starting a job. The schema is + * not required to match an existing table's schema. + *

+ * When this transformation is executed, if the output table does not + * exist, the table is created from the provided schema. Note that even if + * the table exists, it may be recreated if necessary when paired with a + * {@link WriteDisposition#WRITE_TRUNCATE}. + */ + CREATE_IF_NEEDED + } + + /** + * An enumeration type for the BigQuery write disposition strings publicly + * documented as {@code WRITE_TRUNCATE}, {@code WRITE_APPEND}, and + * {@code WRITE_EMPTY}. + */ + public enum WriteDisposition { + /** + * Specifies that write should replace a table. + *

+ * The replacement may occur in multiple steps - for instance by first + * removing the existing table, then creating a replacement, then filling + * it in. This is not an atomic operation, and external programs may + * see the table in any of these intermediate steps. + */ + WRITE_TRUNCATE, + + /** + * Specifies that rows may be appended to an existing table. + */ + WRITE_APPEND, + + /** + * Specifies that the output table must be empty. This is the default + * behavior. + *

+ * If the output table is not empty, the write fails at runtime. + *

+ * This check may occur long before data is written, and does not + * guarantee exclusive access to the table. If two programs are run + * concurrently, each specifying the same output table and + * a {@link WriteDisposition} of {@code WRITE_EMPTY}, it is possible + * for both to succeed. + */ + WRITE_EMPTY + } + + /** + * Sets the name associated with this transformation. + */ + public static Bound named(String name) { + return new Bound().named(name); + } + + /** + * Creates a write transformation for the given table specification. + *

+ * Refer to {@link #parseTableSpec(String)} for the specification format. + */ + public static Bound to(String tableSpec) { + return new Bound().to(tableSpec); + } + + /** Creates a write transformation for the given table. */ + public static Bound to(TableReference table) { + return new Bound().to(table); + } + + /** + * Specifies a table schema to use in table creation. + *

+ * The schema is required only if writing to a table which does not already + * exist, and {@link BigQueryIO.Write.CreateDisposition} is set to + * {@code CREATE_IF_NEEDED}. + */ + public static Bound withSchema(TableSchema schema) { + return new Bound().withSchema(schema); + } + + /** Specifies options for creating the table. */ + public static Bound withCreateDisposition(CreateDisposition disposition) { + return new Bound().withCreateDisposition(disposition); + } + + /** Specifies options for writing to the table. */ + public static Bound withWriteDisposition(WriteDisposition disposition) { + return new Bound().withWriteDisposition(disposition); + } + + /** + * Disables BigQuery table validation which is enabled by default. + */ + public static Bound withoutValidation() { + return new Bound().withoutValidation(); + } + + /** + * A PTransform that can write either a bounded or unbounded + * {@code PCollection}s to a BigQuery table. + */ + public static class Bound + extends PTransform, PDone> { + final TableReference table; + + // Table schema. The schema is required only if the table does not exist. + final TableSchema schema; + + // Options for creating the table. Valid values are CREATE_IF_NEEDED and + // CREATE_NEVER. + final CreateDisposition createDisposition; + + // Options for writing to the table. Valid values are WRITE_TRUNCATE, + // WRITE_APPEND and WRITE_EMPTY. + final WriteDisposition writeDisposition; + + // An option to indicate if table validation is desired. Default is true. + final boolean validate; + + public Bound() { + this.table = null; + this.schema = null; + this.createDisposition = CreateDisposition.CREATE_IF_NEEDED; + this.writeDisposition = WriteDisposition.WRITE_EMPTY; + this.validate = true; + } + + Bound(String name, TableReference ref, TableSchema schema, + CreateDisposition createDisposition, + WriteDisposition writeDisposition, + boolean validate) { + super(name); + this.table = ref; + this.schema = schema; + this.createDisposition = createDisposition; + this.writeDisposition = writeDisposition; + this.validate = validate; + } + + /** + * Sets the name associated with this transformation. + */ + public Bound named(String name) { + return new Bound(name, table, schema, createDisposition, + writeDisposition, validate); + } + + /** + * Specifies the table specification. + *

+ * Refer to {@link #parseTableSpec(String)} for the specification format. + */ + public Bound to(String tableSpec) { + return to(parseTableSpec(tableSpec)); + } + + /** + * Specifies the table to be written to. + */ + public Bound to(TableReference table) { + return new Bound(name, table, schema, createDisposition, + writeDisposition, validate); + } + + /** + * Specifies the table schema, used if the table is created. + */ + public Bound withSchema(TableSchema schema) { + return new Bound(name, table, schema, createDisposition, + writeDisposition, validate); + } + + /** Specifies options for creating the table. */ + public Bound withCreateDisposition(CreateDisposition createDisposition) { + return new Bound(name, table, schema, createDisposition, + writeDisposition, validate); + } + + /** Specifies options for writing the table. */ + public Bound withWriteDisposition(WriteDisposition writeDisposition) { + return new Bound(name, table, schema, createDisposition, + writeDisposition, validate); + } + + /** + * Disable table validation. + */ + public Bound withoutValidation() { + return new Bound(name, table, schema, createDisposition, writeDisposition, false); + } + + @Override + public PDone apply(PCollection input) { + if (table == null) { + throw new IllegalStateException( + "must set the table reference of a BigQueryIO.Write transform"); + } + + if (createDisposition == CreateDisposition.CREATE_IF_NEEDED && + schema == null) { + throw new IllegalArgumentException( + "CreateDisposition is CREATE_IF_NEEDED, " + + "however no schema was provided."); + } + + // In streaming, BigQuery write is taken care of by StreamWithDeDup transform. + BigQueryOptions options = getPipeline().getOptions().as(BigQueryOptions.class); + if (options.isStreaming()) { + return input.apply(new StreamWithDeDup(table, schema)); + } + + return new PDone(); + } + + @Override + protected Coder getDefaultOutputCoder() { + return VoidCoder.of(); + } + + @Override + protected String getKindString() { return "BigQueryIO.Write"; } + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + Bound.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + Bound transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateWriteHelper(transform, context); + } + }); + } + + /** Returns the create disposition. */ + public CreateDisposition getCreateDisposition() { + return createDisposition; + } + + /** Returns the write disposition. */ + public WriteDisposition getWriteDisposition() { + return writeDisposition; + } + + /** Returns the table schema. */ + public TableSchema getSchema() { + return schema; + } + + /** Returns the table reference. */ + public TableReference getTable() { + return table; + } + + /** Returns true if table validation is enabled. */ + public boolean getValidate() { + return validate; + } + } + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Implementation of DoFn to perform streaming BigQuery write. + */ + private static class StreamingWriteFn extends DoFn>, Void> + implements DoFn.RequiresKeyedState { + + /** + * Class to accumulate BigQuery row data as a list of String. + * DoFn implementation must be Serializable, but BigQuery classes, + * such as TableRow are not. Therefore, convert into JSON String + * for accumulation. + */ + private static class JsonTableRows implements Iterable, Serializable { + + /** The list where BigQuery row data is accumulated. */ + private final List jsonRows = new ArrayList<>(); + + /** Iterator of JsonTableRows converts the row in String to TableRow. */ + static class JsonTableRowIterator implements Iterator { + + private final Iterator iteratorInternal; + + /** Constructor. */ + JsonTableRowIterator(List jsonRowList) { + iteratorInternal = jsonRowList.iterator(); + } + + @Override + public boolean hasNext() { + return iteratorInternal.hasNext(); + } + + @Override + public TableRow next() { + try { + // Converts the String back into TableRow. + return JSON_FACTORY.fromString(iteratorInternal.next(), TableRow.class); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void remove() { + iteratorInternal.remove(); + } + } + + /** Returns the iterator. */ + @Override + public Iterator iterator() { + return new JsonTableRowIterator(jsonRows); + } + + /** Adds a BigQuery TableRow. */ + void add(TableRow row) { + try { + // Converts into JSON format. + jsonRows.add(JSON_FACTORY.toString(row)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + /** TableReference in JSON. Use String to make the class Serializable. */ + private final String jsonTableReference; + + /** TableSchema in JSON. Use String to make the class Serializable. */ + private final String jsonTableSchema; + + /** JsonTableRows to accumulate BigQuery rows. */ + private JsonTableRows jsonTableRows; + + /** The list of unique ids for each BigQuery table row. */ + private List uniqueIdsForTableRows; + + /** The list of tables created so far, so we don't try the creation + each time. */ + private static ThreadLocal> createdTables = + new ThreadLocal>() { + @Override protected HashSet initialValue() { + return new HashSet<>(); + } + }; + + /** Constructor. */ + StreamingWriteFn(TableReference table, TableSchema schema) { + try { + jsonTableReference = JSON_FACTORY.toString(table); + jsonTableSchema = JSON_FACTORY.toString(schema); + } catch (IOException e) { + throw new RuntimeException("Cannot initialize BigQuery streaming writer.", e); + } + } + + /** Prepares a target BigQuery table. */ + @Override + public void startBundle(Context context) { + jsonTableRows = new JsonTableRows(); + uniqueIdsForTableRows = new ArrayList<>(); + BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); + Bigquery client = Transport.newBigQueryClient(options).build(); + + // TODO: Support table sharding and the better place to initialize + // BigQuery table. + HashSet tables = createdTables.get(); + if (!tables.contains(jsonTableSchema)) { + try { + TableSchema tableSchema = JSON_FACTORY.fromString( + jsonTableSchema, TableSchema.class); + TableReference tableReference = JSON_FACTORY.fromString( + jsonTableReference, TableReference.class); + + + BigQueryTableInserter inserter = new BigQueryTableInserter(client, tableReference); + inserter.tryCreateTable(tableSchema); + tables.add(jsonTableSchema); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + /** Accumulates the input into JsonTableRows and uniqueIdsForTableRows. */ + @Override + public void processElement(ProcessContext context) { + KV> kv = context.element(); + TableRow tableRow = kv.getValue().getValue(); + uniqueIdsForTableRows.add(kv.getValue().getKey()); + jsonTableRows.add(tableRow); + } + + /** Writes the accumulated rows into BigQuery with streaming API. */ + @Override + public void finishBundle(Context context) { + BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); + Bigquery client = Transport.newBigQueryClient(options).build(); + + try { + TableReference tableReference = JSON_FACTORY.fromString( + jsonTableReference, TableReference.class); + + BigQueryTableInserter inserter = new BigQueryTableInserter(client, tableReference); + inserter.insertAll(jsonTableRows.iterator(), uniqueIdsForTableRows.iterator()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Fn that tags each table row with a unique id. + * To avoid calling UUID.randomUUID() for each element, which can be costly, + * a randomUUID is generated only once per bucket of data. The actual unique + * id is created by concatenating this randomUUID with a sequential number. + */ + private static class TagWithUniqueIds extends DoFn>> { + private transient String randomUUID; + private transient AtomicLong sequenceNo; + + @Override + public void startBundle(Context context) { + randomUUID = UUID.randomUUID().toString(); + sequenceNo = new AtomicLong(); + } + + /** Tag the input with a unique id. */ + @Override + public void processElement(ProcessContext context) { + String uniqueId = randomUUID + Long.toString(sequenceNo.getAndIncrement()); + ThreadLocalRandom randomGenerator = ThreadLocalRandom.current(); + // We output on keys 0-50 to ensure that there's enough batching for + // BigQuery. + context.output(KV.of(randomGenerator.nextInt(0, 50), + KV.of(uniqueId, context.element()))); + } + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * PTransform that performs streaming BigQuery write. To increase consistency, + * it leverages BigQuery best effort de-dup mechanism. + */ + private static class StreamWithDeDup + extends PTransform, PDone> { + + private final TableReference tableReference; + private final TableSchema tableSchema; + + /** Constructor. */ + StreamWithDeDup(TableReference tableReference, TableSchema tableSchema) { + this.tableReference = tableReference; + this.tableSchema = tableSchema; + } + + @Override protected Coder getDefaultOutputCoder() { return VoidCoder.of(); } + + @Override + public PDone apply(PCollection in) { + // A naive implementation would be to simply stream data directly to BigQuery. + // However, this could occassionally lead to duplicated data, e.g., when + // a VM that runs this code is restarted and the code is re-run. + + // The above risk is mitigated in this implementation by relying on + // BigQuery built-in best effort de-dup mechanism. + + // To use this mechanism, each input TableRow is tagged with a generated + // unique id, which is then passed to BigQuery and used to ignore duplicates. + + PCollection>> tagged = + in.apply(ParDo.of(new TagWithUniqueIds())); + + // To prevent having the same TableRow processed more than once with regenerated + // different unique ids, this implementation relies on "checkpointing" which is + // achieved as a side effect of having StreamingWriteFn implement RequiresKeyedState. + tagged.apply(ParDo.of(new StreamingWriteFn(tableReference, tableSchema))); + + // Note that the implementation to return PDone here breaks the + // implicit assumption about the job execution order. If a user + // implements a PTransform that takes PDone returned here as its + // input, the transform may not necessarily be executed after + // the BigQueryIO.Write. + + return new PDone(); + } + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Direct mode read evaluator. + *

+ * This loads the entire table into an in-memory PCollection. + */ + private static void evaluateReadHelper( + Read.Bound transform, + DirectPipelineRunner.EvaluationContext context) { + BigQueryOptions options = context.getPipelineOptions(); + Bigquery client = Transport.newBigQueryClient(options).build(); + TableReference ref = transform.table; + if (ref.getProjectId() == null) { + ref.setProjectId(options.getProject()); + } + + LOG.info("Reading from BigQuery table {}", toTableSpec(ref)); + List elems = CloudSourceUtils.readElemsFromSource(new BigQuerySource(client, ref)); + LOG.info("Number of records read from BigQuery: {}", elems.size()); + context.setPCollection(transform.getOutput(), elems); + } + + /** + * Direct mode write evaluator. + *

+ * This writes the entire table in a single BigQuery request. + * The table will be created if necessary. + */ + private static void evaluateWriteHelper( + Write.Bound transform, + DirectPipelineRunner.EvaluationContext context) { + BigQueryOptions options = context.getPipelineOptions(); + Bigquery client = Transport.newBigQueryClient(options).build(); + TableReference ref = transform.table; + if (ref.getProjectId() == null) { + ref.setProjectId(options.getProject()); + } + + LOG.info("Writing to BigQuery table {}", toTableSpec(ref)); + + try { + BigQueryTableInserter inserter = new BigQueryTableInserter(client, ref); + + inserter.getOrCreateTable(transform.writeDisposition, + transform.createDisposition, transform.schema); + + List tableRows = context.getPCollection(transform.getInput()); + inserter.insertAll(tableRows.iterator()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java new file mode 100644 index 0000000000000..9c7fc0a1c5b33 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java @@ -0,0 +1,603 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import com.google.api.client.auth.oauth2.Credential; +import com.google.api.services.datastore.DatastoreV1.BeginTransactionRequest; +import com.google.api.services.datastore.DatastoreV1.BeginTransactionResponse; +import com.google.api.services.datastore.DatastoreV1.CommitRequest; +import com.google.api.services.datastore.DatastoreV1.Entity; +import com.google.api.services.datastore.DatastoreV1.Query; +import com.google.api.services.datastore.client.Datastore; +import com.google.api.services.datastore.client.DatastoreException; +import com.google.api.services.datastore.client.DatastoreFactory; +import com.google.api.services.datastore.client.DatastoreHelper; +import com.google.api.services.datastore.client.DatastoreOptions; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.DefaultCoder; +import com.google.cloud.dataflow.sdk.coders.EntityCoder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.GcpOptions; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Flatten; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.util.Credentials; +import com.google.cloud.dataflow.sdk.util.RetryHttpRequestInitializer; +import com.google.cloud.dataflow.sdk.values.PBegin; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; +import com.google.cloud.dataflow.sdk.values.PDone; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; + +/** + * Transforms for reading and writing + * Google Cloud Datastore + * entities. + * + *

The DatastoreIO class provides an experimental API to Read and Write a + * {@link PCollection} of Datastore Entity. Currently the class supports + * read operations on both the DirectPipelineRunner and DataflowPipelineRunner, + * and write operations on the DirectPipelineRunner. This API is subject to + * change, and currently requires an authentication workaround described below. + * + *

Datastore is a fully managed NoSQL data storage service. + * An Entity is an object in Datastore, analogous to the a row in traditional + * database table. DatastoreIO supports Read/Write from/to Datastore within + * Dataflow SDK service. + * + *

To use DatastoreIO, users must set up the environment and use gcloud + * to get credential for Datastore: + *

+ * $ export CLOUDSDK_EXTRA_SCOPES=https://www.googleapis.com/auth/datastore
+ * $ gcloud auth login
+ * 
+ * + *

Note that the environment variable CLOUDSDK_EXTRA_SCOPES must be set + * to the same value when executing a Datastore pipeline, as the local auth + * cache is keyed by the requested scopes. + * + *

To read a {@link PCollection} from a query to Datastore, use + * {@link DatastoreIO.Read}, specifying {@link DatastoreIO.Read#from} to specify + * dataset to read, the query to read from, and optionally + * {@link DatastoreIO.Read#named} and {@link DatastoreIO.Read#withHost} to specify + * the name of the pipeline step and the host of Datastore, respectively. + * For example: + * + *

 {@code
+ * // Read a query from Datastore
+ * PipelineOptions options =
+ *     CliPipelineOptionsFactory.create(PipelineOptions.class, args);
+ * Pipeline p = Pipeline.create(options);
+ * PCollection entities =
+ *     p.apply(DatastoreIO.Read
+ *             .named("Read Datastore")
+ *             .from(datasetId, query)
+ *             .withHost(host));
+ * p.run();
+ * } 
+ * + *

To write a {@link PCollection} to a datastore, use + * {@link DatastoreIO.Write}, specifying {@link DatastoreIO.Write#to} to specify + * the datastore to write to, and optionally {@link TextIO.Write#named} to specify + * the name of the pipeline step. For example: + * + *

 {@code
+ * // A simple Write to Datastore with DirectPipelineRunner (writing is not
+ * // yet implemented for other runners):
+ * PCollection entities = ...;
+ * lines.apply(DatastoreIO.Write.to("Write entities", datastore));
+ * p.run();
+ *
+ * } 
+ */ + +public class DatastoreIO { + + private static final Logger LOG = LoggerFactory.getLogger(DatastoreIO.class); + private static final String DEFAULT_HOST = "https://www.googleapis.com"; + + /** + * A PTransform that reads from a Datastore query and returns a + * {@code PCollection} containing each of the rows of the table. + */ + public static class Read { + + /** + * Returns a DatastoreIO.Read PTransform with the given step name. + */ + public static Bound named(String name) { + return new Bound(DEFAULT_HOST).named(name); + } + + /** + * Reads entities retrieved from the dataset and a given query. + */ + public static Bound from(String datasetId, Query query) { + return new Bound(DEFAULT_HOST).from(datasetId, query); + } + + /** + * Returns a DatastoreIO.Read PTransform with specified host. + */ + public static Bound withHost(String host) { + return new Bound(host); + } + + /** + * A PTransform that reads from a Datastore query and returns a bounded + * {@code PCollection}. + */ + public static class Bound extends PTransform> { + String host; + String datasetId; + Query query; + + /** + * Returns a DatastoreIO.Bound object with given query. + * Sets the name, Datastore host, datasetId, query associated + * with this PTransform, and options for this Pipeline. + */ + Bound(String name, String host, String datasetId, Query query) { + super(name); + this.host = host; + this.datasetId = datasetId; + this.query = query; + } + + /** + * Returns a DatastoreIO.Read PTransform with host set up. + */ + Bound(String host) { + this.host = host; + } + + /** + * Returns a new DatastoreIO.Read PTransform with the name + * associated with this transformation. + */ + public Bound named(String name) { + return new Bound(name, host, datasetId, query); + } + + /** + * Returns a new DatastoreIO.Read PTransform with datasetId, + * and query associated with this transformation, and options + * associated with this Pipleine. + */ + public Bound from(String datasetId, Query query) { + return new Bound(name, host, datasetId, query); + } + + /** + * Returns a new DatastoreIO.Read PTransform with the host + * specified. + */ + public Bound withHost(String host) { + return new Bound(name, host, datasetId, query); + } + + @Override + public PCollection apply(PBegin input) { + if (datasetId == null || query == null) { + throw new IllegalStateException( + "need to set datasetId, and query " + + "of a DatastoreIO.Read transform"); + } + + QueryOptions queryOptions = QueryOptions.create(host, datasetId, query); + PCollection output; + try { + DataflowPipelineOptions options = + getPipeline().getOptions().as(DataflowPipelineOptions.class); + PCollection queries = splitQueryOptions(queryOptions, options, input); + + output = queries.apply(ParDo.of(new ReadEntitiesFn())); + getCoderRegistry().registerCoder(Entity.class, EntityCoder.class); + } catch (DatastoreException e) { + LOG.warn("DatastoreException: error while doing Datastore query splitting.", e); + throw new RuntimeException("Error while splitting Datastore query."); + } + + return output; + } + } + } + + ///////////////////// Write Class ///////////////////////////////// + /** + * A PTransform that writes a {@code PCollection} containing + * entities to a Datastore kind. + * + * Current version only supports Write operation running on + * DirectPipelineRunner. If Write is used on DataflowPipelineRunner, + * it throws UnsupportedOperationException and won't continue on the + * operation. + * + */ + public static class Write { + /** + * Returns a DatastoreIO.Write PTransform with the name + * associated with this PTransform. + */ + public static Bound named(String name) { + return new Bound(DEFAULT_HOST).named(name); + } + + /** + * Returns a DatastoreIO.Write PTransform with given datasetId. + */ + public static Bound to(String datasetId) { + return new Bound(DEFAULT_HOST).to(datasetId); + } + + /** + * Returns a DatastoreIO.Write PTransform with specified host. + */ + public static Bound withHost(String host) { + return new Bound(host); + } + + /** + * A PTransform that writes a bounded {@code PCollection} + * to a Datastore. + */ + public static class Bound extends PTransform, PDone> { + String host; + String datasetId; + + /** + * Returns a DatastoreIO.Write PTransform with given host. + */ + Bound(String host) { + this.host = host; + } + + /** + * Returns a DatastoreIO.Write.Bound object. + * Sets the name, datastore agent, and kind associated + * with this transformation. + */ + Bound(String name, String host, String datasetId) { + super(name); + this.host = host; + this.datasetId = datasetId; + } + + /** + * Returns a DatastoreIO.Write PTransform with the name + * associated with this PTransform. + */ + public Bound named(String name) { + return new Bound(name, host, datasetId); + } + + /** + * Returns a DatastoreIO.Write PTransform with given datasetId. + */ + public Bound to(String datasetId) { + return new Bound(name, host, datasetId); + } + + /** + * Returns a new DatastoreIO.Write PTransform with specified host. + */ + public Bound withHost(String host) { + return new Bound(name, host, datasetId); + } + + @Override + public PDone apply(PCollection input) { + if (this.host == null || this.datasetId == null) { + throw new IllegalStateException( + "need to set Datastore host and dataasetId" + + "of a DatastoreIO.Write transform"); + } + + return new PDone(); + } + + @Override + protected String getKindString() { return "DatastoreIO.Write"; } + + @Override + protected Coder getDefaultOutputCoder() { + return VoidCoder.of(); + } + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + Bound.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + Bound transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateWriteHelper(transform, context); + } + }); + } + } + } + + /////////////////////////////////////////////////////////////////// + + /** + * A DoFn that performs query request to Datastore and converts + * each QueryOptions into Entities. + */ + private static class ReadEntitiesFn extends DoFn { + @Override + public void processElement(ProcessContext c) { + Query query = c.element().getQuery(); + Datastore datastore = c.element().getWorkerDatastore( + c.getPipelineOptions().as(GcpOptions.class)); + DatastoreIterator entityIterator = new DatastoreIterator(query, datastore); + + while (entityIterator.hasNext()) { + c.output(entityIterator.next().getEntity()); + } + } + } + + /** + * A class that stores query and datastore setup environments + * (host and datasetId). + */ + @DefaultCoder(AvroCoder.class) + private static class QueryOptions { + // Query to read in byte array. + public byte[] byteQuery; + + // Datastore host to read from. + public String host; + + // Datastore dataset ID to read from. + public String datasetId; + + @SuppressWarnings("unused") + QueryOptions() {} + + /** + * Returns a QueryOption object without account and private key file + * (for supporting query on local Datastore). + * + * @param host the host of Datastore to connect + * @param datasetId the dataset ID of Datastore to query + * @param query the query to perform + */ + QueryOptions(String host, String datasetId, Query query) { + this.host = host; + this.datasetId = datasetId; + this.setQuery(query); + } + + /** + * Creates and returns a QueryOption object for query on local Datastore. + * + * @param host the host of Datastore to connect + * @param datasetId the dataset ID of Datastore to query + * @param query the query to perform + */ + public static QueryOptions create(String host, String datasetId, Query query) { + return new QueryOptions(host, datasetId, query); + } + + /** + * Sets up a query. + * Stores query in a byte array so that we can use AvroCoder to encode/decode + * QueryOptions. + * + * @param q the query to be addressed + */ + public void setQuery(Query q) { + this.byteQuery = q.toByteArray(); + } + + /** + * Returns query. + * + * @return query in this option. + */ + public Query getQuery() { + try { + return Query.parseFrom(this.byteQuery); + } catch (IOException e) { + LOG.warn("IOException: parsing query failed.", e); + throw new RuntimeException("Cannot parse query from byte array."); + } + } + + /** + * Returns the dataset ID. + * + * @return a dataset ID string for Datastore. + */ + public String getDatasetId() { + return this.datasetId; + } + + /** + * Returns a copy of QueryOptions from current options with given query. + * + * @param query a new query to be set + * @return A QueryOptions object for query + */ + public QueryOptions newQuery(Query query) { + return create(host, datasetId, query); + } + + /** + * Returns a Datastore object for connecting to Datastore on workers. + * This method will try to get worker credential from Credentials + * library and constructs a Datastore object which is set up and + * ready to communicate with Datastore. + * + * @return a Datastore object setup with host and dataset. + */ + public Datastore getWorkerDatastore(GcpOptions options) { + DatastoreOptions.Builder builder = new DatastoreOptions.Builder() + .host(this.host) + .dataset(this.datasetId) + .initializer(new RetryHttpRequestInitializer(null)); + + try { + Credential credential = Credentials.getWorkerCredential(options); + builder.credential(credential); + } catch (IOException e) { + LOG.warn("IOException: can't get credential for worker.", e); + throw new RuntimeException("Failed on getting credential for worker."); + } + return DatastoreFactory.get().create(builder.build()); + } + + /** + * Returns a Datastore object for connecting to Datastore for users. + * This method will use the passed in credentials and construct a Datastore + * object which is set up and ready to communicate with Datastore. + * + * @return a Datastore object setup with host and dataset. + */ + public Datastore getUserDatastore(GcpOptions options) { + DatastoreOptions.Builder builder = new DatastoreOptions.Builder() + .host(this.host) + .dataset(this.datasetId) + .initializer(new RetryHttpRequestInitializer(null)); + + Credential credential = options.getGcpCredential(); + if (credential != null) { + builder.credential(credential); + } + return DatastoreFactory.get().create(builder.build()); + } + } + + /** + * Returns a list of QueryOptions by splitting a QueryOptions into sub-queries. + * This method leverages the QuerySplitter in Datastore to split the + * query into sub-queries for further parallel query in Dataflow service. + * + * @return a PCollection of QueryOptions for split queries + */ + private static PCollection splitQueryOptions( + QueryOptions queryOptions, DataflowPipelineOptions options, + PBegin input) + throws DatastoreException { + Query query = queryOptions.getQuery(); + Datastore datastore = queryOptions.getUserDatastore(options); + + // Get splits from the QuerySplit interface. + List splitQueries = DatastoreHelper.getQuerySplitter() + .getSplits(query, options.getNumWorkers(), datastore); + + List> queryList = new LinkedList<>(); + for (Query q : splitQueries) { + PCollection newQuery = input + .apply(Create.of(queryOptions.newQuery(q))); + queryList.add(newQuery); + } + + // This is a workaround to allow for parallelism of a small collection. + return PCollectionList.of(queryList) + .apply(Flatten.create()); + } + + ///////////////////////////////////////////////////////////////////// + + /** + * Direct mode write evaluator. + * This writes the result to Datastore. + */ + private static void evaluateWriteHelper( + Write.Bound transform, + DirectPipelineRunner.EvaluationContext context) { + LOG.info("Writing to Datastore"); + GcpOptions options = context.getPipelineOptions(); + Credential credential = options.getGcpCredential(); + Datastore datastore = DatastoreFactory.get().create( + new DatastoreOptions.Builder() + .host(transform.host) + .dataset(transform.datasetId) + .credential(credential) + .initializer(new RetryHttpRequestInitializer(null)) + .build()); + + List entityList = context.getPCollection(transform.getInput()); + + // Create a map to put entities with same ancestor for writing in a batch. + HashMap> map = new HashMap<>(); + for (Entity e : entityList) { + String keyOfAncestor = e.getKey().getPathElement(0).getKind() + + e.getKey().getPathElement(0).getName(); + List value = map.get(keyOfAncestor); + if (value == null) { + value = new ArrayList<>(); + } + value.add(e); + map.put(keyOfAncestor, value); + } + + // Walk over the map, and write entities bucket by bucket. + int count = 0; + for (String k : map.keySet()) { + List entitiesWithSameAncestor = map.get(k); + List toInsert = new ArrayList<>(); + for (Entity e : entitiesWithSameAncestor) { + toInsert.add(e); + // Note that Datastore has limit as 500 for a batch operation, + // so just flush to Datastore with every 500 entties. + if (toInsert.size() >= 500) { + writeBatch(toInsert, datastore); + toInsert.clear(); + } + } + writeBatch(toInsert, datastore); + count += entitiesWithSameAncestor.size(); + } + + LOG.info("Total number of entities written: {}", count); + } + + /** + * A function for batch writing to Datastore. + */ + private static void writeBatch(List listOfEntities, Datastore datastore) { + try { + BeginTransactionRequest.Builder treq = BeginTransactionRequest.newBuilder(); + BeginTransactionResponse tres = datastore.beginTransaction(treq.build()); + CommitRequest.Builder creq = CommitRequest.newBuilder(); + creq.setTransaction(tres.getTransaction()); + creq.getMutationBuilder().addAllInsertAutoId(listOfEntities); + datastore.commit(creq.build()); + } catch (DatastoreException e) { + LOG.warn("Error while doing datastore operation: {}", e); + throw new RuntimeException("Datastore exception", e); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIterator.java new file mode 100644 index 0000000000000..1b6d92e73c76a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIterator.java @@ -0,0 +1,141 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import com.google.api.services.datastore.DatastoreV1.EntityResult; +import com.google.api.services.datastore.DatastoreV1.Query; +import com.google.api.services.datastore.DatastoreV1.QueryResultBatch; +import com.google.api.services.datastore.DatastoreV1.RunQueryRequest; +import com.google.api.services.datastore.DatastoreV1.RunQueryResponse; +import com.google.api.services.datastore.client.Datastore; +import com.google.api.services.datastore.client.DatastoreException; +import com.google.common.collect.AbstractIterator; + +import java.util.Iterator; + +/** + * An iterator over the records from a query of the datastore. + * + *

Usage: + *

{@code
+ *   // Need to pass query and datastore object.
+ *   DatastoreIterator iterator = new DatastoreIterator(query, datastore);
+ *   while (iterator.hasNext()) {
+ *     Entity e = iterator.next().getEntity();
+ *     ...
+ *   }
+ * }
+ */ +class DatastoreIterator extends AbstractIterator { + /** + * Query to select records. + */ + private Query.Builder query; + + /** + * Datastore to read from. + */ + private Datastore datastore; + + /** + * True if more results may be available. + */ + private boolean moreResults; + + /** + * Iterator over records. + */ + private Iterator entities; + + /** + * Current batch of query results. + */ + private QueryResultBatch currentBatch; + + /** + * Maximum number of results to request per query. + * + *

Must be set, or it may result in an I/O error when querying + * Cloud Datastore. + */ + private static final int QUERY_LIMIT = 5000; + + /** + * Returns a DatastoreIterator with query and Datastore object set. + * + * @param query the query to select records. + * @param datastore a datastore connection to use. + */ + public DatastoreIterator(Query query, Datastore datastore) { + this.query = query.toBuilder().clone(); + this.datastore = datastore; + this.query.setLimit(QUERY_LIMIT); + } + + /** + * Returns an iterator over the next batch of records for the query + * and updates the cursor to get the next batch as needed. + * Query has specified limit and offset from InputSplit. + */ + private Iterator getIteratorAndMoveCursor() + throws DatastoreException{ + if (this.currentBatch != null && this.currentBatch.hasEndCursor()) { + this.query.setStartCursor(this.currentBatch.getEndCursor()); + } + + RunQueryRequest request = RunQueryRequest.newBuilder() + .setQuery(this.query) + .build(); + RunQueryResponse response = this.datastore.runQuery(request); + + this.currentBatch = response.getBatch(); + + // MORE_RESULTS_AFTER_LIMIT is not implemented yet: + // https://groups.google.com/forum/#!topic/gcd-discuss/iNs6M1jA2Vw, so + // use result count to determine if more results might exist. + int numFetch = this.currentBatch.getEntityResultCount(); + moreResults = numFetch == QUERY_LIMIT; + + // May receive a batch of 0 results if the number of records is a multiple + // of the request limit. + if (numFetch == 0) { + return null; + } + + return this.currentBatch.getEntityResultList().iterator(); + } + + @Override + public EntityResult computeNext() { + try { + if (entities == null || (!entities.hasNext() && this.moreResults)) { + entities = getIteratorAndMoveCursor(); + } + + if (entities == null || !entities.hasNext()) { + return endOfData(); + } + + return entities.next(); + + } catch (DatastoreException e) { + throw new RuntimeException( + "Datastore error while iterating over entities", e); + } + } +} + diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java new file mode 100644 index 0000000000000..b9f0514841590 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -0,0 +1,331 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.cloud.dataflow.sdk.values.PInput; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * [Whitelisting Required] Read and Write transforms for Pub/Sub streams. These transforms create + * and consume unbounded {@link com.google.cloud.dataflow.sdk.values.PCollection}s. + * + *

Important: PubsubIO is experimental. It is not supported by the + * {@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner} and is only supported in the + * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner} for users whitelisted in a + * streaming early access program and who enable + * {@link com.google.cloud.dataflow.sdk.options.StreamingOptions#setStreaming(boolean)}. + * + *

You should expect this class to change significantly in future versions of the SDK + * or be removed entirely. + */ +public class PubsubIO { + + /** + * Project IDs must contain 6-63 lowercase letters, digits, or dashes. + * IDs must start with a letter and may not end with a dash. + * This regex isn't exact - this allows for patterns that would be rejected by + * the service, but this is sufficient for basic parsing of table references. + */ + private static final Pattern PROJECT_ID_REGEXP = + Pattern.compile("[a-z][-a-z0-9:.]{4,61}[a-z0-9]"); + + private static final Pattern SUBSCRIPTION_REGEXP = + Pattern.compile("/subscriptions/([^/]+)/(.+)"); + + private static final Pattern TOPIC_REGEXP = + Pattern.compile("/topics/([^/]+)/(.+)"); + + private static final Pattern PUBSUB_NAME_REGEXP = + Pattern.compile("[a-z][-._a-z0-9]+[a-z0-9]"); + + private static final int PUBSUB_NAME_MAX_LENGTH = 255; + + private static final String SUBSCRIPTION_RANDOM_TEST_PREFIX = "_random/"; + private static final String TOPIC_DEV_NULL_TEST_NAME = "/topics/dev/null"; + + /** + * Utility class to validate topic and subscription names. + */ + public static class Validator { + public static void validateTopicName(String topic) { + if (topic.equals(TOPIC_DEV_NULL_TEST_NAME)) { + return; + } + Matcher match = TOPIC_REGEXP.matcher(topic); + if (!match.matches()) { + throw new IllegalArgumentException( + "Pubsub topic is not in /topics/project_id/topic_name format: " + + topic); + } + validateProjectName(match.group(1)); + validatePubsubName(match.group(2)); + } + + public static void validateSubscriptionName(String subscription) { + if (subscription.startsWith(SUBSCRIPTION_RANDOM_TEST_PREFIX)) { + return; + } + Matcher match = SUBSCRIPTION_REGEXP.matcher(subscription); + if (!match.matches()) { + throw new IllegalArgumentException( + "Pubsub subscription is not in /subscriptions/project_id/subscription_name format: " + + subscription); + } + validateProjectName(match.group(1)); + validatePubsubName(match.group(2)); + } + + private static void validateProjectName(String project) { + Matcher match = PROJECT_ID_REGEXP.matcher(project); + if (!match.matches()) { + throw new IllegalArgumentException( + "Illegal project name specified in Pubsub subscription: " + project); + } + } + + private static void validatePubsubName(String name) { + if (name.length() > PUBSUB_NAME_MAX_LENGTH) { + throw new IllegalArgumentException( + "Pubsub object name is longer than 255 characters: " + name); + } + + if (name.startsWith("goog")) { + throw new IllegalArgumentException( + "Pubsub object name cannot start with goog: " + name); + } + + Matcher match = PUBSUB_NAME_REGEXP.matcher(name); + if (!match.matches()) { + throw new IllegalArgumentException( + "Illegal Pubsub object name specified: " + name + + " Please see Javadoc for naming rules."); + } + } + } + + /** + * A PTransform that continuously reads from a Pubsub stream and + * returns a {@code PCollection} containing the items from + * the stream. + */ + // TODO: Support non-String encodings. + public static class Read { + public static Bound named(String name) { + return new Bound().named(name); + } + + /** + * Creates and returns a PubsubIO.Read PTransform for reading from + * a Pubsub topic with the specified publisher topic. Format for + * Cloud Pubsub topic names should be of the form /topics//, + * where is the name of the publishing project. + * The component must comply with the below requirements. + *

    + *
  • Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods + * ('.').
  • + *
  • Must be between 3 and 255 characters.
  • + *
  • Must begin with a letter.
  • + *
  • Must end with a letter or a number.
  • + *
  • Cannot begin with 'goog' prefix.
  • + *
+ */ + public static Bound topic(String topic) { + return new Bound().topic(topic); + } + + /** + * Creates and returns a PubsubIO.Read PTransform for reading from + * a specific Pubsub subscription. Mutually exclusive with + * PubsubIO.Read.topic(). + * Cloud Pubsub subscription names should be of the form + * /subscriptions//<, + * where is the name of the project the subscription belongs to. + * The component must comply with the below requirements. + *
    + *
  • Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods + * ('.').
  • + *
  • Must be between 3 and 255 characters.
  • + *
  • Must begin with a letter.
  • + *
  • Must end with a letter or a number.
  • + *
  • Cannot begin with 'goog' prefix.
  • + *
+ */ + public static Bound subscription(String subscription) { + return new Bound().subscription(subscription); + } + + /** + * A PTransform that reads from a PubSub source and returns + * a unbounded PCollection containing the items from the stream. + */ + public static class Bound + extends PTransform> { + /** The Pubsub topic to read from. */ + String topic; + /** The Pubsub subscription to read from */ + String subscription; + + Bound() {} + + Bound(String name, String subscription, String topic) { + super(name); + if (subscription != null) { + Validator.validateSubscriptionName(subscription); + } + if (topic != null) { + Validator.validateTopicName(topic); + } + this.subscription = subscription; + this.topic = topic; + } + + public Bound named(String name) { + return new Bound(name, subscription, topic); + } + + public Bound subscription(String subscription) { + return new Bound(name, subscription, topic); + } + + public Bound topic(String topic) { + return new Bound(name, subscription, topic); + } + + @Override + public PCollection apply(PInput input) { + if (topic == null && subscription == null) { + throw new IllegalStateException( + "need to set either the topic or the subscription for " + + "a PubsubIO.Read transform"); + } + if (topic != null && subscription != null) { + throw new IllegalStateException( + "Can't set both the topic and the subscription for a " + + "PubsubIO.Read transform"); + } + return PCollection.createPrimitiveOutputInternal( + new GlobalWindow()); + } + + @Override + protected Coder getDefaultOutputCoder() { + return StringUtf8Coder.of(); + } + + @Override + protected String getKindString() { return "PubsubIO.Read"; } + + public String getTopic() { + return topic; + } + + public String getSubscription() { + return subscription; + } + + static { + // TODO: Figure out how to make this work under + // DirectPipelineRunner. + } + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * A PTransform that continuously writes a + * {@code PCollection} to a Pubsub stream. + */ + // TODO: Support non-String encodings. + public static class Write { + public static Bound named(String name) { + return new Bound().named(name); + } + + /** The topic to publish to. + * Cloud Pubsub topic names should be /topics//, + * where is the name of the publishing project. + */ + public static Bound topic(String topic) { + return new Bound().topic(topic); + } + + /** + * A PTransfrom that writes a unbounded {@code PCollection} + * to a PubSub stream. + */ + public static class Bound + extends PTransform, PDone> { + /** The Pubsub topic to publish to. */ + String topic; + + Bound() {} + + Bound(String name, String topic) { + super(name); + if (topic != null) { + Validator.validateTopicName(topic); + this.topic = topic; + } + } + + public Bound named(String name) { + return new Bound(name, topic); + } + + public Bound topic(String topic) { + return new Bound(name, topic); + } + + @Override + public PDone apply(PCollection input) { + if (topic == null) { + throw new IllegalStateException( + "need to set the topic of a PubsubIO.Write transform"); + } + return new PDone(); + } + + @Override + protected Coder getDefaultOutputCoder() { + return VoidCoder.of(); + } + + @Override + protected String getKindString() { return "PubsubIO.Write"; } + + public String getTopic() { + return topic; + } + + static { + // TODO: Figure out how to make this work under + // DirectPipelineRunner. + } + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ShardNameTemplate.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ShardNameTemplate.java new file mode 100644 index 0000000000000..5ab0a99084b84 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ShardNameTemplate.java @@ -0,0 +1,75 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +/** + * Standard shard naming templates. + * + *

Shard naming templates are strings which may contain placeholders for + * the shard number and shard count. When constructing a filename for a + * particular shard number, the upper-case letters 'S' and 'N' are replaced + * with the 0-padded shard number and shard count respectively. + * + *

Left-padding of the numbers enables lexicographical sorting of the + * resulting filenames. If the shard number or count are too large for the + * space provided in the template, then the result may no longer sort + * lexicographically. For example, a shard template of "S-of-N", for 200 + * shards, will result in outputs named "0-of-200", ... '10-of-200', + * '100-of-200", etc. + * + *

Shard numbers start with 0, so the last shard number is the shard count + * minus one. For example, the template "-SSSSS-of-NNNNN" will be + * instantiated as "-00000-of-01000" for the first shard (shard 0) of a + * 1000-way sharded output. + * + *

A shard name template is typically provided along with a name prefix + * and suffix, which allows constructing complex paths which have embedded + * shard information. For example, outputs in the form + * "gs://bucket/path-01-of-99.txt" could be constructed by providing the + * individual components: + * + *

{@code
+ *   pipeline.apply(
+ *       TextIO.Write.to("gs://bucket/path")
+ *                   .withShardNameTemplate("-SS-of-NN")
+ *                   .withSuffix(".txt"))
+ * }
+ * + *

In the example above, you could make parts of the output configurable + * by users without the user having to specify all components of the output + * name. + * + *

If a shard name template does not contain any repeating 'S', then + * the output shard count must be 1, as otherwise the same filename would be + * generated for multiple shards. + */ +public class ShardNameTemplate { + /** + * Shard name containing the index and max. + * + *

Eg: [prefix]-00000-of-00100[suffix] and + * [prefix]-00001-of-00100[suffix] + */ + public static final String INDEX_OF_MAX = "-SSSSS-of-NNNNN"; + + /** + * Shard is a file within a directory. + * + *

Eg: [prefix]/part-00000[suffix] and [prefix]/part-00001[suffix] + */ + public static final String DIRECTORY_CONTAINER = "/part-SSSSS"; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java new file mode 100644 index 0000000000000..5d1cb205b4224 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java @@ -0,0 +1,567 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import static com.google.cloud.dataflow.sdk.util.CloudSourceUtils.readElemsFromSource; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.worker.TextSink; +import com.google.cloud.dataflow.sdk.runners.worker.TextSource; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.cloud.dataflow.sdk.values.PInput; + +import java.io.IOException; +import java.util.List; +import java.util.regex.Pattern; + +import javax.annotation.Nullable; + +/** + * Transforms for reading and writing text files. + * + *

To read a {@link PCollection} from one or more text files, use + * {@link TextIO.Read}, specifying {@link TextIO.Read#from} to specify + * the path of the file(s) to read from (e.g., a local filename or + * filename pattern if running locally, or a Google Cloud Storage + * filename or filename pattern of the form + * {@code "gs:///"}), and optionally + * {@link TextIO.Read#named} to specify the name of the pipeline step + * and/or {@link TextIO.Read#withCoder} to specify the Coder to use to + * decode the text lines into Java values. For example: + * + *

 {@code
+ * Pipeline p = ...;
+ *
+ * // A simple Read of a local file (only runs locally):
+ * PCollection lines =
+ *     p.apply(TextIO.Read.from("/path/to/file.txt"));
+ *
+ * // A fully-specified Read from a GCS file (runs locally and via the
+ * // Google Cloud Dataflow service):
+ * PCollection numbers =
+ *     p.apply(TextIO.Read.named("ReadNumbers")
+ *                        .from("gs://my_bucket/path/to/numbers-*.txt")
+ *                        .withCoder(TextualIntegerCoder.of()));
+ * } 
+ * + *

To write a {@link PCollection} to one or more text files, use + * {@link TextIO.Write}, specifying {@link TextIO.Write#to} to specify + * the path of the file to write to (e.g., a local filename or sharded + * filename pattern if running locally, or a Google Cloud Storage + * filename or sharded filename pattern of the form + * {@code "gs:///"}), and optionally + * {@link TextIO.Write#named} to specify the name of the pipeline step + * and/or {@link TextIO.Write#withCoder} to specify the Coder to use + * to encode the Java values into text lines. For example: + * + *

 {@code
+ * // A simple Write to a local file (only runs locally):
+ * PCollection lines = ...;
+ * lines.apply(TextIO.Write.to("/path/to/file.txt"));
+ *
+ * // A fully-specified Write to a sharded GCS file (runs locally and via the
+ * // Google Cloud Dataflow service):
+ * PCollection numbers = ...;
+ * numbers.apply(TextIO.Write.named("WriteNumbers")
+ *                           .to("gs://my_bucket/path/to/numbers")
+ *                           .withSuffix(".txt")
+ *                           .withCoder(TextualIntegerCoder.of()));
+ * } 
+ */ +public class TextIO { + public static final Coder DEFAULT_TEXT_CODER = StringUtf8Coder.of(); + + /** + * A root PTransform that reads from a text file (or multiple text + * files matching a pattern) and returns a PCollection containing + * the decoding of each of the lines of the text file(s). The + * default decoding just returns the lines. + */ + public static class Read { + /** + * Returns a TextIO.Read PTransform with the given step name. + */ + public static Bound named(String name) { + return new Bound<>(DEFAULT_TEXT_CODER).named(name); + } + + /** + * Returns a TextIO.Read PTransform that reads from the file(s) + * with the given name or pattern. This can be a local filename + * or filename pattern (if running locally), or a Google Cloud + * Storage filename or filename pattern of the form + * {@code "gs:///"}) (if running locally or via + * the Google Cloud Dataflow service). Standard + * Java Filesystem glob patterns ("*", "?", "[..]") are supported. + */ + public static Bound from(String filepattern) { + return new Bound<>(DEFAULT_TEXT_CODER).from(filepattern); + } + + /** + * Returns a TextIO.Read PTransform that uses the given + * {@code Coder} to decode each of the lines of the file into a + * value of type {@code T}. + * + *

By default, uses {@link StringUtf8Coder}, which just + * returns the text lines as Java strings. + * + * @param the type of the decoded elements, and the elements + * of the resulting PCollection + */ + public static Bound withCoder(Coder coder) { + return new Bound<>(coder); + } + + // TODO: strippingNewlines, gzipped, etc. + + /** + * A root PTransform that reads from a text file (or multiple text files + * matching a pattern) and returns a bounded PCollection containing the + * decoding of each of the lines of the text file(s). The default + * decoding just returns the lines. + * + * @param the type of each of the elements of the resulting + * PCollection, decoded from the lines of the text file + */ + public static class Bound + extends PTransform> { + /** The filepattern to read from. */ + @Nullable final String filepattern; + + /** The Coder to use to decode each line. */ + @Nullable final Coder coder; + + Bound(Coder coder) { + this(null, null, coder); + } + + Bound(String name, String filepattern, Coder coder) { + super(name); + this.coder = coder; + this.filepattern = filepattern; + } + + /** + * Returns a new TextIO.Read PTransform that's like this one but + * with the given step name. Does not modify this object. + */ + public Bound named(String name) { + return new Bound<>(name, filepattern, coder); + } + + /** + * Returns a new TextIO.Read PTransform that's like this one but + * that reads from the file(s) with the given name or pattern. + * (See {@link TextIO.Read#from} for a description of + * filepatterns.) Does not modify this object. + */ + public Bound from(String filepattern) { + return new Bound<>(name, filepattern, coder); + } + + /** + * Returns a new TextIO.Read PTransform that's like this one but + * that uses the given {@code Coder} to decode each of the + * lines of the file into a value of type {@code T1}. Does not + * modify this object. + * + * @param the type of the decoded elements, and the + * elements of the resulting PCollection + */ + public Bound withCoder(Coder coder) { + return new Bound<>(name, filepattern, coder); + } + + @Override + public PCollection apply(PInput input) { + if (filepattern == null) { + throw new IllegalStateException( + "need to set the filepattern of a TextIO.Read transform"); + } + // Force the output's Coder to be what the read is using, and + // unchangeable later, to ensure that we read the input in the + // format specified by the Read transform. + return PCollection.createPrimitiveOutputInternal(new GlobalWindow()) + .setCoder(coder); + } + + @Override + protected Coder getDefaultOutputCoder() { + return coder; + } + + @Override + protected String getKindString() { return "TextIO.Read"; } + + public String getFilepattern() { + return filepattern; + } + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + Bound.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + Bound transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateReadHelper(transform, context); + } + }); + } + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * A PTransform that writes a PCollection to a text file (or + * multiple text files matching a sharding pattern), with each + * PCollection element being encoded into its own line. + */ + public static class Write { + /** + * Returns a TextIO.Write PTransform with the given step name. + */ + public static Bound named(String name) { + return new Bound<>(DEFAULT_TEXT_CODER).named(name); + } + + /** + * Returns a TextIO.Write PTransform that writes to the file(s) + * with the given prefix. This can be a local filename + * (if running locally), or a Google Cloud Storage filename of + * the form {@code "gs:///"}) + * (if running locally or via the Google Cloud Dataflow service). + * + *

The files written will begin with this prefix, followed by + * a shard identifier (see {@link Bound#withNumShards}, and end + * in a common extension, if given by {@link Bound#withSuffix}. + */ + public static Bound to(String prefix) { + return new Bound<>(DEFAULT_TEXT_CODER).to(prefix); + } + + /** + * Returns a TextIO.Write PTransform that writes to the file(s) with the + * given filename suffix. + */ + public static Bound withSuffix(String nameExtension) { + return new Bound<>(DEFAULT_TEXT_CODER).withSuffix(nameExtension); + } + + /** + * Returns a TextIO.Write PTransform that uses the provided shard count. + * + *

Constraining the number of shards is likely to reduce + * the performance of a pipeline. Setting this value is not recommended + * unless you require a specific number of output files. + * + * @param numShards the number of shards to use, or 0 to let the system + * decide. + */ + public static Bound withNumShards(int numShards) { + return new Bound<>(DEFAULT_TEXT_CODER).withNumShards(numShards); + } + + /** + * Returns a TextIO.Write PTransform that uses the given shard name + * template. + * + *

See {@link ShardNameTemplate} for a description of shard templates. + */ + public static Bound withShardNameTemplate(String shardTemplate) { + return new Bound<>(DEFAULT_TEXT_CODER) + .withShardNameTemplate(shardTemplate); + } + + /** + * Returns a TextIO.Write PTransform that forces a single file as + * output. + */ + public static Bound withoutSharding() { + return new Bound<>(DEFAULT_TEXT_CODER).withoutSharding(); + } + + /** + * Returns a TextIO.Write PTransform that uses the given + * {@code Coder} to encode each of the elements of the input + * {@code PCollection} into an output text line. + * + *

By default, uses {@link StringUtf8Coder}, which writes input + * Java strings directly as output lines. + * + * @param the type of the elements of the input PCollection + */ + public static Bound withCoder(Coder coder) { + return new Bound<>(coder); + } + + // TODO: appendingNewlines, gzipped, header, footer, etc. + + /** + * A PTransform that writes a bounded PCollection to a text file (or + * multiple text files matching a sharding pattern), with each + * PCollection element being encoded into its own line. + * + * @param the type of the elements of the input PCollection + */ + public static class Bound + extends PTransform, PDone> { + /** The filename to write to. */ + @Nullable final String filenamePrefix; + /** Suffix to use for each filename. */ + final String filenameSuffix; + + /** The Coder to use to decode each line. */ + final Coder coder; + + /** Requested number of shards. 0 for automatic. */ + final int numShards; + + /** Shard template string. */ + final String shardTemplate; + + Bound(Coder coder) { + this(null, null, "", coder, 0, ShardNameTemplate.INDEX_OF_MAX); + } + + Bound(String name, String filenamePrefix, String filenameSuffix, + Coder coder, int numShards, + String shardTemplate) { + super(name); + this.coder = coder; + this.filenamePrefix = filenamePrefix; + this.filenameSuffix = filenameSuffix; + this.numShards = numShards; + this.shardTemplate = shardTemplate; + } + + /** + * Returns a new TextIO.Write PTransform that's like this one but + * with the given step name. Does not modify this object. + */ + public Bound named(String name) { + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + shardTemplate); + } + + /** + * Returns a new TextIO.Write PTransform that's like this one but + * that writes to the file(s) with the given filename prefix. + * + *

See {@link Write#to(String) Write.to(String)} for more information. + * + *

Does not modify this object. + */ + public Bound to(String filenamePrefix) { + validateOutputComponent(filenamePrefix); + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + shardTemplate); + } + + /** + * Returns a new TextIO.Write PTransform that's like this one but + * that writes to the file(s) with the given filename suffix. + * + *

Does not modify this object. + * + * @see ShardNameTemplate + */ + public Bound withSuffix(String nameExtension) { + validateOutputComponent(nameExtension); + return new Bound<>(name, filenamePrefix, nameExtension, coder, numShards, + shardTemplate); + } + + /** + * Returns a new TextIO.Write PTransform that's like this one but + * that uses the provided shard count. + * + *

Constraining the number of shards is likely to reduce + * the performance of a pipeline. Setting this value is not recommended + * unless you require a specific number of output files. + * + *

Does not modify this object. + * + * @param numShards the number of shards to use, or 0 to let the system + * decide. + * @see ShardNameTemplate + */ + public Bound withNumShards(int numShards) { + Preconditions.checkArgument(numShards >= 0); + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + shardTemplate); + } + + /** + * Returns a new TextIO.Write PTransform that's like this one but + * that uses the given shard name template. + * + *

Does not modify this object. + * + * @see ShardNameTemplate + */ + public Bound withShardNameTemplate(String shardTemplate) { + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + shardTemplate); + } + + /** + * Returns a new TextIO.Write PTransform that's like this one but + * that forces a single file as output. + * + *

This is a shortcut for + * {@code .withNumShards(1).withShardNameTemplate("")} + * + *

Does not modify this object. + */ + public Bound withoutSharding() { + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, 1, ""); + } + + /** + * Returns a new TextIO.Write PTransform that's like this one + * but that uses the given {@code Coder} to encode each of + * the elements of the input {@code PCollection} into an + * output text line. Does not modify this object. + * + * @param the type of the elements of the input PCollection + */ + public Bound withCoder(Coder coder) { + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + shardTemplate); + } + + @Override + public PDone apply(PCollection input) { + if (filenamePrefix == null) { + throw new IllegalStateException( + "need to set the filename prefix of a TextIO.Write transform"); + } + return new PDone(); + } + + /** + * Returns the current shard name template string. + */ + public String getShardNameTemplate() { + return shardTemplate; + } + + @Override + protected Coder getDefaultOutputCoder() { + return VoidCoder.of(); + } + + @Override + protected String getKindString() { return "TextIO.Write"; } + + public String getFilenamePrefix() { + return filenamePrefix; + } + + public String getShardTemplate() { + return shardTemplate; + } + + public int getNumShards() { + return numShards; + } + + public String getFilenameSuffix() { + return filenameSuffix; + } + + public Coder getCoder() { + return coder; + } + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + Bound.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + Bound transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateWriteHelper(transform, context); + } + }); + } + } + } + + // Pattern which matches old-style shard output patterns, which are now + // disallowed. + private static final Pattern SHARD_OUTPUT_PATTERN = + Pattern.compile("@([0-9]+|\\*)"); + + private static void validateOutputComponent(String partialFilePattern) { + Preconditions.checkArgument( + !SHARD_OUTPUT_PATTERN.matcher(partialFilePattern).find(), + "Output name components are not allowed to contain @* or @N patterns: " + + partialFilePattern); + } + + ////////////////////////////////////////////////////////////////////////////// + + private static void evaluateReadHelper( + Read.Bound transform, + DirectPipelineRunner.EvaluationContext context) { + TextSource source = new TextSource<>( + transform.filepattern, true, null, null, transform.coder); + List elems = readElemsFromSource(source); + context.setPCollection(transform.getOutput(), elems); + } + + private static void evaluateWriteHelper( + Write.Bound transform, + DirectPipelineRunner.EvaluationContext context) { + List elems = context.getPCollection(transform.getInput()); + int numShards = transform.numShards; + if (numShards < 1) { + // System gets to choose. For direct mode, choose 1. + numShards = 1; + } + TextSink> writer = TextSink.createForDirectPipelineRunner( + transform.filenamePrefix, transform.getShardNameTemplate(), + transform.filenameSuffix, numShards, + true, null, null, transform.coder); + try (Sink.SinkWriter> sink = writer.writer()) { + for (T elem : elems) { + sink.add(WindowedValue.valueInGlobalWindow(elem)); + } + } catch (IOException exn) { + throw new RuntimeException( + "unable to write to output file \"" + transform.filenamePrefix + "\"", + exn); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/package-info.java new file mode 100644 index 0000000000000..886255e271d23 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/package-info.java @@ -0,0 +1,37 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Defines transforms for reading and writing common storage formats, including + * {@link com.google.cloud.dataflow.sdk.io.AvroIO}, + * {@link com.google.cloud.dataflow.sdk.io.BigQueryIO}, and + * {@link com.google.cloud.dataflow.sdk.io.TextIO}. + * + *

The classes in this package provide {@code Read} transforms which create PCollections + * from existing storage: + *

{@code
+ * PCollection inputData = pipeline.apply(
+ *     BigQueryIO.Read.named("Read")
+ *                    .from("clouddataflow-readonly:samples.weather_stations");
+ * }
+ * and {@code Write} transforms which persist PCollections to external storage: + *
 {@code
+ * PCollection numbers = ...;
+ * numbers.apply(TextIO.Write.named("WriteNumbers")
+ *                           .to("gs://my_bucket/path/to/numbers"));
+ * } 
+ */ +package com.google.cloud.dataflow.sdk.io; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ApplicationNameOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ApplicationNameOptions.java new file mode 100644 index 0000000000000..327e5c08445cb --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ApplicationNameOptions.java @@ -0,0 +1,33 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +/** + * Options that allow setting the application name. + */ +public interface ApplicationNameOptions extends PipelineOptions { + /** + * Name of application, for display purposes. + *

+ * Defaults to the name of the class which constructs the + * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner}. + */ + @Description("Application name. Defaults to the name of the class which " + + "constructs the Pipeline.") + String getAppName(); + void setAppName(String value); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BigQueryOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BigQueryOptions.java new file mode 100644 index 0000000000000..b764f20918b02 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BigQueryOptions.java @@ -0,0 +1,29 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +/** + * Properties needed when using BigQuery with the Dataflow SDK. + */ +public interface BigQueryOptions extends ApplicationNameOptions, GcpOptions, + PipelineOptions, StreamingOptions { + @Description("Temporary staging dataset ID for BigQuery " + + "table operations") + @Default.String("bigquery.googleapis.com/cloud_dataflow") + String getTempDatasetId(); + void setTempDatasetId(String value); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BlockingDataflowPipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BlockingDataflowPipelineOptions.java new file mode 100644 index 0000000000000..cdd5019b5df14 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BlockingDataflowPipelineOptions.java @@ -0,0 +1,46 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner; + +import com.fasterxml.jackson.annotation.JsonIgnore; + +import java.io.PrintStream; + +/** + * Options which are used to configure the {@link BlockingDataflowPipelineRunner}. + */ +public interface BlockingDataflowPipelineOptions extends DataflowPipelineOptions { + /** + * Output stream for job status messages. + */ + @JsonIgnore + @Default.InstanceFactory(StandardOutputFactory.class) + PrintStream getJobMessageOutput(); + void setJobMessageOutput(PrintStream value); + + /** + * Returns a default of {@link System#out}. + */ + public static class StandardOutputFactory implements DefaultValueFactory { + @Override + public PrintStream create(PipelineOptions options) { + return System.out; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java new file mode 100644 index 0000000000000..76de6e6dd8bf6 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java @@ -0,0 +1,67 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import java.util.List; + +/** + * Options used for testing and debugging the Dataflow SDK. + */ +public interface DataflowPipelineDebugOptions extends PipelineOptions { + /** + * Dataflow endpoint to use. + * + *

Defaults to the current version of the Google Cloud Dataflow + * API, at the time the current SDK version was released. + * + *

If the string contains "://", then this is treated as a url, + * otherwise {@link #getApiRootUrl()} is used as the root + * url. + */ + @Description("Cloud Dataflow Endpoint") + @Default.String("dataflow/v1b3/projects/") + String getDataflowEndpoint(); + void setDataflowEndpoint(String value); + + /** + * The list of backend experiments to enable. + * + *

Dataflow provides a number of experimental features that can be enabled + * with this flag. + * + *

Please sync with the Dataflow team when enabling any experiments. + */ + @Description("Backend experiments to enable.") + List getExperiments(); + void setExperiments(List value); + + /** + * The API endpoint to use when communicating with the Dataflow service. + */ + @Description("Google Cloud root API") + @Default.String("https://www.googleapis.com/") + String getApiRootUrl(); + void setApiRootUrl(String value); + + /** + * The path to write the translated Dataflow specification out to + * at job submission time. + */ + @Description("File for writing dataflow job descriptions") + String getDataflowJobFile(); + void setDataflowJobFile(String value); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java new file mode 100644 index 0000000000000..7d05088732328 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java @@ -0,0 +1,128 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.api.services.dataflow.Dataflow; +import com.google.cloud.dataflow.sdk.runners.DataflowPipeline; +import com.google.cloud.dataflow.sdk.util.Transport; +import com.google.common.base.MoreObjects; + +import com.fasterxml.jackson.annotation.JsonIgnore; + +import org.joda.time.DateTimeUtils; +import org.joda.time.DateTimeZone; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; + +/** + * Options which can be used to configure the {@link DataflowPipeline}. + */ +public interface DataflowPipelineOptions extends + PipelineOptions, GcpOptions, ApplicationNameOptions, DataflowPipelineDebugOptions, + DataflowPipelineShuffleOptions, DataflowPipelineWorkerPoolOptions, BigQueryOptions, + GcsOptions, StreamingOptions { + + /** + * GCS path for temporary files. + *

+ * Must be a valid Cloud Storage url, beginning with the prefix "gs://" + *

+ * At least one of {@link #getTempLocation()} or {@link #getStagingLocation()} must be set. If + * {@link #getTempLocation()} is not set, then the Dataflow pipeline defaults to using + * {@link #getStagingLocation()}. + */ + @Description("GCS path for temporary files, eg \"gs://bucket/object\". " + + "Defaults to stagingLocation.") + String getTempLocation(); + void setTempLocation(String value); + + /** + * GCS path for staging local files. + *

+ * If {@link #getStagingLocation()} is not set, then the Dataflow pipeline defaults to a staging + * directory within {@link #getTempLocation}. + *

+ * At least one of {@link #getTempLocation()} or {@link #getStagingLocation()} must be set. + */ + @Description("GCS staging path. Defaults to a staging directory" + + " with the tempLocation") + String getStagingLocation(); + void setStagingLocation(String value); + + /** + * The job name is used as an idempotence key within the Dataflow service. If there + * is an existing job which is currently active, another job with the same name will + * not be able to be created. + */ + @Description("Dataflow job name, to uniquely identify active jobs. " + + "Defaults to using the ApplicationName-UserDame-Date.") + @Default.InstanceFactory(JobNameFactory.class) + String getJobName(); + void setJobName(String value); + + /** + * Returns a normalized job name constructed from {@link ApplicationNameOptions#getAppName()}, the + * local system user name (if available), and the current time. The normalization makes sure that + * the job name matches the required pattern of [a-z]([-a-z0-9]*[a-z0-9])? and length limit of 40 + * characters. + *

+ * This job name factory is only able to generate one unique name per second per application and + * user combination. + */ + public static class JobNameFactory implements DefaultValueFactory { + private static final DateTimeFormatter FORMATTER = + DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC); + private static final int MAX_APP_NAME = 19; + private static final int MAX_USER_NAME = 9; + + @Override + public String create(PipelineOptions options) { + String appName = options.as(ApplicationNameOptions.class).getAppName(); + String normalizedAppName = appName == null || appName.length() == 0 ? "dataflow" + : appName.toLowerCase() + .replaceAll("[^a-z0-9]", "0") + .replaceAll("^[^a-z]", "a"); + String userName = MoreObjects.firstNonNull(System.getProperty("user.name"), ""); + String normalizedUserName = userName.toLowerCase() + .replaceAll("[^a-z0-9]", "0"); + String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis()); + + // Maximize the amount of the app name and user name we can use. + normalizedAppName = normalizedAppName.substring(0, + Math.min(normalizedAppName.length(), + MAX_APP_NAME + Math.max(0, MAX_USER_NAME - normalizedUserName.length()))); + normalizedUserName = normalizedUserName.substring(0, + Math.min(userName.length(), + MAX_USER_NAME + Math.max(0, MAX_APP_NAME - normalizedAppName.length()))); + return normalizedAppName + "-" + normalizedUserName + "-" + datePart; + } + } + + /** Alternative Dataflow client */ + @JsonIgnore + @Default.InstanceFactory(DataflowClientFactory.class) + Dataflow getDataflowClient(); + void setDataflowClient(Dataflow value); + + /** Returns the default Dataflow client built from the passed in PipelineOptions. */ + public static class DataflowClientFactory implements DefaultValueFactory { + @Override + public Dataflow create(PipelineOptions options) { + return Transport.newDataflowClient(options.as(DataflowPipelineOptions.class)).build(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineShuffleOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineShuffleOptions.java new file mode 100644 index 0000000000000..f59f5eb5d78cd --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineShuffleOptions.java @@ -0,0 +1,58 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +/** + * Options for Shuffle workers. Most users should not need to adjust the settings in this section. + */ +public interface DataflowPipelineShuffleOptions { + /** + * Disk source image to use by shuffle VMs for jobs. + * @see Compute Engine Images + */ + @Description("Dataflow shuffle VM disk image.") + String getShuffleDiskSourceImage(); + void setShuffleDiskSourceImage(String value); + + /** + * Number of workers to use with the shuffle appliance, or 0 to use + * the default number of workers. + */ + @Description("Number of shuffle workers, when using remote execution") + int getShuffleNumWorkers(); + void setShuffleNumWorkers(int value); + + /** + * Remote shuffle worker disk size, in gigabytes, or 0 to use the + * default size. + */ + @Description("Remote shuffle worker disk size, in gigabytes, or 0 to use the default size.") + int getShuffleDiskSizeGb(); + void setShuffleDiskSizeGb(int value); + + /** + * GCE availability zone for launching shuffle workers. + * + *

Default is up to the service. + */ + @Description("GCE availability zone for launching shuffle workers. " + + "Default is up to the service") + String getShuffleZone(); + void setShuffleZone(String value); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java new file mode 100644 index 0000000000000..6cd9839318630 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java @@ -0,0 +1,116 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import java.util.List; + +/** + * Options which are used to configure the Dataflow pipeline worker pool. + */ +public interface DataflowPipelineWorkerPoolOptions { + /** + * Disk source image to use by VMs for jobs. + * @see Compute Engine Images + */ + @Description("Dataflow VM disk image.") + String getDiskSourceImage(); + void setDiskSourceImage(String value); + + /** + * Number of workers to use in remote execution. + */ + @Description("Number of workers, when using remote execution") + @Default.Integer(3) + int getNumWorkers(); + void setNumWorkers(int value); + + /** + * Remote worker disk size, in gigabytes, or 0 to use the default size. + */ + @Description("Remote worker disk size, in gigabytes, or 0 to use the default size.") + int getDiskSizeGb(); + void setDiskSizeGb(int value); + + /** + * GCE availability zone for launching workers. + * + *

Default is up to the service. + */ + @Description("GCE availability zone for launching workers. " + + "Default is up to the service") + String getZone(); + void setZone(String value); + + /** + * Type of API for handling cluster management,i.e. resizing, healthchecking, etc. + */ + public enum ClusterManagerApiType { + COMPUTE_ENGINE("compute.googleapis.com"), + REPLICA_POOL("replicapool.googleapis.com"); + + private final String apiServiceName; + + private ClusterManagerApiType(String apiServiceName) { + this.apiServiceName = apiServiceName; + } + + public String getApiServiceName() { + return this.apiServiceName; + } + } + + @Description("Type of API for handling cluster management,i.e. resizing, healthchecking, etc.") + @Default.InstanceFactory(ClusterManagerApiTypeFactory.class) + ClusterManagerApiType getClusterManagerApi(); + void setClusterManagerApi(ClusterManagerApiType value); + + /** Returns the default COMPUTE_ENGINE ClusterManagerApiType. */ + public static class ClusterManagerApiTypeFactory implements + DefaultValueFactory { + @Override + public ClusterManagerApiType create(PipelineOptions options) { + return ClusterManagerApiType.COMPUTE_ENGINE; + } + } + + /** + * Machine type to create worker VMs as. + */ + @Description("Dataflow VM machine type for workers.") + String getWorkerMachineType(); + void setWorkerMachineType(String value); + + /** + * Machine type to create VMs as. + */ + @Description("Dataflow VM machine type.") + String getMachineType(); + void setMachineType(String value); + + /** + * List of local files to make available to workers. + *

+ * Jars are placed on the worker's classpath. + *

+ * The default value is the list of jars from the main program's classpath. + */ + @Description("Files to stage on GCS and make available to " + + "workers. The default value is all files from the classpath.") + List getFilesToStage(); + void setFilesToStage(List value); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerHarnessOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerHarnessOptions.java new file mode 100644 index 0000000000000..0b8e1f809cc23 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerHarnessOptions.java @@ -0,0 +1,35 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +/** + * Options which are used exclusively within the Dataflow worker harness. + * These options have no effect at pipeline creation time. + */ +public interface DataflowWorkerHarnessOptions extends DataflowPipelineOptions { + /** + * ID of the worker running this pipeline. + */ + String getWorkerId(); + void setWorkerId(String value); + + /** + * ID of the job this pipeline represents. + */ + String getJobId(); + void setJobId(String value); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Default.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Default.java new file mode 100644 index 0000000000000..321fe744ca49b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Default.java @@ -0,0 +1,130 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * {@link Default} represents a set of annotations which can be used to annotate getter properties + * on {@link PipelineOptions} with information representing the default value to be returned + * if no value is specified. + */ +public @interface Default { + /** + * This represents that the default of the option is the specified {@link java.lang.Class} value. + */ + @Target(ElementType.METHOD) + @Retention(RetentionPolicy.RUNTIME) + public @interface Class { + java.lang.Class value(); + } + + /** + * This represents that the default of the option is the specified {@link java.lang.String} + * value. + */ + @Target(ElementType.METHOD) + @Retention(RetentionPolicy.RUNTIME) + public @interface String { + java.lang.String value(); + } + + /** + * This represents that the default of the option is the specified boolean primitive value. + */ + @Target(ElementType.METHOD) + @Retention(RetentionPolicy.RUNTIME) + public @interface Boolean { + boolean value(); + } + + /** + * This represents that the default of the option is the specified char primitive value. + */ + @Target(ElementType.METHOD) + @Retention(RetentionPolicy.RUNTIME) + public @interface Character { + char value(); + } + + /** + * This represents that the default of the option is the specified byte primitive value. + */ + @Target(ElementType.METHOD) + @Retention(RetentionPolicy.RUNTIME) + public @interface Byte { + byte value(); + } + /** + * This represents that the default of the option is the specified short primitive value. + */ + @Target(ElementType.METHOD) + @Retention(RetentionPolicy.RUNTIME) + public @interface Short { + short value(); + } + /** + * This represents that the default of the option is the specified int primitive value. + */ + @Target(ElementType.METHOD) + @Retention(RetentionPolicy.RUNTIME) + public @interface Integer { + int value(); + } + + /** + * This represents that the default of the option is the specified long primitive value. + */ + @Target(ElementType.METHOD) + @Retention(RetentionPolicy.RUNTIME) + public @interface Long { + long value(); + } + + /** + * This represents that the default of the option is the specified float primitive value. + */ + @Target(ElementType.METHOD) + @Retention(RetentionPolicy.RUNTIME) + public @interface Float { + float value(); + } + + /** + * This represents that the default of the option is the specified double primitive value. + */ + @Target(ElementType.METHOD) + @Retention(RetentionPolicy.RUNTIME) + public @interface Double { + double value(); + } + + /** + * Value must be of type {@link DefaultValueFactory} and have a default constructor. + * Value is instantiated and then used as a type factory to generate the default. + *

+ * See {@link DefaultValueFactory} for more details. + */ + @Target(ElementType.METHOD) + @Retention(RetentionPolicy.RUNTIME) + public @interface InstanceFactory { + java.lang.Class> value(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DefaultValueFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DefaultValueFactory.java new file mode 100644 index 0000000000000..18fd7827798c5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DefaultValueFactory.java @@ -0,0 +1,38 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +/** + * An interface used with {@link Default.InstanceFactory} annotation to specify the class which will + * be an instance factory to produce default values for a given getter on {@link PipelineOptions}. + * When a property on a {@link PipelineOptions} is fetched, and is currently unset, the default + * value factory will be instantiated and invoked. + *

+ * Care must be taken to not produce an infinite loop when accessing other fields on the + * {@link PipelineOptions} object. + * + * @param The type of object this factory produces. + */ +public interface DefaultValueFactory { + /** + * Creates a default value for a getter marked with {@link Default.InstanceFactory}. + * + * @param options The current pipeline options. + * @return The default value to be used for the given pipeline options. + */ + T create(PipelineOptions options); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Description.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Description.java new file mode 100644 index 0000000000000..9de8b1cd25805 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Description.java @@ -0,0 +1,31 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Machine-readable description for options in {@link PipelineOptions}. + */ +@Target(value = ElementType.METHOD) +@Retention(RetentionPolicy.RUNTIME) +public @interface Description { + String value(); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DirectPipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DirectPipelineOptions.java new file mode 100644 index 0000000000000..85a280d991934 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DirectPipelineOptions.java @@ -0,0 +1,28 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.cloud.dataflow.sdk.runners.DirectPipeline; + +/** + * Options which can be used to configure the {@link DirectPipeline}. + */ +public interface DirectPipelineOptions extends + ApplicationNameOptions, BigQueryOptions, GcsOptions, GcpOptions, + PipelineOptions, StreamingOptions { + +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java new file mode 100644 index 0000000000000..7dbaa5fb32d9f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java @@ -0,0 +1,150 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.api.client.auth.oauth2.Credential; +import com.google.cloud.dataflow.sdk.util.Credentials; + +import com.fasterxml.jackson.annotation.JsonIgnore; + +import java.io.File; +import java.io.IOException; +import java.security.GeneralSecurityException; + +/** + * Options used to configure Google Cloud Platform project and credentials. + *

+ * These options configure which of the following 4 different mechanisms for obtaining a credential + * are used: + *

    + *
  1. + * It can fetch the + * + * application default credentials. + *
  2. + *
  3. + * It can run the gcloud tool in a subprocess to obtain a credential. + * This is the preferred mechanism. The property "GCloudPath" can be + * used to specify where we search for gcloud data. + *
  4. + *
  5. + * The user can specify a client secrets file and go through the OAuth2 + * webflow. The credential will then be cached in the user's home + * directory for reuse. + *
  6. + *
  7. + * The user can specify a file containing a service account private key along + * with the service account name. + *
  8. + *
+ * The default mechanism is to use the + * + * application default credentials falling back to gcloud. The other options can be + * used by setting the corresponding properties. + */ +public interface GcpOptions extends PipelineOptions { + /** + * Project id to use when launching jobs. + */ + @Description("Project id. Required when running a Dataflow in the cloud.") + String getProject(); + void setProject(String value); + + /** + * This option controls which file to use when attempting to create the credentials using the + * OAuth 2 webflow. + */ + @Description("Path to a file containing Google API secret") + String getSecretsFile(); + void setSecretsFile(String value); + + /** + * This option controls which file to use when attempting to create the credentials using the + * service account method. + *

+ * This option if specified, needs be combined with the + * {@link GcpOptions#getServiceAccountName() serviceAccountName}. + */ + @Description("Path to a file containing the P12 service credentials") + String getServiceAccountKeyfile(); + void setServiceAccountKeyfile(String value); + + /** + * This option controls which service account to use when attempting to create the credentials + * using the service account method. + *

+ * This option if specified, needs be combined with the + * {@link GcpOptions#getServiceAccountKeyfile() serviceAccountKeyfile}. + */ + @Description("Name of the service account for Google APIs") + String getServiceAccountName(); + void setServiceAccountName(String value); + + @Description("The path to the gcloud binary. " + + " Default is to search the system path.") + String getGCloudPath(); + void setGCloudPath(String value); + + /** + * Directory for storing dataflow credentials. + */ + @Description("Directory for storing dataflow credentials") + @Default.InstanceFactory(CredentialDirFactory.class) + String getCredentialDir(); + void setCredentialDir(String value); + + /** + * Returns the default credential directory of ${user.home}/.store/data-flow. + */ + public static class CredentialDirFactory implements DefaultValueFactory { + @Override + public String create(PipelineOptions options) { + File home = new File(System.getProperty("user.home")); + File store = new File(home, ".store"); + File dataflow = new File(store, "data-flow"); + return dataflow.getPath(); + } + } + + @Description("The credential identifier when using a persistent" + + " credential store") + @Default.String("cloud_dataflow") + String getCredentialId(); + void setCredentialId(String value); + + /** Alternative Google Cloud Platform Credential */ + @JsonIgnore + @Description("Google Cloud Platform user credentials.") + @Default.InstanceFactory(GcpUserCredentialsFactory.class) + Credential getGcpCredential(); + void setGcpCredential(Credential value); + + /** + * Attempts to load the user credentials. See + * {@link Credentials#getUserCredential(GcpOptions)} for more details. + */ + public static class GcpUserCredentialsFactory implements DefaultValueFactory { + @Override + public Credential create(PipelineOptions options) { + try { + return Credentials.getUserCredential(options.as(GcpOptions.class)); + } catch (IOException | GeneralSecurityException e) { + throw new RuntimeException("Unable to obtain credential", e); + } + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java new file mode 100644 index 0000000000000..543c9cac6c406 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java @@ -0,0 +1,77 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.cloud.dataflow.sdk.util.AppEngineEnvironment; +import com.google.cloud.dataflow.sdk.util.GcsUtil; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +import com.fasterxml.jackson.annotation.JsonIgnore; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +/** + * Options used to configure Google Cloud Storage. + */ +public interface GcsOptions extends + ApplicationNameOptions, GcpOptions, PipelineOptions { + /** Alternative GcsUtil instance */ + @JsonIgnore + @Default.InstanceFactory(GcsUtil.GcsUtilFactory.class) + GcsUtil getGcsUtil(); + void setGcsUtil(GcsUtil value); + + //////////////////////////////////////////////////////////////////////////// + // Allows the user to provide an alternative ExecutorService if their + // environment does not support the default implementation. + @JsonIgnore + @Default.InstanceFactory(ExecutorServiceFactory.class) + ExecutorService getExecutorService(); + void setExecutorService(ExecutorService value); + + /** + * Returns the default {@link ExecutorService} to use within the Dataflow SDK. The + * {@link ExecutorService} is compatible with AppEngine. + */ + public static class ExecutorServiceFactory implements DefaultValueFactory { + @Override + public ExecutorService create(PipelineOptions options) { + ThreadFactoryBuilder threadFactoryBuilder = new ThreadFactoryBuilder(); + threadFactoryBuilder.setThreadFactory(MoreExecutors.platformThreadFactory()); + if (!AppEngineEnvironment.IS_APP_ENGINE) { + // AppEngine doesn't allow modification of threads to be daemon threads. + threadFactoryBuilder.setDaemon(true); + } + /* The SDK requires an unbounded thread pool because a step may create X writers + * each requiring their own thread to perform the writes otherwise a writer may + * block causing deadlock for the step because the writers buffer is full. + * Also, the MapTaskExecutor launches the steps in reverse order and completes + * them in forward order thus requiring enough threads so that each step's writers + * can be active. + */ + return new ThreadPoolExecutor( + 0, Integer.MAX_VALUE, // Allow an unlimited number of re-usable threads. + Long.MAX_VALUE, TimeUnit.NANOSECONDS, // Keep non-core threads alive forever. + new SynchronousQueue(), + threadFactoryBuilder.build()); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java new file mode 100644 index 0000000000000..d626b90d3c520 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java @@ -0,0 +1,62 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.cloud.dataflow.sdk.options.ProxyInvocationHandler.Deserializer; +import com.google.cloud.dataflow.sdk.options.ProxyInvocationHandler.Serializer; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; + +/** + * Dataflow SDK pipeline configuration options. + *

+ * Serialization + *

+ * For runners which execute their work remotely, every property available within PipelineOptions + * must either be serializable using Jackson's {@link ObjectMapper} or the getter method for the + * property annotated with {@link JsonIgnore @JsonIgnore}. + *

+ * It is an error to have the same property available in multiple interfaces with only some + * of them being annotated with {@link JsonIgnore @JsonIgnore}. It is also an error to mark a + * setter for a property with {@link JsonIgnore @JsonIgnore}. + */ +@JsonSerialize(using = Serializer.class) +@JsonDeserialize(using = Deserializer.class) +public interface PipelineOptions { + /** + * Transforms this object into an object of type . must extend {@link PipelineOptions}. + *

+ * If is not registered with the {@link PipelineOptionsFactory}, then we attempt to + * verify that is composable with every interface that this instance of the PipelineOptions + * has seen. + * + * @param kls The class of the type to transform to. + * @return An object of type kls. + */ + T as(Class kls); + + @Validation.Required + @Description("The runner which will be used when executing the pipeline.") + @Default.Class(DirectPipelineRunner.class) + Class> getRunner(); + void setRunner(Class> kls); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java new file mode 100644 index 0000000000000..89a31b07e888a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java @@ -0,0 +1,862 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.cloud.dataflow.sdk.PipelineResult; +import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; +import com.google.cloud.dataflow.sdk.runners.worker.DataflowWorkerHarness; +import com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineOptions; +import com.google.common.base.Equivalence; +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.base.Throwables; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableListMultimap; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.ListMultimap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Queues; +import com.google.common.collect.SetMultimap; +import com.google.common.collect.Sets; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.beans.BeanInfo; +import java.beans.IntrospectionException; +import java.beans.Introspector; +import java.beans.PropertyDescriptor; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.lang.reflect.Proxy; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.SortedMap; +import java.util.SortedSet; + +/** + * Constructs a {@link PipelineOptions} or any derived interface which is composable to any other + * derived interface of {@link PipelineOptions} via the {@link PipelineOptions#as} method. Being + * able to compose one derived interface of {@link PipelineOptions} to another has the following + * restrictions: + *

    + *
  • Any property with the same name must have the same return type for all derived interfaces + * of {@link PipelineOptions}. + *
  • Every bean property of any interface derived from {@link PipelineOptions} must have a + * getter and setter method. + *
  • Every method must conform to being a getter or setter for a JavaBean. + *
  • The derived interface of {@link PipelineOptions} must be composable with every interface + * registered with this factory. + *
+ *

+ * See the JavaBeans + * specification for more details as to what constitutes a property. + */ +public class PipelineOptionsFactory { + + /** + * Creates and returns an object which implements {@link PipelineOptions}. + * This sets the {@link ApplicationNameOptions#getAppName() "appName"} to the calling + * {@link Class#getSimpleName() classes simple name}. + * + * @return An object which implements {@link PipelineOptions}. + */ + public static PipelineOptions create() { + return new Builder(getAppName(3)).as(PipelineOptions.class); + } + + /** + * Creates and returns an object which implements @{code }. + * This sets the {@link ApplicationNameOptions#getAppName() "appName"} to the calling + * {@link Class#getSimpleName() classes simple name}. + *

+ * Note that @{code } must be composable with every registered interface with this factory. + * See {@link PipelineOptionsFactory#validateWellFormed(Class, Set)} for more details. + * + * @return An object which implements @{code }. + */ + public static T as(Class klass) { + return new Builder(getAppName(3)).as(klass); + } + + /** + * Sets the command line arguments to parse when constructing the {@link PipelineOptions}. + *

+ * Example GNU style command line arguments: + *

+   *   --project=MyProject (simple property, will set the "project" property to "MyProject")
+   *   --readOnly=true (for boolean properties, will set the "readOnly" property to "true")
+   *   --readOnly (shorthand for boolean properties, will set the "readOnly" property to "true")
+   *   --x=1 --x=2 --x=3 (list style property, will set the "x" property to [1, 2, 3])
+   *   --x=1,2,3 (shorthand list style property, will set the "x" property to [1, 2, 3])
+   * 
+ * Properties are able to bound to {@link String} and Java primitives @{code boolean}, + * @{code byte}, @{code short}, @{code int}, @{code long}, @{code float}, @{code double} and + * their primitive wrapper classes. + *

+ * List style properties are able to be bound to @{code boolean[]}, @{code char[]}, + * @{code short[]}, @{code int[]}, @{code long[]}, @{code float[]}, @{code double[]}, + * @{code String[]} and @{code List}. + */ + public static Builder fromArgs(String[] args) { + return new Builder(getAppName(3)).fromArgs(args); + } + + /** + * After creation we will validate that {@link PipelineOptions} conforms to all the + * validation criteria from {@code }. See + * {@link PipelineOptionsValidator#validate(Class, PipelineOptions)} for more details about + * validation. + */ + public Builder withValidation() { + return new Builder(getAppName(3)).withValidation(); + } + + /** A fluent PipelineOptions builder. */ + public static class Builder { + private final String defaultAppName; + private final String[] args; + private final boolean validation; + + // Do not allow direct instantiation + private Builder(String defaultAppName) { + this(defaultAppName, null, false); + } + + private Builder(String defaultAppName, String[] args, boolean validation) { + this.defaultAppName = defaultAppName; + this.args = args; + this.validation = validation; + } + + /** + * Sets the command line arguments to parse when constructing the {@link PipelineOptions}. + *

+ * Example GNU style command line arguments: + *

+     *   --project=MyProject (simple property, will set the "project" property to "MyProject")
+     *   --readOnly=true (for boolean properties, will set the "readOnly" property to "true")
+     *   --readOnly (shorthand for boolean properties, will set the "readOnly" property to "true")
+     *   --x=1 --x=2 --x=3 (list style property, will set the "x" property to [1, 2, 3])
+     *   --x=1,2,3 (shorthand list style property, will set the "x" property to [1, 2, 3])
+     * 
+ * Properties are able to bound to {@link String} and Java primitives @{code boolean}, + * @{code byte}, @{code short}, @{code int}, @{code long}, @{code float}, @{code double} and + * their primitive wrapper classes. + *

+ * List style properties are able to be bound to @{code boolean[]}, @{code char[]}, + * @{code short[]}, @{code int[]}, @{code long[]}, @{code float[]}, @{code double[]}, + * @{code String[]} and @{code List}. + */ + public Builder fromArgs(String[] args) { + Preconditions.checkNotNull(args, "Arguments should not be null."); + return new Builder(defaultAppName, args, validation); + } + + /** + * After creation we will validate that {@link PipelineOptions} conforms to all the + * validation criteria from {@code }. See + * {@link PipelineOptionsValidator#validate(Class, PipelineOptions)} for more details about + * validation. + */ + public Builder withValidation() { + return new Builder(defaultAppName, args, true); + } + + /** + * Creates and returns an object which implements {@link PipelineOptions} using the values + * configured on this builder during construction. + * + * @return An object which implements {@link PipelineOptions}. + */ + public PipelineOptions create() { + return as(PipelineOptions.class); + } + + /** + * Creates and returns an object which implements @{code } using the values configured on + * this builder during construction. + *

+ * Note that {@code } must be composable with every registered interface with this factory. + * See {@link PipelineOptionsFactory#validateWellFormed(Class, Set)} for more details. + * + * @return An object which implements @{code }. + */ + public T as(Class klass) { + Map initialOptions = Maps.newHashMap(); + + // Attempt to parse the arguments into the set of initial options to use + if (args != null) { + ListMultimap options = parseCommandLine(args); + LOG.debug("Provided Arguments: {}", options); + initialOptions = parseObjects(klass, options); + } + + // Create our proxy + ProxyInvocationHandler handler = new ProxyInvocationHandler(initialOptions); + T t = handler.as(klass); + + // Set the application name to the default if none was set. + ApplicationNameOptions appNameOptions = t.as(ApplicationNameOptions.class); + if (appNameOptions.getAppName() == null) { + appNameOptions.setAppName(defaultAppName); + } + + if (validation) { + PipelineOptionsValidator.validate(klass, t); + } + return t; + } + } + + /** + * Returns the simple name of calling class at the stack trace {@code level}. + */ + private static String getAppName(int level) { + StackTraceElement[] stackTrace = Thread.currentThread().getStackTrace(); + try { + return Class.forName(stackTrace[level].getClassName()).getSimpleName(); + } catch (ClassNotFoundException e) { + return "unknown"; + } + } + + /** + * Stores the generated proxyClass and its respective {@link BeanInfo} object. + * + * @param The type of the proxyClass. + */ + static class Registration { + private final Class proxyClass; + private final List propertyDescriptors; + + public Registration(Class proxyClass, List beanInfo) { + this.proxyClass = proxyClass; + this.propertyDescriptors = beanInfo; + } + + List getPropertyDescriptors() { + return propertyDescriptors; + } + + Class getProxyClass() { + return proxyClass; + } + } + + + private static final Logger LOG = LoggerFactory.getLogger(PipelineOptionsFactory.class); + private static final Class[] EMPTY_CLASS_ARRAY = new Class[0]; + private static final ObjectMapper MAPPER = new ObjectMapper(); + + // TODO: Add dynamic registration of pipeline runners. + private static final Map>> + SUPPORTED_PIPELINE_RUNNERS = + ImmutableMap.>>builder() + .put(DirectPipelineRunner.class.getSimpleName(), + DirectPipelineRunner.class) + .put(DataflowPipelineRunner.class.getSimpleName(), + DataflowPipelineRunner.class) + .put(BlockingDataflowPipelineRunner.class.getSimpleName(), + BlockingDataflowPipelineRunner.class) + .build(); + + /** Methods which are ignored when validating the proxy class */ + private static final Set IGNORED_METHODS; + + /** The set of options which have been registered and visible to the user. */ + private static final Set> REGISTERED_OPTIONS = + Sets.newConcurrentHashSet(); + + /** A cache storing a mapping from a given interface to its registration record. */ + private static final Map, Registration> INTERFACE_CACHE = + Maps.newConcurrentMap(); + + /** A cache storing a mapping from a set of interfaces to its registration record. */ + private static final Map>, Registration> COMBINED_CACHE = + Maps.newConcurrentMap(); + + static { + try { + IGNORED_METHODS = ImmutableSet.builder() + .add(Object.class.getMethod("getClass")) + .add(Object.class.getMethod("wait")) + .add(Object.class.getMethod("wait", long.class)) + .add(Object.class.getMethod("wait", long.class, int.class)) + .add(Object.class.getMethod("notify")) + .add(Object.class.getMethod("notifyAll")) + .add(Proxy.class.getMethod("getInvocationHandler", Object.class)) + .build(); + } catch (NoSuchMethodException | SecurityException e) { + LOG.error("Unable to find expected method", e); + throw new ExceptionInInitializerError(e); + } + + // TODO Add support for dynamically loading and registering the options interfaces. + register(PipelineOptions.class); + register(DirectPipelineOptions.class); + register(DataflowPipelineOptions.class); + register(BlockingDataflowPipelineOptions.class); + register(TestDataflowPipelineOptions.class); + } + + /** + * This registers the interface with this factory. This interface must conform to the following + * restrictions: + *

    + *
  • Any property with the same name must have the same return type for all derived + * interfaces of {@link PipelineOptions}. + *
  • Every bean property of any interface derived from {@link PipelineOptions} must have a + * getter and setter method. + *
  • Every method must conform to being a getter or setter for a JavaBean. + *
  • The derived interface of {@link PipelineOptions} must be composable with every interface + * registered with this factory. + *
+ * + * @param iface The interface object to manually register. + */ + public static synchronized void register(Class iface) { + Preconditions.checkNotNull(iface); + Preconditions.checkArgument(iface.isInterface(), "Only interface types are supported."); + + if (REGISTERED_OPTIONS.contains(iface)) { + return; + } + validateWellFormed(iface, REGISTERED_OPTIONS); + REGISTERED_OPTIONS.add(iface); + } + + /** + * Validates that the interface conforms to the following: + *
    + *
  • Any property with the same name must have the same return type for all derived + * interfaces of {@link PipelineOptions}. + *
  • Every bean property of any interface derived from {@link PipelineOptions} must have a + * getter and setter method. + *
  • Every method must conform to being a getter or setter for a JavaBean. + *
  • The derived interface of {@link PipelineOptions} must be composable with every interface + * part of allPipelineOptionsClasses. + *
  • Only getters may be annotated with {@link JsonIgnore @JsonIgnore}. + *
  • If any getter is annotated with {@link JsonIgnore @JsonIgnore}, then all getters for + * this property must be annotated with {@link JsonIgnore @JsonIgnore}. + *
+ * + * @param iface The interface to validate. + * @param validatedPipelineOptionsInterfaces The set of validated pipeline options interfaces to + * validate against. + * @return A registration record containing the proxy class and bean info for iface. + */ + static synchronized Registration validateWellFormed( + Class iface, Set> validatedPipelineOptionsInterfaces) { + Preconditions.checkArgument(iface.isInterface(), "Only interface types are supported."); + + Set> combinedPipelineOptionsInterfaces = + FluentIterable.from(validatedPipelineOptionsInterfaces).append(iface).toSet(); + // Validate that the view of all currently passed in options classes is well formed. + if (!COMBINED_CACHE.containsKey(combinedPipelineOptionsInterfaces)) { + Class allProxyClass = Proxy.getProxyClass(PipelineOptionsFactory.class.getClassLoader(), + combinedPipelineOptionsInterfaces.toArray(EMPTY_CLASS_ARRAY)); + try { + List propertyDescriptors = + getPropertyDescriptors(allProxyClass); + validateClass(iface, validatedPipelineOptionsInterfaces, + allProxyClass, propertyDescriptors); + COMBINED_CACHE.put(combinedPipelineOptionsInterfaces, + new Registration((Class) allProxyClass, propertyDescriptors)); + } catch (IntrospectionException e) { + throw Throwables.propagate(e); + } + } + + // Validate that the local view of the class is well formed. + if (!INTERFACE_CACHE.containsKey(iface)) { + Class proxyClass = Proxy.getProxyClass( + PipelineOptionsFactory.class.getClassLoader(), new Class[] {iface}); + try { + List propertyDescriptors = + getPropertyDescriptors(proxyClass); + validateClass(iface, validatedPipelineOptionsInterfaces, proxyClass, propertyDescriptors); + INTERFACE_CACHE.put(iface, + new Registration((Class) proxyClass, propertyDescriptors)); + } catch (IntrospectionException e) { + throw Throwables.propagate(e); + } + } + return (Registration) INTERFACE_CACHE.get(iface); + } + + public static Set> getRegisteredOptions() { + return Collections.unmodifiableSet(REGISTERED_OPTIONS); + } + + static List getPropertyDescriptors( + Set> interfaces) { + return COMBINED_CACHE.get(interfaces).getPropertyDescriptors(); + } + + + /** + * Creates a set of {@link DataflowWorkerHarnessOptions} based of a set of known system + * properties. This is meant to only be used from the {@link DataflowWorkerHarness} as a method to + * bootstrap the worker harness. + * + * @return A {@link DataflowWorkerHarnessOptions} object configured for the + * {@link DataflowWorkerHarness}. + */ + @Deprecated + public static DataflowWorkerHarnessOptions createFromSystemProperties() { + DataflowWorkerHarnessOptions options = as(DataflowWorkerHarnessOptions.class); + options.setRunner(null); + if (System.getProperties().containsKey("root_url")) { + options.setApiRootUrl(System.getProperty("root_url")); + } + if (System.getProperties().containsKey("service_path")) { + options.setDataflowEndpoint(System.getProperty("service_path")); + } + if (System.getProperties().containsKey("temp_gcs_directory")) { + options.setTempLocation(System.getProperty("temp_gcs_directory")); + } + if (System.getProperties().containsKey("service_account_name")) { + options.setServiceAccountName(System.getProperty("service_account_name")); + } + if (System.getProperties().containsKey("service_account_keyfile")) { + options.setServiceAccountKeyfile(System.getProperty("service_account_keyfile")); + } + if (System.getProperties().containsKey("worker_id")) { + options.setWorkerId(System.getProperty("worker_id")); + } + if (System.getProperties().containsKey("project_id")) { + options.setProject(System.getProperty("project_id")); + } + if (System.getProperties().containsKey("job_id")) { + options.setJobId(System.getProperty("job_id")); + } + return options; + } + + /** + * Returns all the methods visible from the provided interfaces. + * + * @param interfaces The interfaces to use when searching for all their methods. + * @return An iterable of {@link Method}s which interfaces expose. + */ + static Iterable getClosureOfMethodsOnInterfaces( + Iterable> interfaces) { + return FluentIterable.from(interfaces).transformAndConcat( + new Function, Iterable>() { + @Override + public Iterable apply(Class input) { + return getClosureOfMethodsOnInterface(input); + } + }); + } + + /** + * Returns all the methods visible from {@code iface}. + * + * @param iface The interface to use when searching for all its methods. + * @return An iterable of {@link Method}s which {@code iface} exposes. + */ + static Iterable getClosureOfMethodsOnInterface(Class iface) { + Preconditions.checkNotNull(iface); + Preconditions.checkArgument(iface.isInterface()); + ImmutableList.Builder builder = ImmutableList.builder(); + Queue> interfacesToProcess = Queues.newArrayDeque(); + interfacesToProcess.add(iface); + while (!interfacesToProcess.isEmpty()) { + Class current = interfacesToProcess.remove(); + builder.add(current.getMethods()); + interfacesToProcess.addAll(Arrays.asList(current.getInterfaces())); + } + return builder.build(); + } + + /** + * This method is meant to emulate the behavior of {@link Introspector#getBeanInfo(Class, int)} + * to construct the list of {@link PropertyDescriptor}. + *

+ * TODO: Swap back to using Introspector once the proxy class issue with AppEngine is resolved. + */ + private static List getPropertyDescriptors(Class beanClass) + throws IntrospectionException { + // The sorting is important to make this method stable. + SortedSet methods = Sets.newTreeSet(MethodComparator.INSTANCE); + methods.addAll(Arrays.asList(beanClass.getMethods())); + // Build a map of property names to getters. + SortedMap propertyNamesToGetters = Maps.newTreeMap(); + for (Method method : methods) { + String methodName = method.getName(); + if ((!methodName.startsWith("get") + && !methodName.startsWith("is")) + || method.getParameterTypes().length != 0 + || method.getReturnType() == void.class) { + continue; + } + String propertyName = Introspector.decapitalize( + methodName.startsWith("is") ? methodName.substring(2) : methodName.substring(3)); + propertyNamesToGetters.put(propertyName, method); + } + + List descriptors = Lists.newArrayList(); + + /* + * Add all the getter/setter pairs to the list of descriptors removing the getter once + * it has been paired up. + */ + for (Method method : methods) { + String methodName = method.getName(); + if (!methodName.startsWith("set") + || method.getParameterTypes().length != 1 + || method.getReturnType() != void.class) { + continue; + } + String propertyName = Introspector.decapitalize(methodName.substring(3)); + descriptors.add(new PropertyDescriptor( + propertyName, propertyNamesToGetters.remove(propertyName), method)); + } + + // Add the remaining getters with missing setters. + for (Map.Entry getterToMethod : propertyNamesToGetters.entrySet()) { + descriptors.add(new PropertyDescriptor( + getterToMethod.getKey(), getterToMethod.getValue(), null)); + } + return descriptors; + } + + /** + * Validates that a given class conforms to the following properties: + *

    + *
  • Any property with the same name must have the same return type for all derived + * interfaces of {@link PipelineOptions}. + *
  • Every bean property of any interface derived from {@link PipelineOptions} must have a + * getter and setter method. + *
  • Every method must conform to being a getter or setter for a JavaBean. + *
  • Only getters may be annotated with {@link JsonIgnore @JsonIgnore}. + *
  • If any getter is annotated with {@link JsonIgnore @JsonIgnore}, then all getters for + * this property must be annotated with {@link JsonIgnore @JsonIgnore}. + *
+ * + * @param iface The interface to validate. + * @param validatedPipelineOptionsInterfaces The set of validated pipeline options interfaces to + * validate against. + * @param klass The proxy class representing the interface. + * @param descriptors A list of {@link PropertyDescriptor}s to use when validating. + */ + private static void validateClass(Class iface, + Set> validatedPipelineOptionsInterfaces, + Class klass, List descriptors) { + Set methods = Sets.newHashSet(IGNORED_METHODS); + // Ignore static methods, "equals", "hashCode", "toString" and "as" on the generated class. + for (Method method : klass.getMethods()) { + if (Modifier.isStatic(method.getModifiers())) { + methods.add(method); + } + } + try { + methods.add(klass.getMethod("equals", Object.class)); + methods.add(klass.getMethod("hashCode")); + methods.add(klass.getMethod("toString")); + methods.add(klass.getMethod("as", Class.class)); + } catch (NoSuchMethodException | SecurityException e) { + throw Throwables.propagate(e); + } + + // Verify that there are no methods with the same name with two different return types. + Iterable interfaceMethods = FluentIterable + .from(getClosureOfMethodsOnInterface(iface)) + .toSortedSet(MethodComparator.INSTANCE); + SetMultimap, Method> methodNameToMethodMap = + HashMultimap.create(); + for (Method method : interfaceMethods) { + methodNameToMethodMap.put(MethodNameEquivalence.INSTANCE.wrap(method), method); + } + for (Map.Entry, Collection> entry + : methodNameToMethodMap.asMap().entrySet()) { + Set> returnTypes = FluentIterable.from(entry.getValue()) + .transform(ReturnTypeFetchingFunction.INSTANCE).toSet(); + SortedSet collidingMethods = FluentIterable.from(entry.getValue()) + .toSortedSet(MethodComparator.INSTANCE); + Preconditions.checkArgument(returnTypes.size() == 1, + "Method [%s] has multiple definitions %s with different return types for [%s].", + entry.getKey().get().getName(), + collidingMethods, + iface.getName()); + } + + // Verify that there is no getter with a mixed @JsonIgnore annotation and verify + // that no setter has @JsonIgnore. + Iterable allInterfaceMethods = FluentIterable + .from(getClosureOfMethodsOnInterfaces(validatedPipelineOptionsInterfaces)) + .append(getClosureOfMethodsOnInterface(iface)) + .toSortedSet(MethodComparator.INSTANCE); + SetMultimap, Method> methodNameToAllMethodMap = + HashMultimap.create(); + for (Method method : allInterfaceMethods) { + methodNameToAllMethodMap.put(MethodNameEquivalence.INSTANCE.wrap(method), method); + } + for (PropertyDescriptor descriptor : descriptors) { + if (IGNORED_METHODS.contains(descriptor.getReadMethod()) + || IGNORED_METHODS.contains(descriptor.getWriteMethod())) { + continue; + } + Set getters = + methodNameToAllMethodMap.get( + MethodNameEquivalence.INSTANCE.wrap(descriptor.getReadMethod())); + Set gettersWithJsonIgnore = + FluentIterable.from(getters).filter(JsonIgnorePredicate.INSTANCE).toSet(); + + Iterable getterClassNames = FluentIterable.from(getters) + .transform(MethodToDeclaringClassFunction.INSTANCE) + .transform(ClassNameFunction.INSTANCE); + Iterable gettersWithJsonIgnoreClassNames = FluentIterable.from(gettersWithJsonIgnore) + .transform(MethodToDeclaringClassFunction.INSTANCE) + .transform(ClassNameFunction.INSTANCE); + + Preconditions.checkArgument(gettersWithJsonIgnore.isEmpty() + || getters.size() == gettersWithJsonIgnore.size(), + "Expected getter for property [%s] to be marked with @JsonIgnore on all %s, " + + "found only on %s", + descriptor.getName(), getterClassNames, gettersWithJsonIgnoreClassNames); + + Set settersWithJsonIgnore = FluentIterable.from( + methodNameToAllMethodMap.get( + MethodNameEquivalence.INSTANCE.wrap(descriptor.getWriteMethod()))) + .filter(JsonIgnorePredicate.INSTANCE).toSet(); + + Iterable settersWithJsonIgnoreClassNames = FluentIterable.from(settersWithJsonIgnore) + .transform(MethodToDeclaringClassFunction.INSTANCE) + .transform(ClassNameFunction.INSTANCE); + + Preconditions.checkArgument(settersWithJsonIgnore.isEmpty(), + "Expected setter for property [%s] to not be marked with @JsonIgnore on %s", + descriptor.getName(), settersWithJsonIgnoreClassNames); + } + + // Verify that each property has a matching read and write method. + for (PropertyDescriptor propertyDescriptor : descriptors) { + Preconditions.checkArgument( + IGNORED_METHODS.contains(propertyDescriptor.getWriteMethod()) + || propertyDescriptor.getReadMethod() != null, + "Expected getter for property [%s] of type [%s] on [%s].", + propertyDescriptor.getName(), + propertyDescriptor.getPropertyType().getName(), + iface.getName()); + Preconditions.checkArgument( + IGNORED_METHODS.contains(propertyDescriptor.getReadMethod()) + || propertyDescriptor.getWriteMethod() != null, + "Expected setter for property [%s] of type [%s] on [%s].", + propertyDescriptor.getName(), + propertyDescriptor.getPropertyType().getName(), + iface.getName()); + methods.add(propertyDescriptor.getReadMethod()); + methods.add(propertyDescriptor.getWriteMethod()); + } + + // Verify that no additional methods are on an interface that aren't a bean property. + Set unknownMethods = Sets.difference(Sets.newHashSet(klass.getMethods()), methods); + Preconditions.checkArgument(unknownMethods.isEmpty(), + "Methods %s on [%s] do not conform to being bean properties.", + FluentIterable.from(unknownMethods).transform(MethodFormatterFunction.INSTANCE), + iface.getName()); + } + + /** A {@link Comparator} which uses the generic method signature to sort them. */ + private static class MethodComparator implements Comparator { + static final MethodComparator INSTANCE = new MethodComparator(); + @Override + public int compare(Method o1, Method o2) { + return o1.toGenericString().compareTo(o2.toGenericString()); + } + } + + /** A {@link Function} which gets the methods return type. */ + private static class ReturnTypeFetchingFunction implements Function> { + static final ReturnTypeFetchingFunction INSTANCE = new ReturnTypeFetchingFunction(); + @Override + public Class apply(Method input) { + return input.getReturnType(); + } + } + + /** A {@link Function} which turns a method into a simple method signature. */ + private static class MethodFormatterFunction implements Function { + static final MethodFormatterFunction INSTANCE = new MethodFormatterFunction(); + @Override + public String apply(Method input) { + String parameterTypes = FluentIterable.of(input.getParameterTypes()) + .transform(ClassNameFunction.INSTANCE) + .toSortedList(String.CASE_INSENSITIVE_ORDER) + .toString(); + return ClassNameFunction.INSTANCE.apply(input.getReturnType()) + " " + input.getName() + + "(" + parameterTypes.substring(1, parameterTypes.length() - 1) + ")"; + } + } + + /** A {@link Function} with returns the classes name. */ + private static class ClassNameFunction implements Function, String> { + static final ClassNameFunction INSTANCE = new ClassNameFunction(); + @Override + public String apply(Class input) { + return input.getName(); + } + } + + /** A {@link Function} with returns the declaring class for the method. */ + private static class MethodToDeclaringClassFunction implements Function> { + static final MethodToDeclaringClassFunction INSTANCE = new MethodToDeclaringClassFunction(); + @Override + public Class apply(Method input) { + return input.getDeclaringClass(); + } + } + + /** An {@link Equivalence} which considers two methods equivalent if they share the same name. */ + private static class MethodNameEquivalence extends Equivalence { + static final MethodNameEquivalence INSTANCE = new MethodNameEquivalence(); + @Override + protected boolean doEquivalent(Method a, Method b) { + return a.getName().equals(b.getName()); + } + + @Override + protected int doHash(Method t) { + return t.getName().hashCode(); + } + } + + /** + * A {@link Predicate} which returns true if the method is annotated with + * {@link JsonIgnore @JsonIgnore}. + */ + static class JsonIgnorePredicate implements Predicate { + static final JsonIgnorePredicate INSTANCE = new JsonIgnorePredicate(); + @Override + public boolean apply(Method input) { + return input.isAnnotationPresent(JsonIgnore.class); + } + } + + /** + * Splits string arguments based upon expected pattern of --argName=value. + *

+ * Example GNU style command line arguments: + *

+   *   --project=MyProject (simple property, will set the "project" property to "MyProject")
+   *   --readOnly=true (for boolean properties, will set the "readOnly" property to "true")
+   *   --readOnly (shorthand for boolean properties, will set the "readOnly" property to "true")
+   *   --x=1 --x=2 --x=3 (list style property, will set the "x" property to [1, 2, 3])
+   *   --x=1,2,3 (shorthand list style property, will set the "x" property to [1, 2, 3])
+   * 
+ * Properties are able to bound to {@link String} and Java primitives boolean, byte, + * short, int, long, float, double and their primitive wrapper classes. + *

+ * List style properties are able to be bound to boolean[], char[], short[], + * int[], long[], float[], double[], String[] and List. + *

+ */ + private static ListMultimap parseCommandLine(String[] args) { + ImmutableListMultimap.Builder builder = ImmutableListMultimap.builder(); + for (String arg : args) { + Preconditions.checkArgument(arg.startsWith("--"), + "Unknown argument %s in command line %s", arg, Arrays.toString(args)); + int index = arg.indexOf("="); + // Make sure that '=' isn't the first character after '--' or the last character + Preconditions.checkArgument(index != 2 && index != arg.length() - 1, + "Unknown argument %s in command line %s", arg, Arrays.toString(args)); + if (index > 0) { + builder.put(arg.substring(2, index), arg.substring(index + 1, arg.length())); + } else { + builder.put(arg.substring(2), "true"); + } + } + return builder.build(); + } + + /** + * Using the parsed string arguments, we convert the strings to the expected + * return type of the methods which are found on the passed in class. + *

+ * For any return type that is expected to be an array or a collection, we further + * split up each string on ','. + *

+ * We special case the "runner" option. It is mapped to the class of the {@link PipelineRunner} + * based off of the {@link PipelineRunner}s simple class name. + */ + private static Map parseObjects( + Class klass, ListMultimap options) { + Map propertyNamesToGetters = Maps.newHashMap(); + PipelineOptionsFactory.validateWellFormed(klass, getRegisteredOptions()); + Iterable propertyDescriptors = + PipelineOptionsFactory.getPropertyDescriptors( + FluentIterable.from(getRegisteredOptions()).append(klass).toSet()); + for (PropertyDescriptor descriptor : propertyDescriptors) { + propertyNamesToGetters.put(descriptor.getName(), descriptor.getReadMethod()); + } + Map convertedOptions = Maps.newHashMap(); + for (Map.Entry> entry : options.asMap().entrySet()) { + if (!propertyNamesToGetters.containsKey(entry.getKey())) { + LOG.warn("Ignoring argument {}={}", entry.getKey(), entry.getValue()); + continue; + } + + Method method = propertyNamesToGetters.get(entry.getKey()); + JavaType type = MAPPER.getTypeFactory().constructType(method.getGenericReturnType()); + if ("runner".equals(entry.getKey())) { + String runner = Iterables.getOnlyElement(entry.getValue()); + Preconditions.checkArgument(SUPPORTED_PIPELINE_RUNNERS.containsKey(runner), + "Unknown 'runner' specified %s, supported pipeline runners %s", + runner, SUPPORTED_PIPELINE_RUNNERS.keySet()); + convertedOptions.put("runner", SUPPORTED_PIPELINE_RUNNERS.get(runner)); + } else if (method.getReturnType().isArray() + || Collection.class.isAssignableFrom(method.getReturnType())) { + // Split any strings with "," + List values = FluentIterable.from(entry.getValue()) + .transformAndConcat(new Function>() { + @Override + public Iterable apply(String input) { + return Arrays.asList(input.split(",")); + } + }).toList(); + convertedOptions.put(entry.getKey(), MAPPER.convertValue(values, type)); + } else { + String value = Iterables.getOnlyElement(entry.getValue()); + convertedOptions.put(entry.getKey(), MAPPER.convertValue(value, type)); + } + } + return convertedOptions; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java new file mode 100644 index 0000000000000..bb7bcf3de831f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java @@ -0,0 +1,59 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.common.base.Preconditions; + +import java.lang.annotation.Annotation; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; + +/** + * Validates that the {@link PipelineOptions} conforms to all the {@link Validation} criteria. + */ +public class PipelineOptionsValidator { + /** + * Validates that the passed {@link PipelineOptions} conforms to all the validation criteria from + * the passed in interface. + *

+ * Note that the interface requested must conform to the validation criteria specified on + * {@link PipelineOptions#as(Class)}. + * + * @param klass The interface to fetch validation criteria from. + * @param options The {@link PipelineOptions} to validate. + * @return The type + */ + public static T validate(Class klass, PipelineOptions options) { + Preconditions.checkNotNull(klass); + Preconditions.checkNotNull(options); + Preconditions.checkArgument(Proxy.isProxyClass(options.getClass())); + Preconditions.checkArgument(Proxy.getInvocationHandler(options) + instanceof ProxyInvocationHandler); + + ProxyInvocationHandler handler = + (ProxyInvocationHandler) Proxy.getInvocationHandler(options); + for (Method method : PipelineOptionsFactory.getClosureOfMethodsOnInterface(klass)) { + for (Annotation annotation : method.getAnnotations()) { + if (annotation instanceof Validation.Required) { + Preconditions.checkArgument(handler.invoke(options, method, null) != null, + "Expected non-null property to be set for [" + method + "]."); + } + } + } + return options.as(klass); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java new file mode 100644 index 0000000000000..aefbe1dec294e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java @@ -0,0 +1,390 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory.JsonIgnorePredicate; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory.Registration; +import com.google.cloud.dataflow.sdk.util.InstanceBuilder; +import com.google.common.base.Defaults; +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.ClassToInstanceMap; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import com.google.common.collect.MutableClassToInstanceMap; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.node.ObjectNode; + +import java.beans.PropertyDescriptor; +import java.io.IOException; +import java.lang.annotation.Annotation; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; +import java.lang.reflect.Type; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; + +/** + * Represents and {@link InvocationHandler} for a {@link Proxy}. The invocation handler uses bean + * introspection of the proxy class to store and retrieve values based off of the property name. + *

+ * Unset properties use the {@Default} metadata on the getter to return values. If there + * is no {@Default} annotation on the getter, then a default as + * per the Java Language Specification for the expected return type is returned. + *

+ * In addition to the getter/setter pairs, this proxy invocation handler supports + * {@link Object#equals(Object)}, {@link Object#hashCode()}, {@link Object#toString()} and + * {@link PipelineOptions#as(Class)}. + */ +class ProxyInvocationHandler implements InvocationHandler { + private static final ObjectMapper MAPPER = new ObjectMapper(); + /** + * No two instances of this class are considered equivalent hence we generate a random hash code + * between 0 and {@link Integer#MAX_VALUE}. + */ + private final int hashCode = (int) Math.random() * Integer.MAX_VALUE; + private final Set> knownInterfaces; + private final ClassToInstanceMap interfaceToProxyCache; + private final Map options; + private final Map jsonOptions; + private final Map gettersToPropertyNames; + private final Map settersToPropertyNames; + + ProxyInvocationHandler(Map options) { + this(options, Maps.newHashMap()); + } + + private ProxyInvocationHandler(Map options, Map jsonOptions) { + this.options = options; + this.jsonOptions = jsonOptions; + this.knownInterfaces = new HashSet<>(PipelineOptionsFactory.getRegisteredOptions()); + gettersToPropertyNames = Maps.newHashMap(); + settersToPropertyNames = Maps.newHashMap(); + interfaceToProxyCache = MutableClassToInstanceMap.create(); + } + + @Override + public Object invoke(Object proxy, Method method, Object[] args) { + if (args == null && "toString".equals(method.getName())) { + return toString(); + } else if (args != null && args.length == 1 && "equals".equals(method.getName())) { + return equals(args[0]); + } else if (args == null && "hashCode".equals(method.getName())) { + return hashCode(); + } else if (args != null && "as".equals(method.getName()) && args[0] instanceof Class) { + return as((Class) args[0]); + } + String methodName = method.getName(); + synchronized (this) { + if (gettersToPropertyNames.keySet().contains(methodName)) { + String propertyName = gettersToPropertyNames.get(methodName); + if (!options.containsKey(propertyName)) { + // Lazy bind the default to the method. + Object value = jsonOptions.containsKey(propertyName) + ? getValueFromJson(propertyName, method) + : getDefault((PipelineOptions) proxy, method); + options.put(propertyName, value); + } + return options.get(propertyName); + } else if (settersToPropertyNames.containsKey(methodName)) { + options.put(settersToPropertyNames.get(methodName), args[0]); + return Void.TYPE; + } + } + throw new RuntimeException("Unknown method [" + method + "] invoked with args [" + + Arrays.toString(args) + "]."); + } + + /** + * Backing implementation for {@link PipelineOptions#as(Class)}. + * + * @param iface The interface which the returned object needs to implement. + * @return An object which implements the interface . + */ + synchronized T as(Class iface) { + Preconditions.checkNotNull(iface); + Preconditions.checkArgument(iface.isInterface()); + if (!interfaceToProxyCache.containsKey(iface)) { + Registration registration = + PipelineOptionsFactory.validateWellFormed(iface, knownInterfaces); + List propertyDescriptors = registration.getPropertyDescriptors(); + Class proxyClass = registration.getProxyClass(); + gettersToPropertyNames.putAll(generateGettersToPropertyNames(propertyDescriptors)); + settersToPropertyNames.putAll(generateSettersToPropertyNames(propertyDescriptors)); + knownInterfaces.add(iface); + interfaceToProxyCache.putInstance(iface, + InstanceBuilder.ofType(proxyClass) + .fromClass(proxyClass) + .withArg(InvocationHandler.class, this) + .build()); + } + return interfaceToProxyCache.getInstance(iface); + } + + + /** + * Returns true if the other object is a ProxyInvocationHandler or is a Proxy object and has the + * same ProxyInvocationHandler as this. + * + * @param obj The object to compare against this. + * @return true iff the other object is a ProxyInvocationHandler or is a Proxy object and has the + * same ProxyInvocationHandler as this. + */ + @Override + public boolean equals(Object obj) { + return obj != null && ((obj instanceof ProxyInvocationHandler && this == obj) + || (Proxy.isProxyClass(obj.getClass()) && this == Proxy.getInvocationHandler(obj))); + } + + /** + * Each instance of this ProxyInvocationHandler is unique and has a random hash code. + * + * @return A hash code that was generated randomly. + */ + @Override + public int hashCode() { + return hashCode; + } + + /** + * This will output all the currently set values. + * + * @return A string representation of this. + */ + @Override + public synchronized String toString() { + StringBuilder b = new StringBuilder(); + b.append("Current Settings:\n"); + for (Map.Entry entry : new TreeMap<>(options).entrySet()) { + b.append(" " + entry.getKey() + ": " + entry.getValue() + "\n"); + } + return b.toString(); + } + + /** + * Uses a Jackson {@link ObjectMapper} to attempt type conversion. + * + * @param method The method whose return type you would like to return. + * @param propertyName The name of the property which is being returned. + * @return An object matching the return type of the method passed in. + */ + private Object getValueFromJson(String propertyName, Method method) { + try { + JavaType type = MAPPER.getTypeFactory().constructType(method.getGenericReturnType()); + JsonNode jsonNode = jsonOptions.get(propertyName); + return MAPPER.readValue(jsonNode.toString(), type); + } catch (IOException e) { + throw new RuntimeException("Unable to parse representation", e); + } + } + + /** + * Returns a default value for the method based upon {@Default} metadata on the getter + * to return values. If there is no {@Default} annotation on the getter, then a default as + * per the Java Language Specification for the expected return type is returned. + * + * @param proxy The proxy object for which we are attempting to get the default. + * @param method The getter method which was invoked. + * @return The default value from an {@link Default} annotation if present, otherwise a default + * value as per the Java Language Specification. + */ + private Object getDefault(PipelineOptions proxy, Method method) { + for (Annotation annotation : method.getAnnotations()) { + if (annotation instanceof Default.Class) { + return ((Default.Class) annotation).value(); + } else if (annotation instanceof Default.String) { + return ((Default.String) annotation).value(); + } else if (annotation instanceof Default.Boolean) { + return ((Default.Boolean) annotation).value(); + } else if (annotation instanceof Default.Character) { + return ((Default.Character) annotation).value(); + } else if (annotation instanceof Default.Byte) { + return ((Default.Byte) annotation).value(); + } else if (annotation instanceof Default.Short) { + return ((Default.Short) annotation).value(); + } else if (annotation instanceof Default.Integer) { + return ((Default.Integer) annotation).value(); + } else if (annotation instanceof Default.Long) { + return ((Default.Long) annotation).value(); + } else if (annotation instanceof Default.Float) { + return ((Default.Float) annotation).value(); + } else if (annotation instanceof Default.Double) { + return ((Default.Double) annotation).value(); + } else if (annotation instanceof Default.String) { + return ((Default.String) annotation).value(); + } else if (annotation instanceof Default.String) { + return ((Default.String) annotation).value(); + } else if (annotation instanceof Default.String) { + return ((Default.String) annotation).value(); + } else if (annotation instanceof Default.InstanceFactory) { + return InstanceBuilder.ofType(((Default.InstanceFactory) annotation).value()) + .build() + .create(proxy); + } + } + + /* + * We need to make sure that we return something appropriate for the return type. Thus we return + * a default value as defined by the JLS. + */ + return Defaults.defaultValue(method.getReturnType()); + } + + /** + * Returns a map from the getters method name to the name of the property based upon the passed in + * {@link PropertyDescriptor}s property descriptors. + * + * @param propertyDescriptors A list of {@link PropertyDescriptor}s to use when generating the + * map. + * @return A map of getter method name to property name. + */ + private static Map generateGettersToPropertyNames( + List propertyDescriptors) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (PropertyDescriptor descriptor : propertyDescriptors) { + if (descriptor.getReadMethod() != null) { + builder.put(descriptor.getReadMethod().getName(), descriptor.getName()); + } + } + return builder.build(); + } + + /** + * Returns a map from the setters method name to its matching getters method name based upon the + * passed in {@link PropertyDescriptor}s property descriptors. + * + * @param propertyDescriptors A list of {@link PropertyDescriptor}s to use when generating the + * map. + * @return A map of setter method name to getter method name. + */ + private static Map generateSettersToPropertyNames( + List propertyDescriptors) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (PropertyDescriptor descriptor : propertyDescriptors) { + if (descriptor.getWriteMethod() != null) { + builder.put(descriptor.getWriteMethod().getName(), descriptor.getName()); + } + } + return builder.build(); + } + + static class Serializer extends JsonSerializer { + @Override + public void serialize(PipelineOptions value, JsonGenerator jgen, SerializerProvider provider) + throws IOException, JsonProcessingException { + ProxyInvocationHandler handler = (ProxyInvocationHandler) Proxy.getInvocationHandler(value); + Map options = Maps.newHashMap(handler.jsonOptions); + options.putAll(handler.options); + removeIgnoredOptions(handler.knownInterfaces, options); + ensureSerializable(handler.knownInterfaces, options); + jgen.writeStartObject(); + jgen.writeFieldName("options"); + jgen.writeObject(options); + jgen.writeEndObject(); + } + + /** + * We remove all properties within the passed in options where there getter is annotated with + * {@link JsonIgnore @JsonIgnore} from the passed in options using the passed in interfaces. + */ + private void removeIgnoredOptions( + Set> interfaces, Map options) { + // Find all the method names which are annotated with JSON ignore. + Set jsonIgnoreMethodNames = FluentIterable.from( + PipelineOptionsFactory.getClosureOfMethodsOnInterfaces(interfaces)) + .filter(JsonIgnorePredicate.INSTANCE).transform(new Function() { + @Override + public String apply(Method input) { + return input.getName(); + } + }).toSet(); + + // Remove all options which have the same method name as the descriptor. + for (PropertyDescriptor descriptor + : PipelineOptionsFactory.getPropertyDescriptors(interfaces)) { + if (jsonIgnoreMethodNames.contains(descriptor.getReadMethod().getName())) { + options.remove(descriptor.getName()); + } + } + } + + /** + * We use an {@link ObjectMapper} to verify that the passed in options are serializable + * and deserializable. + */ + private void ensureSerializable(Set> interfaces, + Map options) throws IOException { + // Construct a map from property name to the return type of the getter. + Map propertyToReturnType = Maps.newHashMap(); + for (PropertyDescriptor descriptor + : PipelineOptionsFactory.getPropertyDescriptors(interfaces)) { + if (descriptor.getReadMethod() != null) { + propertyToReturnType.put(descriptor.getName(), + descriptor.getReadMethod().getGenericReturnType()); + } + } + + // Attempt to serialize and deserialize each property. + for (Map.Entry entry : options.entrySet()) { + String serializedValue = MAPPER.writeValueAsString(entry.getValue()); + JavaType type = MAPPER.getTypeFactory() + .constructType(propertyToReturnType.get(entry.getKey())); + MAPPER.readValue(serializedValue, type); + } + } + } + + static class Deserializer extends JsonDeserializer { + @Override + public PipelineOptions deserialize(JsonParser jp, DeserializationContext ctxt) + throws IOException, JsonProcessingException { + ObjectNode objectNode = (ObjectNode) jp.readValueAsTree(); + ObjectNode optionsNode = (ObjectNode) objectNode.get("options"); + + Map fields = Maps.newHashMap(); + for (Iterator> iterator = optionsNode.fields(); + iterator.hasNext(); ) { + Map.Entry field = iterator.next(); + fields.put(field.getKey(), field.getValue()); + } + PipelineOptions options = + new ProxyInvocationHandler(Maps.newHashMap(), fields) + .as(PipelineOptions.class); + return options; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java new file mode 100644 index 0000000000000..725d845d5b9f2 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java @@ -0,0 +1,38 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +/** + * [Whitelisting Required] Options used to configure the streaming backend. + * + *

Important: Streaming support is experimental. It is only supported in the + * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner} for users whitelisted in a + * streaming early access program. + * + *

You should expect this class to change significantly in future + * versions of the SDK or be removed entirely. + */ +public interface StreamingOptions extends + ApplicationNameOptions, GcpOptions, PipelineOptions { + /** + * Note that this feature is currently experimental and only available to users whitelisted in + * a streaming early access program. + */ + @Description("True if running in streaming mode (experimental)") + boolean isStreaming(); + void setStreaming(boolean value); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Validation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Validation.java new file mode 100644 index 0000000000000..10f205fcadb96 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Validation.java @@ -0,0 +1,39 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * {@link Validation} represents a set of annotations which can be used to annotate getter + * properties on {@link PipelineOptions} with information representing the validation criteria to + * be used when validating with the {@link PipelineOptionsValidator}. + */ + +public @interface Validation { + /** + * This criteria specifies that the value must be not null. Note that this annotation + * should only be applied to methods which return nullable objects. + */ + @Target(value = ElementType.METHOD) + @Retention(RetentionPolicy.RUNTIME) + public @interface Required { + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/package-info.java new file mode 100644 index 0000000000000..557e377676b71 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/package-info.java @@ -0,0 +1,25 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Defines {@link com.google.cloud.dataflow.sdk.options.PipelineOptions} for + * configuring pipeline execution. + * + *

{@link com.google.cloud.dataflow.sdk.options.PipelineOptions} encapsulates the various + * parameters that describe how a pipeline should be run. {@code PipelineOptions} are created + * using a {@link com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory}. + */ +package com.google.cloud.dataflow.sdk.options; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/package-info.java new file mode 100644 index 0000000000000..e27ac01476606 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/package-info.java @@ -0,0 +1,35 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Provides a simple, powerful model for building both batch and + * streaming parallel data processing + * {@link com.google.cloud.dataflow.sdk.Pipeline}s. + * + *

To use the Google Cloud Dataflow SDK, you build a + * {@link com.google.cloud.dataflow.sdk.Pipeline} which manages a graph of + * {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s + * and the {@link com.google.cloud.dataflow.sdk.values.PCollection}s that + * the PTransforms consume and produce. + * + *

Each Pipeline has a + * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} to specify + * where and how it should run after pipeline construction is complete. + * + */ +package com.google.cloud.dataflow.sdk; + + diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunner.java new file mode 100644 index 0000000000000..61fb09746921d --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunner.java @@ -0,0 +1,136 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.PipelineResult; +import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.util.MonitoringUtil; +import com.google.cloud.dataflow.sdk.util.MonitoringUtil.JobState; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.POutput; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +import javax.annotation.Nullable; + +/** + * A PipelineRunner that's like {@link DataflowPipelineRunner} + * but that waits for the launched job to finish. + * + *

Prints out job status updates and console messages while it waits. + * + *

Returns the final job state, or throws an exception if the job + * fails or cannot be monitored. + */ +public class BlockingDataflowPipelineRunner extends + PipelineRunner { + private static final Logger LOG = LoggerFactory.getLogger(BlockingDataflowPipelineRunner.class); + + /** + * Holds the status of a run request. + */ + public static class PipelineJobState implements PipelineResult { + private final JobState state; + + public PipelineJobState(JobState state) { + this.state = state; + } + + public JobState getJobState() { + return state; + } + } + + // Defaults to an infinite wait period. + // TODO: make this configurable after removal of option map. + private static final long BUILTIN_JOB_TIMEOUT_SEC = -1L; + + private DataflowPipelineRunner dataflowPipelineRunner = null; + private MonitoringUtil.JobMessagesHandler jobMessagesHandler; + + protected BlockingDataflowPipelineRunner( + DataflowPipelineRunner internalRunner, + MonitoringUtil.JobMessagesHandler jobMessagesHandler) { + this.dataflowPipelineRunner = internalRunner; + this.jobMessagesHandler = jobMessagesHandler; + } + + /** + * Constructs a runner from the provided options. + */ + public static BlockingDataflowPipelineRunner fromOptions( + PipelineOptions options) { + BlockingDataflowPipelineOptions dataflowOptions = + PipelineOptionsValidator.validate(BlockingDataflowPipelineOptions.class, options); + DataflowPipelineRunner dataflowPipelineRunner = + DataflowPipelineRunner.fromOptions(dataflowOptions); + + return new BlockingDataflowPipelineRunner(dataflowPipelineRunner, + new MonitoringUtil.PrintHandler(dataflowOptions.getJobMessageOutput())); + } + + @Override + public PipelineJobState run(Pipeline p) { + DataflowPipelineJob job = dataflowPipelineRunner.run(p); + + @Nullable JobState result; + try { + result = job.waitToFinish( + BUILTIN_JOB_TIMEOUT_SEC, TimeUnit.SECONDS, jobMessagesHandler); + } catch (IOException | InterruptedException ex) { + throw new RuntimeException("Exception caught during job execution", ex); + } + + if (result == null) { + throw new RuntimeException("No result provided: " + + "possible error requesting job status."); + } + + LOG.info("Job finished with status {}", result); + if (result.isTerminal()) { + return new PipelineJobState(result); + } + + // TODO: introduce an exception which can wrap a JobState, + // so that detailed error information can be retrieved. + throw new RuntimeException("Job failed with state " + result); + } + + @Override + public Output apply( + PTransform transform, Input input) { + return dataflowPipelineRunner.apply(transform, input); + } + + /** + * Sets callbacks to invoke during execution see {@code DataflowPipelineRunnerHooks}. + * Important: setHooks is experimental. Please consult with the Dataflow team before using it. + * You should expect this class to change significantly in future versions of the SDK or be + * removed entirely. + */ + public void setHooks(DataflowPipelineRunnerHooks hooks) { + this.dataflowPipelineRunner.setHooks(hooks); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipeline.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipeline.java new file mode 100644 index 0000000000000..310b4d97a323f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipeline.java @@ -0,0 +1,49 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; + +/** + * A DataflowPipeline, which returns a + * {@link DataflowPipelineJob} subclass of PipelineResult + * from {@link com.google.cloud.dataflow.sdk.Pipeline#run()}. + */ +public class DataflowPipeline extends Pipeline { + + /** + * Creates and returns a new DataflowPipeline instance for tests. + */ + public static DataflowPipeline create(DataflowPipelineOptions options) { + return new DataflowPipeline(options); + } + + private DataflowPipeline(DataflowPipelineOptions options) { + super(DataflowPipelineRunner.fromOptions(options), options); + } + + @Override + public DataflowPipelineJob run() { + return (DataflowPipelineJob) super.run(); + } + + @Override + public DataflowPipelineRunner getRunner() { + return (DataflowPipelineRunner) super.getRunner(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java new file mode 100644 index 0000000000000..c1facb0288b84 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java @@ -0,0 +1,169 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime; + +import com.google.api.client.googleapis.json.GoogleJsonResponseException; +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.JobMessage; +import com.google.cloud.dataflow.sdk.PipelineResult; +import com.google.cloud.dataflow.sdk.util.MonitoringUtil; +import com.google.cloud.dataflow.sdk.util.MonitoringUtil.JobState; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.SocketTimeoutException; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import javax.annotation.Nullable; + +/** + * A DataflowPipelineJob represents a job submitted to Dataflow using + * {@link DataflowPipelineRunner}. + */ +public class DataflowPipelineJob implements PipelineResult { + private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineJob.class); + + /** + * The id for the job. + */ + private String jobId; + + /** + * Google cloud project to associate this pipeline with. + */ + private String project; + + /** + * Client for the Dataflow service. This can be used to query the service + * for information about the job. + */ + private Dataflow dataflowClient; + + /** + * Construct the job. + * + * @param projectId the project id + * @param jobId the job id + * @param client the workflow client + */ + public DataflowPipelineJob( + String projectId, String jobId, Dataflow client) { + project = projectId; + this.jobId = jobId; + dataflowClient = client; + } + + public String getJobId() { + return jobId; + } + + public String getProjectId() { + return project; + } + + public Dataflow getDataflowClient() { + return dataflowClient; + } + + /** + * Wait for the job to finish and return the final status. + * + * @param timeToWait The time to wait in units timeUnit for the job to finish. + * @param timeUnit The unit of time for timeToWait. + * Provide a negative value for an infinite wait. + * @param messageHandler If non null this handler will be invoked for each + * batch of messages received. + * @return The final state of the job or null on timeout or if the + * thread is interrupted. + * @throws IOException If there is a persistent problem getting job + * information. + * @throws InterruptedException + */ + @Nullable + public JobState waitToFinish( + long timeToWait, + TimeUnit timeUnit, + MonitoringUtil.JobMessagesHandler messageHandler) + throws IOException, InterruptedException { + // The polling interval for job status information. + long interval = TimeUnit.SECONDS.toMillis(2); + + // The time at which to stop. + long endTime = timeToWait >= 0 + ? System.currentTimeMillis() + timeUnit.toMillis(timeToWait) + : Long.MAX_VALUE; + + MonitoringUtil monitor = new MonitoringUtil(project, dataflowClient); + + long lastTimestamp = 0; + int errorGettingMessages = 0; + int errorGettingJobStatus = 0; + while (true) { + if (System.currentTimeMillis() >= endTime) { + // Timed out. + return null; + } + + if (messageHandler != null) { + // Process all the job messages that have accumulated so far. + try { + List allMessages = monitor.getJobMessages( + jobId, lastTimestamp); + + if (!allMessages.isEmpty()) { + lastTimestamp = + fromCloudTime(allMessages.get(allMessages.size() - 1).getTime()).getMillis(); + messageHandler.process(allMessages); + } + } catch (GoogleJsonResponseException | SocketTimeoutException e) { + if (++errorGettingMessages > 5) { + // We want to continue to wait for the job to finish so + // we ignore this error, but warn occasionally if it keeps happening. + LOG.warn("There are problems accessing job messages: ", e); + errorGettingMessages = 0; + } + } + } + + // Check if the job is done. + try { + Job job = dataflowClient.v1b3().projects().jobs().get(project, jobId).execute(); + JobState state = JobState.toState(job.getCurrentState()); + if (state.isTerminal()) { + return state; + } + } catch (GoogleJsonResponseException | SocketTimeoutException e) { + if (++errorGettingJobStatus > 5) { + // We want to continue to wait for the job to finish so + // we ignore this error, but warn occasionally if it keeps happening. + LOG.warn("There were problems getting job status: ", e); + errorGettingJobStatus = 0; + } + } + + // Job not yet done. Wait a little, then check again. + long sleepTime = Math.min( + endTime - System.currentTimeMillis(), interval); + TimeUnit.MILLISECONDS.sleep(sleepTime); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java new file mode 100644 index 0000000000000..ed01b8345c186 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -0,0 +1,315 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import com.google.api.client.googleapis.json.GoogleJsonResponseException; +import com.google.api.client.util.Joiner; +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.api.services.dataflow.model.Job; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.GroupByKey; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo; +import com.google.cloud.dataflow.sdk.util.GcsUtil; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.MonitoringUtil; +import com.google.cloud.dataflow.sdk.util.PackageUtil; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.Transport; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.POutput; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; + +import com.fasterxml.jackson.core.JsonProcessingException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.PrintWriter; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * A {@link PipelineRunner} that executes the operations in the + * pipeline by first translating them to the Dataflow representation + * using the {@link DataflowPipelineTranslator} and then submitting + * them to a Dataflow service for execution. + */ +public class DataflowPipelineRunner extends PipelineRunner { + private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineRunner.class); + + /** Provided configuration options. */ + private final DataflowPipelineOptions options; + + /** The directory on GCS where files should be uploaded. */ + private final GcsPath gcsStaging; + + /** The directory on GCS where temporary files are stored. */ + private final GcsPath gcsTemp; + + /** Client for the Dataflow service. This is used to actually submit jobs. */ + private final Dataflow dataflowClient; + + /** Translator for this DataflowPipelineRunner, based on options. */ + private final DataflowPipelineTranslator translator; + + /** A set of user defined functions to invoke at different points in execution. */ + private DataflowPipelineRunnerHooks hooks; + + // Environment version information + private static final String ENVIRONMENT_MAJOR_VERSION = "0"; + + /** + * Construct a runner from the provided options. + * + * @param options Properties which configure the runner. + * @return The newly created runner. + */ + public static DataflowPipelineRunner fromOptions(PipelineOptions options) { + DataflowPipelineOptions dataflowOptions = + PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options); + ArrayList missing = new ArrayList<>(); + + if (dataflowOptions.getProject() == null) { + missing.add("project"); + } + if (dataflowOptions.getAppName() == null) { + missing.add("appName"); + } + if (missing.size() > 0) { + throw new IllegalArgumentException( + "Missing required values: " + Joiner.on(',').join(missing)); + } + + Preconditions.checkArgument(!(Strings.isNullOrEmpty(dataflowOptions.getTempLocation()) + && Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())), + "Missing required value: at least one of tempLocation or stagingLocation must be set."); + if (Strings.isNullOrEmpty(dataflowOptions.getTempLocation())) { + dataflowOptions.setTempLocation(dataflowOptions.getStagingLocation()); + } else if (Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())) { + dataflowOptions.setStagingLocation( + GcsPath.fromUri(dataflowOptions.getTempLocation()).resolve("staging").toString()); + } + + if (dataflowOptions.getFilesToStage() == null) { + dataflowOptions.setFilesToStage(detectClassPathResourcesToStage( + DataflowPipelineRunner.class.getClassLoader())); + LOG.info("No specified files to stage. Defaulting to files: {}", + dataflowOptions.getFilesToStage()); + } + + // Verify jobName according to service requirements. + String jobName = dataflowOptions.getJobName().toLowerCase(); + Preconditions.checkArgument( + jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"), + "JobName invalid; the name must consist of only the characters " + + "[-a-z0-9], starting with a letter and ending with a letter " + + "or number"); + Preconditions.checkArgument(jobName.length() <= 40, + "JobName too long; must be no more than 40 characters in length"); + + return new DataflowPipelineRunner(dataflowOptions); + } + + private DataflowPipelineRunner(DataflowPipelineOptions options) { + this.options = options; + this.dataflowClient = options.getDataflowClient(); + this.gcsTemp = GcsPath.fromUri(options.getTempLocation()); + this.gcsStaging = GcsPath.fromUri(options.getStagingLocation()); + this.translator = DataflowPipelineTranslator.fromOptions(options); + + // (Re-)register standard IO factories. Clobbers any prior credentials. + IOChannelUtils.registerStandardIOFactories(options); + } + + @Override + @SuppressWarnings("unchecked") + public Output apply( + PTransform transform, Input input) { + if (transform instanceof Combine.GroupedValues) { + // TODO: Redundant with translator registration? + return (Output) PCollection.createPrimitiveOutputInternal( + ((PCollection) input).getWindowingFn()); + } else if (transform instanceof GroupByKey) { + // The DataflowPipelineRunner implementation of GroupByKey will sort values by timestamp, + // so no need for an explicit sort transform. + boolean runnerSortsByTimestamp = true; + return (Output) ((GroupByKey) transform).applyHelper( + (PCollection) input, options.isStreaming(), runnerSortsByTimestamp); + } else { + return super.apply(transform, input); + } + } + + @Override + public DataflowPipelineJob run(Pipeline pipeline) { + LOG.info("Executing pipeline on the Dataflow Service, which will have billing implications " + + "related to Google Compute Engine usage and other Google Cloud Services."); + + GcsUtil gcsUtil = options.getGcsUtil(); + List packages = + PackageUtil.stageClasspathElementsToGcs(gcsUtil, options.getFilesToStage(), gcsStaging); + + Job newJob = translator.translate(pipeline, packages); + + String version = DataflowReleaseInfo.getReleaseInfo().getVersion(); + System.out.println("Dataflow SDK version: " + version); + + newJob.getEnvironment().setUserAgent(DataflowReleaseInfo.getReleaseInfo()); + // The Dataflow Service may write to the temporary directory directly, so + // must be verified. + newJob.getEnvironment().setTempStoragePrefix(verifyGcsPath(gcsTemp).toResourceName()); + newJob.getEnvironment().setDataset(options.getTempDatasetId()); + newJob.getEnvironment().setClusterManagerApiService( + options.getClusterManagerApi().getApiServiceName()); + newJob.getEnvironment().setExperiments(options.getExperiments()); + + // Requirements about the service. + Map environmentVersion = new HashMap<>(); + // TODO: Specify the environment major version. + // environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, + // ENVIRONMENT_MAJOR_VERSION); + newJob.getEnvironment().setVersion(environmentVersion); + // Default jobType is DATA_PARALLEL which is for java batch. + String jobType = "DATA_PARALLEL"; + + if (options.isStreaming()) { + jobType = "STREAMING"; + } + environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_JOB_TYPE_KEY, jobType); + + if (hooks != null) { + hooks.modifyEnvironmentBeforeSubmission(newJob.getEnvironment()); + } + + if (!Strings.isNullOrEmpty(options.getDataflowJobFile())) { + try (PrintWriter printWriter = new PrintWriter( + new File(options.getDataflowJobFile()))) { + String workSpecJson = DataflowPipelineTranslator.jobToString(newJob); + printWriter.print(workSpecJson); + LOG.info("Printed workflow specification to {}", options.getDataflowJobFile()); + } catch (JsonProcessingException ex) { + LOG.warn("Cannot translate workflow spec to json for debug."); + } catch (FileNotFoundException ex) { + LOG.warn("Cannot create workflow spec output file."); + } + } + + Job jobResult; + try { + jobResult = dataflowClient.v1b3().projects().jobs() + .create(options.getProject(), newJob) + .execute(); + } catch (GoogleJsonResponseException e) { + throw new RuntimeException( + "Failed to create a workflow job: " + + (e.getDetails() != null ? e.getDetails().getMessage() : e), e); + } catch (IOException e) { + throw new RuntimeException("Failed to create a workflow job", e); + } + + LOG.info("To access the Dataflow monitoring console, please navigate to {}", + MonitoringUtil.getJobMonitoringPageURL(options.getProject(), jobResult.getId())); + System.out.println("Submitted job: " + jobResult.getId()); + + // Use a raw client for post-launch monitoring, as status calls may fail + // regularly and need not be retried automatically. + return new DataflowPipelineJob(options.getProject(), jobResult.getId(), + Transport.newRawDataflowClient(options).build()); + } + + /** + * Returns the DataflowPipelineTranslator associated with this object. + */ + public DataflowPipelineTranslator getTranslator() { + return translator; + } + + /** + * Sets callbacks to invoke during execution see {@code DataflowPipelineRunnerHooks}. + * Important: setHooks is experimental. Please consult with the Dataflow team before using it. + * You should expect this class to change significantly in future versions of the SDK or be + * removed entirely. + */ + public void setHooks(DataflowPipelineRunnerHooks hooks) { + this.hooks = hooks; + } + + + ///////////////////////////////////////////////////////////////////////////// + + @Override + public String toString() { return "DataflowPipelineRunner#" + hashCode(); } + + /** + * Verifies that a path can be used by the Dataflow Service API. + * @return the supplied path + */ + public static GcsPath verifyGcsPath(GcsPath path) { + Preconditions.checkArgument(path.isAbsolute(), + "Must provide absolute paths for Dataflow"); + Preconditions.checkArgument(!path.getObject().contains("//"), + "Dataflow Service does not allow objects with consecutive slashes"); + return path; + } + + /** + * Attempts to detect all the resources the class loader has access to. This does not recurse + * to class loader parents stopping it from pulling in resources from the system class loader. + * + * @param classLoader The URLClassLoader to use to detect resources to stage. + * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one + * of the resources the class loader exposes is not a file resource. + * @return A list of absolute paths to the resources the class loader uses. + */ + protected static List detectClassPathResourcesToStage(ClassLoader classLoader) { + if (!(classLoader instanceof URLClassLoader)) { + String message = String.format("Unable to use ClassLoader to detect classpath elements. " + + "Current ClassLoader is %s, only URLClassLoaders are supported.", classLoader); + LOG.error(message); + throw new IllegalArgumentException(message); + } + + List files = new ArrayList<>(); + for (URL url : ((URLClassLoader) classLoader).getURLs()) { + try { + files.add(new File(url.toURI()).getAbsolutePath()); + } catch (IllegalArgumentException | URISyntaxException e) { + String message = String.format("Unable to convert url (%s) to file.", url); + LOG.error(message); + throw new IllegalArgumentException(message, e); + } + } + return files; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerHooks.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerHooks.java new file mode 100644 index 0000000000000..ba822e876e481 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerHooks.java @@ -0,0 +1,40 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import com.google.api.services.dataflow.model.Environment; + +/** + * An instance of this class can be passed to the + * DataflowPipeline runner to add user defined hooks to be + * invoked at various times during pipeline execution. + * + * Important: DataflowPipelineRunnerHooks is experimental. Please consult with + * the Dataflow team before using it. You should expect this class to change significantly + * in future versions of the SDK or be removed entirely. + * + */ +public class DataflowPipelineRunnerHooks { + /** + * Allows the user to modify the environment of their job before their job is submitted + * to the service for execution. + * + * @param environment The environment of the job. Users can make change to this instance in order + * to change the environment with which their job executes on the service. + */ + public void modifyEnvironmentBeforeSubmission(Environment environment) {} +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java new file mode 100644 index 0000000000000..6f39a2bae5b8a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -0,0 +1,963 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray; +import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray; +import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString; +import static com.google.cloud.dataflow.sdk.util.StringUtils.jsonStringToByteArray; +import static com.google.cloud.dataflow.sdk.util.Structs.addDictionary; +import static com.google.cloud.dataflow.sdk.util.Structs.addList; +import static com.google.cloud.dataflow.sdk.util.Structs.addLong; +import static com.google.cloud.dataflow.sdk.util.Structs.addObject; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.api.client.util.Preconditions; +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.api.services.dataflow.model.Disk; +import com.google.api.services.dataflow.model.Environment; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.Step; +import com.google.api.services.dataflow.model.TaskRunnerSettings; +import com.google.api.services.dataflow.model.WorkerPool; +import com.google.api.services.dataflow.model.WorkerSettings; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.io.AvroIO; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.DatastoreIO; +import com.google.cloud.dataflow.sdk.io.PubsubIO; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.runners.dataflow.AvroIOTranslator; +import com.google.cloud.dataflow.sdk.runners.dataflow.BigQueryIOTranslator; +import com.google.cloud.dataflow.sdk.runners.dataflow.DatastoreIOTranslator; +import com.google.cloud.dataflow.sdk.runners.dataflow.PubsubIOTranslator; +import com.google.cloud.dataflow.sdk.runners.dataflow.TextIOTranslator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Flatten; +import com.google.cloud.dataflow.sdk.transforms.GroupByKey.GroupByKeyOnly; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.OutputReference; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionTuple; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.POutput; +import com.google.cloud.dataflow.sdk.values.PValue; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TypedPValue; +import com.google.common.base.Strings; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import javax.annotation.Nullable; + +/** + * DataflowPipelineTranslator knows how to translate Pipeline objects + * into Dataflow API Jobs. + */ +public class DataflowPipelineTranslator { + // Must be kept in sync with their internal counterparts. + public static final String HARNESS_WORKER_POOL = "harness"; + public static final String SHUFFLE_WORKER_POOL = "shuffle"; + private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineTranslator.class); + + /** + * A map from PTransform class to the corresponding + * TransformTranslator to use to translate that transform. + * + *

A static map that contains system-wide defaults. + */ + private static Map transformTranslators = + new HashMap<>(); + + /** Provided configuration options. */ + private final DataflowPipelineOptions options; + + /** + * Constructs a translator from the provided options. + * + * @param options Properties which configure the translator. + * + * @return The newly created translator. + */ + public static DataflowPipelineTranslator fromOptions( + DataflowPipelineOptions options) { + return new DataflowPipelineTranslator(options); + } + + private DataflowPipelineTranslator(DataflowPipelineOptions options) { + this.options = options; + } + + /** + * Translates a Pipeline into a Job + */ + public Job translate(Pipeline pipeline, List packages) { + Translator translator = new Translator(pipeline); + return translator.translate(packages); + } + + public static String jobToString(Job job) + throws JsonProcessingException { + return new ObjectMapper().writerWithDefaultPrettyPrinter() + .writeValueAsString(job); + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Records that instances of the specified PTransform class + * should be translated by default by the corresponding + * TransformTranslator. + */ + public static void registerTransformTranslator( + Class transformClass, + TransformTranslator transformTranslator) { + if (transformTranslators.put(transformClass, transformTranslator) != null) { + throw new IllegalArgumentException( + "defining multiple translators for " + transformClass); + } + } + + /** + * Returns the TransformTranslator to use for instances of the + * specified PTransform class, or null if none registered. + */ + @SuppressWarnings("unchecked") + public + TransformTranslator getTransformTranslator(Class transformClass) { + return transformTranslators.get(transformClass); + } + + /** + * An translator of a PTransform. + */ + public interface TransformTranslator { + public void translate(PT transform, + TranslationContext context); + } + + /** + * The interface provided to registered callbacks for interacting + * with the DataflowPipelineRunner, including reading and writing the + * values of PCollections and side inputs ({@link PCollectionViews}). + */ + public interface TranslationContext { + /** + * Returns the configured pipeline options. + */ + DataflowPipelineOptions getPipelineOptions(); + + /** + * Adds a step to the Dataflow workflow for the given transform, with + * the given Dataflow step type. + * This step becomes "current" for the purpose of {@link #addInput} and + * {@link #addOutput}. + */ + public void addStep(PTransform transform, String type); + + /** + * Adds a pre-defined step to the Dataflow workflow. The given PTransform should be + * consistent with the Step, in terms of input, output and coder types. + * + *

This is a low-level operation, when using this method it is up to + * the caller to ensure that names do not collide. + */ + public void addStep(PTransform transform, Step step); + + /** + * Sets the encoding for the current Dataflow step. + */ + public void addEncodingInput(Coder value); + + /** + * Adds an input with the given name and value to the current + * Dataflow step. + */ + public void addInput(String name, String value); + + /** + * Adds an input with the given name and value to the current + * Dataflow step. + */ + public void addInput(String name, Long value); + + /** + * Adds an input with the given name to the previously added Dataflow + * step, coming from the specified input PValue. + */ + public void addInput(String name, PInput value); + + /** + * Adds an input with the given name and value to the current + * Dataflow step. + * + *

This applies any verification of paths required by the Dataflow + * service. + */ + public void addInput(String name, GcsPath path); + + /** + * Adds an input which is a dictionary of strings to objects. + */ + public void addInput(String name, Map elements); + + /** + * Adds an input which is a list of objects. + */ + public void addInput(String name, List> elements); + + /** + * Adds an output with the given name to the previously added + * Dataflow step, producing the specified output {@code PValue}, + * including its {@code Coder} if a {@code TypedPValue}. If the + * {@code PValue} is a {@code PCollection}, wraps its coder inside + * a {@code WindowedValueCoder}. + */ + public void addOutput(String name, PValue value); + + /** + * Adds an output with the given name to the previously added + * Dataflow step, producing the specified output {@code PValue}, + * including its {@code Coder} if a {@code TypedPValue}. If the + * {@code PValue} is a {@code PCollection}, wraps its coder inside + * a {@code ValueOnlyCoder}. + */ + public void addValueOnlyOutput(String name, PValue value); + + /** + * Adds an output with the given name to the previously added + * CollectionToSingleton Dataflow step, consuming the specified + * input {@code PValue} and producing the specified output + * {@code PValue}. This step requires special treatment for its + * output encoding. + */ + public void addCollectionToSingletonOutput(String name, + PValue inputValue, + PValue outputValue); + + /** + * Encode a PValue reference as an output reference. + */ + public OutputReference asOutputReference(PValue value); + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Translates a Pipeline into the Dataflow representation. + */ + class Translator implements PipelineVisitor, TranslationContext { + /** The Pipeline to translate. */ + private final Pipeline pipeline; + + /** The Cloud Dataflow Job representation. */ + private final Job job = new Job(); + + /** + * Translator is stateful, as addProperty calls refer to the current step. + */ + private Step currentStep; + + /** + * A Map from PTransforms to their unique Dataflow step names. + */ + private final Map stepNames = new HashMap<>(); + + /** + * A Map from PValues to their output names used by their producer + * Dataflow steps. + */ + private final Map outputNames = new HashMap<>(); + + /** + * A Map from PValues to the Coders used for them. + */ + private final Map> outputCoders = new HashMap<>(); + + /** + * Constructs a Translator that will translate the specified + * Pipeline into Dataflow objects. + */ + public Translator(Pipeline pipeline) { + this.pipeline = pipeline; + } + + /** + * Translates this Translator's pipeline onto its writer. + * @return a Job definition filled in with the type of job, the environment, + * and the job steps. + */ + public Job translate(List packages) { + job.setName(options.getJobName().toLowerCase()); + + Environment environment = new Environment(); + job.setEnvironment(environment); + + WorkerPool workerPool = new WorkerPool(); + + workerPool.setKind(HARNESS_WORKER_POOL); + + // Pass the URL and endpoint to use to the worker pool. + WorkerSettings workerSettings = new WorkerSettings(); + workerSettings.setBaseUrl(options.getApiRootUrl()); + workerSettings.setServicePath(options.getDataflowEndpoint()); + + TaskRunnerSettings taskRunnerSettings = new TaskRunnerSettings(); + taskRunnerSettings.setParallelWorkerSettings(workerSettings); + + workerPool.setTaskrunnerSettings(taskRunnerSettings); + + WorkerPool shufflePool = new WorkerPool(); + shufflePool.setKind(SHUFFLE_WORKER_POOL); + + if (options.isStreaming()) { + job.setType("JOB_TYPE_STREAMING"); + } else { + job.setType("JOB_TYPE_BATCH"); + } + + if (options.getWorkerMachineType() != null) { + workerPool.setMachineType(options.getWorkerMachineType()); + } + + workerPool.setPackages(packages); + workerPool.setNumWorkers(options.getNumWorkers()); + shufflePool.setNumWorkers(options.getNumWorkers()); + if (options.getDiskSourceImage() != null) { + workerPool.setDiskSourceImage(options.getDiskSourceImage()); + shufflePool.setDiskSourceImage(options.getDiskSourceImage()); + } + + if (options.getMachineType() != null) { + workerPool.setMachineType(options.getMachineType()); + } + if (options.isStreaming()) { + // Use separate data disk for streaming. + Disk disk = new Disk(); + disk.setSizeGb(10); + disk.setDiskType( + // TODO: Fill in the project and zone. + "compute.googleapis.com/projects//zones//diskTypes/pd-standard"); + // TODO: introduce a separate location for Windmill binary in the + // TaskRunner so it wouldn't interfere with the data disk mount point. + disk.setMountPoint("/windmill"); + workerPool.setDataDisks(Collections.singletonList(disk)); + } + if (!Strings.isNullOrEmpty(options.getZone())) { + workerPool.setZone(options.getZone()); + shufflePool.setZone(options.getZone()); + } + if (options.getDiskSizeGb() > 0) { + workerPool.setDiskSizeGb(options.getDiskSizeGb()); + shufflePool.setDiskSizeGb(options.getDiskSizeGb()); + } + + // Set up any specific shuffle pool parameters + if (options.getShuffleNumWorkers() > 0) { + shufflePool.setNumWorkers(options.getShuffleNumWorkers()); + } + if (options.getShuffleDiskSourceImage() != null) { + shufflePool.setDiskSourceImage(options.getShuffleDiskSourceImage()); + } + if (!Strings.isNullOrEmpty(options.getShuffleZone())) { + shufflePool.setZone(options.getShuffleZone()); + } + if (options.getShuffleDiskSizeGb() > 0) { + shufflePool.setDiskSizeGb(options.getShuffleDiskSizeGb()); + } + + List workerPools = new LinkedList<>(); + + workerPools.add(workerPool); + if (!options.isStreaming()) { + workerPools.add(shufflePool); + } + environment.setWorkerPools(workerPools); + + pipeline.traverseTopologically(this); + return job; + } + + @Override + public DataflowPipelineOptions getPipelineOptions() { + return options; + } + + @Override + public void enterCompositeTransform(TransformTreeNode node) { + } + + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + } + + @SuppressWarnings("unchecked") + @Override + public void visitTransform(TransformTreeNode node) { + PTransform transform = node.getTransform(); + TransformTranslator translator = + getTransformTranslator(transform.getClass()); + if (translator == null) { + throw new IllegalStateException( + "no translator registered for " + transform); + } + LOG.debug("Translating {}", transform); + translator.translate(transform, this); + } + + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + LOG.debug("Checking translation of {}", value); + if (options.isStreaming() + && value instanceof PCollectionView) { + throw new UnsupportedOperationException( + "PCollectionViews are not supported in streaming Dataflow."); + } + if (value.getProducingTransformInternal() == null) { + throw new RuntimeException( + "internal error: expecting a PValue " + + "to have a producingTransform"); + } + if (!producer.isCompositeNode()) { + // Primitive transforms are the only ones assigned step names. + asOutputReference(value); + } + } + + @Override + public void addStep(PTransform transform, String type) { + String stepName = genStepName(); + if (stepNames.put(transform, stepName) != null) { + throw new IllegalArgumentException( + transform + " already has a name specified"); + } + // Start the next "steps" list item. + List steps = job.getSteps(); + if (steps == null) { + steps = new LinkedList<>(); + job.setSteps(steps); + } + + currentStep = new Step(); + currentStep.setName(stepName); + currentStep.setKind(type); + steps.add(currentStep); + addInput(PropertyNames.USER_NAME, pipeline.getFullName(transform)); + } + + @Override + public void addStep(PTransform transform, Step original) { + Step step = original.clone(); + String stepName = step.getName(); + if (stepNames.put(transform, stepName) != null) { + throw new IllegalArgumentException(transform + " already has a name specified"); + } + + Map properties = step.getProperties(); + if (properties != null) { + @Nullable List> outputInfoList = null; + try { + // TODO: This should be done via a Structs accessor. + outputInfoList = (List>) properties.get(PropertyNames.OUTPUT_INFO); + } catch (Exception e) { + throw new RuntimeException("Inconsistent dataflow pipeline translation", e); + } + if (outputInfoList != null && outputInfoList.size() > 0) { + Map firstOutputPort = outputInfoList.get(0); + @Nullable String name; + try { + name = getString(firstOutputPort, PropertyNames.OUTPUT_NAME); + } catch (Exception e) { + name = null; + } + if (name != null) { + registerOutputName(pipeline.getOutput(transform), name); + } + } + } + + List steps = job.getSteps(); + if (steps == null) { + steps = new LinkedList<>(); + job.setSteps(steps); + } + currentStep = step; + steps.add(step); + } + + @Override + public void addEncodingInput(Coder coder) { + CloudObject encoding = SerializableUtils.ensureSerializable(coder); + addObject(getProperties(), PropertyNames.ENCODING, encoding); + } + + @Override + public void addInput(String name, String value) { + addString(getProperties(), name, value); + } + + @Override + public void addInput(String name, Long value) { + addLong(getProperties(), name, value); + } + + @Override + public void addInput(String name, Map elements) { + addDictionary(getProperties(), name, elements); + } + + @Override + public void addInput(String name, List> elements) { + addList(getProperties(), name, elements); + } + + @Override + public void addInput(String name, PInput value) { + if (value instanceof PValue) { + addInput(name, asOutputReference((PValue) value)); + } else { + throw new IllegalStateException("Input must be a PValue"); + } + } + + @Override + public void addInput(String name, GcsPath path) { + addInput(name, DataflowPipelineRunner.verifyGcsPath(path).toResourceName()); + } + + @Override + public void addOutput(String name, PValue value) { + Coder coder; + if (value instanceof TypedPValue) { + coder = ((TypedPValue) value).getCoder(); + if (value instanceof PCollection) { + // Wrap the PCollection element Coder inside a WindowedValueCoder. + coder = WindowedValue.getFullCoder( + coder, + ((PCollection) value).getWindowingFn().windowCoder()); + } + } else { + // No output coder to encode. + coder = null; + } + addOutput(name, value, coder); + } + + @Override + public void addValueOnlyOutput(String name, PValue value) { + Coder coder; + if (value instanceof TypedPValue) { + coder = ((TypedPValue) value).getCoder(); + if (value instanceof PCollection) { + // Wrap the PCollection element Coder inside a ValueOnly + // WindowedValueCoder. + coder = WindowedValue.getValueOnlyCoder(coder); + } + } else { + // No output coder to encode. + coder = null; + } + addOutput(name, value, coder); + } + + @Override + public void addCollectionToSingletonOutput(String name, + PValue inputValue, + PValue outputValue) { + Coder inputValueCoder = + Preconditions.checkNotNull(outputCoders.get(inputValue)); + // The inputValueCoder for the input PCollection should be some + // WindowedValueCoder of the input PCollection's element + // coder. + Preconditions.checkState( + inputValueCoder instanceof WindowedValue.WindowedValueCoder); + // The outputValueCoder for the output should be an + // IterableCoder of the inputValueCoder. This is a property + // of the backend "CollectionToSingleton" step. + Coder outputValueCoder = IterableCoder.of(inputValueCoder); + addOutput(name, outputValue, outputValueCoder); + } + + /** + * Adds an output with the given name to the previously added + * Dataflow step, producing the specified output {@code PValue} + * with the given {@code Coder} (if not {@code null}). + */ + @SuppressWarnings("unchecked") + private void addOutput(String name, PValue value, Coder valueCoder) { + registerOutputName(value, name); + + Map properties = getProperties(); + @Nullable List> outputInfoList = null; + try { + // TODO: This should be done via a Structs accessor. + outputInfoList = (List>) properties.get(PropertyNames.OUTPUT_INFO); + } catch (Exception e) { + throw new RuntimeException("Inconsistent dataflow pipeline translation", e); + } + if (outputInfoList == null) { + outputInfoList = new ArrayList<>(); + // TODO: This should be done via a Structs accessor. + properties.put(PropertyNames.OUTPUT_INFO, outputInfoList); + } + + Map outputInfo = new HashMap<>(); + addString(outputInfo, PropertyNames.OUTPUT_NAME, name); + addString(outputInfo, PropertyNames.USER_NAME, value.getName()); + + if (valueCoder != null) { + // Verify that encoding can be decoded, in order to catch serialization + // failures as early as possible. + CloudObject encoding = SerializableUtils.ensureSerializable(valueCoder); + addObject(outputInfo, PropertyNames.ENCODING, encoding); + outputCoders.put(value, valueCoder); + } + + outputInfoList.add(outputInfo); + } + + @Override + public OutputReference asOutputReference(PValue value) { + PTransform transform = + value.getProducingTransformInternal(); + String stepName = stepNames.get(transform); + if (stepName == null) { + throw new IllegalArgumentException(transform + " doesn't have a name specified"); + } + + String outputName = outputNames.get(value); + if (outputName == null) { + throw new IllegalArgumentException( + "output " + value + " doesn't have a name specified"); + } + + return new OutputReference(stepName, outputName); + } + + private Map getProperties() { + Map properties = currentStep.getProperties(); + if (properties == null) { + properties = new HashMap<>(); + currentStep.setProperties(properties); + } + return properties; + } + + /** + * Returns a fresh Dataflow step name. + */ + private String genStepName() { + return "s" + (stepNames.size() + 1); + } + + /** + * Records the name of the given output PValue, + * within its producing transform. + */ + private void registerOutputName(POutput value, String name) { + if (outputNames.put(value, name) != null) { + throw new IllegalArgumentException( + "output " + value + " already has a name specified"); + } + } + } + + ///////////////////////////////////////////////////////////////////////////// + + @Override + public String toString() { + return "DataflowPipelineTranslator#" + hashCode(); + } + + + /////////////////////////////////////////////////////////////////////////// + + static { + registerTransformTranslator( + View.CreatePCollectionView.class, + new TransformTranslator() { + @Override + public void translate( + View.CreatePCollectionView transform, + TranslationContext context) { + translateTyped(transform, context); + } + + private void translateTyped( + View.CreatePCollectionView transform, + TranslationContext context) { + context.addStep(transform, "CollectionToSingleton"); + context.addInput(PropertyNames.PARALLEL_INPUT, transform.getInput()); + context.addCollectionToSingletonOutput( + PropertyNames.OUTPUT, + transform.getInput(), + transform.getOutput()); + } + }); + + DataflowPipelineTranslator.registerTransformTranslator( + Combine.GroupedValues.class, + new DataflowPipelineTranslator.TransformTranslator() { + @SuppressWarnings("unchecked") + @Override + public void translate( + Combine.GroupedValues transform, + DataflowPipelineTranslator.TranslationContext context) { + translateHelper(transform, context); + } + + private void translateHelper( + final Combine.GroupedValues transform, + DataflowPipelineTranslator.TranslationContext context) { + context.addStep(transform, "CombineValues"); + context.addInput(PropertyNames.PARALLEL_INPUT, transform.getInput()); + context.addInput( + PropertyNames.SERIALIZED_FN, + byteArrayToJsonString(serializeToByteArray(transform.getFn()))); + context.addEncodingInput(transform.getAccumulatorCoder()); + context.addOutput(PropertyNames.OUTPUT, transform.getOutput()); + } + }); + + registerTransformTranslator( + Create.class, + new TransformTranslator() { + @Override + public void translate( + Create transform, + TranslationContext context) { + createHelper(transform, context); + } + + private void createHelper( + Create transform, + TranslationContext context) { + context.addStep(transform, "CreateCollection"); + + Coder coder = transform.getOutput().getCoder(); + List elements = new LinkedList<>(); + for (T elem : transform.getElements()) { + byte[] encodedBytes; + try { + encodedBytes = encodeToByteArray(coder, elem); + } catch (CoderException exn) { + // TODO: Put in better element printing: + // truncate if too long. + throw new IllegalArgumentException( + "unable to encode element " + elem + " of " + transform + + " using " + coder, + exn); + } + String encodedJson = byteArrayToJsonString(encodedBytes); + assert Arrays.equals(encodedBytes, + jsonStringToByteArray(encodedJson)); + elements.add(CloudObject.forString(encodedJson)); + } + context.addInput(PropertyNames.ELEMENT, elements); + context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); + } + }); + + registerTransformTranslator( + Flatten.FlattenPCollectionList.class, + new TransformTranslator() { + @Override + public void translate( + Flatten.FlattenPCollectionList transform, + TranslationContext context) { + flattenHelper(transform, context); + } + + private void flattenHelper( + Flatten.FlattenPCollectionList transform, + TranslationContext context) { + context.addStep(transform, "Flatten"); + + List inputs = new LinkedList<>(); + for (PCollection input : transform.getInput().getAll()) { + inputs.add(context.asOutputReference(input)); + } + context.addInput(PropertyNames.INPUTS, inputs); + context.addOutput(PropertyNames.OUTPUT, transform.getOutput()); + // TODO: Need to specify orderedness. + } + }); + + registerTransformTranslator( + GroupByKeyOnly.class, + new TransformTranslator() { + @Override + public void translate( + GroupByKeyOnly transform, + TranslationContext context) { + groupByKeyHelper(transform, context); + } + + private void groupByKeyHelper( + GroupByKeyOnly transform, + TranslationContext context) { + context.addStep(transform, "GroupByKey"); + context.addInput(PropertyNames.PARALLEL_INPUT, transform.getInput()); + context.addOutput(PropertyNames.OUTPUT, transform.getOutput()); + // TODO: sortsValues + } + }); + + registerTransformTranslator( + ParDo.BoundMulti.class, + new TransformTranslator() { + @Override + public void translate( + ParDo.BoundMulti transform, + TranslationContext context) { + translateMultiHelper(transform, context); + } + + private void translateMultiHelper( + ParDo.BoundMulti transform, + TranslationContext context) { + context.addStep(transform, "ParallelDo"); + translateInputs(transform.getInput(), transform.getSideInputs(), context); + translateFn(transform.getFn(), context); + translateOutputs(transform.getOutput(), context); + } + }); + + registerTransformTranslator( + ParDo.Bound.class, + new TransformTranslator() { + @Override + public void translate( + ParDo.Bound transform, + TranslationContext context) { + translateSingleHelper(transform, context); + } + + private void translateSingleHelper( + ParDo.Bound transform, + TranslationContext context) { + context.addStep(transform, "ParallelDo"); + translateInputs(transform.getInput(), transform.getSideInputs(), context); + translateFn(transform.getFn(), context); + context.addOutput("out", transform.getOutput()); + } + }); + + /////////////////////////////////////////////////////////////////////////// + // IO Translation. + + registerTransformTranslator( + AvroIO.Read.Bound.class, new AvroIOTranslator.ReadTranslator()); + registerTransformTranslator( + AvroIO.Write.Bound.class, new AvroIOTranslator.WriteTranslator()); + + registerTransformTranslator( + BigQueryIO.Read.Bound.class, new BigQueryIOTranslator.ReadTranslator()); + registerTransformTranslator( + BigQueryIO.Write.Bound.class, new BigQueryIOTranslator.WriteTranslator()); + + registerTransformTranslator( + DatastoreIO.Write.Bound.class, new DatastoreIOTranslator.WriteTranslator()); + + registerTransformTranslator( + PubsubIO.Read.Bound.class, new PubsubIOTranslator.ReadTranslator()); + registerTransformTranslator( + PubsubIO.Write.Bound.class, new PubsubIOTranslator.WriteTranslator()); + + registerTransformTranslator( + TextIO.Read.Bound.class, new TextIOTranslator.ReadTranslator()); + registerTransformTranslator( + TextIO.Write.Bound.class, new TextIOTranslator.WriteTranslator()); + } + + private static void translateInputs( + PCollection input, + List> sideInputs, + TranslationContext context) { + context.addInput(PropertyNames.PARALLEL_INPUT, input); + translateSideInputs(sideInputs, context); + } + + // Used for ParDo + private static void translateSideInputs( + List> sideInputs, + TranslationContext context) { + Map nonParInputs = new HashMap<>(); + + for (PCollectionView view : sideInputs) { + nonParInputs.put( + view.getTagInternal().getId(), + context.asOutputReference(view)); + } + + context.addInput(PropertyNames.NON_PARALLEL_INPUTS, nonParInputs); + } + + private static void translateFn( + Serializable fn, + TranslationContext context) { + context.addInput(PropertyNames.USER_FN, fn.getClass().getName()); + context.addInput( + PropertyNames.SERIALIZED_FN, + byteArrayToJsonString(serializeToByteArray(fn))); + if (fn instanceof DoFn.RequiresKeyedState) { + context.addInput(PropertyNames.USES_KEYED_STATE, "true"); + } + } + + private static void translateOutputs( + PCollectionTuple outputs, + TranslationContext context) { + for (Map.Entry, PCollection> entry + : outputs.getAll().entrySet()) { + TupleTag tag = entry.getKey(); + PCollection output = entry.getValue(); + context.addOutput(tag.getId(), output); + // TODO: Need to specify orderedness. + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipeline.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipeline.java new file mode 100644 index 0000000000000..e3cd18ecfda3b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipeline.java @@ -0,0 +1,50 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions; + +/** + * A DirectPipeline, which returns a + * {@link DirectPipelineRunner.EvaluationResults} subclass of PipelineResult + * from {@link com.google.cloud.dataflow.sdk.Pipeline#run()}. + */ +public class DirectPipeline extends Pipeline { + + /** + * Creates and returns a new DirectPipeline instance for tests. + */ + public static DirectPipeline createForTest() { + DirectPipelineRunner runner = DirectPipelineRunner.createForTest(); + return new DirectPipeline(runner, runner.getPipelineOptions()); + } + + private DirectPipeline(DirectPipelineRunner runner, DirectPipelineOptions options) { + super(runner, options); + } + + @Override + public DirectPipelineRunner.EvaluationResults run() { + return (DirectPipelineRunner.EvaluationResults) super.run(); + } + + @Override + public DirectPipelineRunner getRunner() { + return (DirectPipelineRunner) super.getRunner(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java new file mode 100644 index 0000000000000..a19b2055a0b99 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java @@ -0,0 +1,844 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor; +import com.google.cloud.dataflow.sdk.PipelineResult; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.ListCoder; +import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.util.TestCredential; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.POutput; +import com.google.cloud.dataflow.sdk.values.PValue; +import com.google.cloud.dataflow.sdk.values.TypedPValue; +import com.google.common.base.Function; +import com.google.common.collect.Lists; + +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; + +/** + * Executes the operations in the pipeline directly, in this process, without + * any optimization. Useful for small local execution and tests. + * + *

Throws an exception from {@link #run} if execution fails. + */ +public class DirectPipelineRunner + extends PipelineRunner { + private static final Logger LOG = LoggerFactory.getLogger(DirectPipelineRunner.class); + + /** + * A map from PTransform class to the corresponding + * TransformEvaluator to use to evaluate that transform. + * + *

A static map that contains system-wide defaults. + */ + private static Map defaultTransformEvaluators = + new HashMap<>(); + + /** + * A map from PTransform class to the corresponding + * TransformEvaluator to use to evaluate that transform. + * + *

An instance map that contains bindings for this DirectPipelineRunner. + * Bindings in this map override those in the default map. + */ + private Map localTransformEvaluators = + new HashMap<>(); + + /** + * Records that instances of the specified PTransform class + * should be evaluated by default by the corresponding + * TransformEvaluator. + */ + public static > + void registerDefaultTransformEvaluator( + Class transformClass, + TransformEvaluator transformEvaluator) { + if (defaultTransformEvaluators.put(transformClass, transformEvaluator) + != null) { + throw new IllegalArgumentException( + "defining multiple evaluators for " + transformClass); + } + } + + /** + * Records that instances of the specified PTransform class + * should be evaluated by the corresponding TransformEvaluator. + * Overrides any bindings specified by + * {@link #registerDefaultTransformEvaluator}. + */ + public > + void registerTransformEvaluator( + Class transformClass, + TransformEvaluator transformEvaluator) { + if (localTransformEvaluators.put(transformClass, transformEvaluator) + != null) { + throw new IllegalArgumentException( + "defining multiple evaluators for " + transformClass); + } + } + + /** + * Returns the TransformEvaluator to use for instances of the + * specified PTransform class, or null if none registered. + */ + @SuppressWarnings("unchecked") + public > + TransformEvaluator getTransformEvaluator(Class transformClass) { + TransformEvaluator transformEvaluator = + localTransformEvaluators.get(transformClass); + if (transformEvaluator == null) { + transformEvaluator = defaultTransformEvaluators.get(transformClass); + } + return transformEvaluator; + } + + /** + * Constructs a DirectPipelineRunner from the given options. + */ + public static DirectPipelineRunner fromOptions(PipelineOptions options) { + DirectPipelineOptions directOptions = + PipelineOptionsValidator.validate(DirectPipelineOptions.class, options); + LOG.debug("Creating DirectPipelineRunner"); + return new DirectPipelineRunner(directOptions); + } + + /** + * Constructs a runner with default properties for testing. + * + * @return The newly created runner. + */ + public static DirectPipelineRunner createForTest() { + DirectPipelineOptions options = PipelineOptionsFactory.as(DirectPipelineOptions.class); + options.setGcpCredential(new TestCredential()); + return new DirectPipelineRunner(options); + } + + /** + * Enable runtime testing to verify that all functions and {@link Coder} + * instances can be serialized. + * + *

Enabled by default. + * + *

This method modifies the {@code DirectPipelineRunner} instance and + * returns itself. + */ + public DirectPipelineRunner withSerializabilityTesting(boolean enable) { + this.testSerializability = enable; + return this; + } + + /** + * Enable runtime testing to verify that all values can be encoded. + * + *

Enabled by default. + * + *

This method modifies the {@code DirectPipelineRunner} instance and + * returns itself. + */ + public DirectPipelineRunner withEncodabilityTesting(boolean enable) { + this.testEncodability = enable; + return this; + } + + /** + * Enable runtime testing to verify that functions do not depend on order + * of the elements. + * + *

This is accomplished by randomizing the order of elements. + * + *

Enabled by default. + * + *

This method modifies the {@code DirectPipelineRunner} instance and + * returns itself. + */ + public DirectPipelineRunner withUnorderednessTesting(boolean enable) { + this.testUnorderedness = enable; + return this; + } + + @Override + @SuppressWarnings("unchecked") + public Output apply( + PTransform transform, Input input) { + if (transform instanceof Combine.GroupedValues) { + return (Output) applyTestCombine((Combine.GroupedValues) transform, (PCollection) input); + } else { + return super.apply(transform, input); + } + } + + private PCollection> applyTestCombine( + Combine.GroupedValues transform, + PCollection>> input) { + return input.apply(ParDo.of(TestCombineDoFn.create(transform, testSerializability))) + .setCoder(transform.getDefaultOutputCoder()); + } + + /** + * The implementation may split the KeyedCombineFn into ADD, MERGE + * and EXTRACT phases (see CombineValuesFn). In order to emulate + * this for the DirectPipelineRunner and provide an experience + * closer to the service, go through heavy seralizability checks for + * the equivalent of the results of the ADD phase, but after the + * GroupByKey shuffle, and the MERGE phase. Doing these checks + * ensure that not only is the accumulator coder serializable, but + * the accumulator coder can actually serialize the data in + * question. + */ + // @VisibleForTesting + public static class TestCombineDoFn + extends DoFn>, KV> { + private final KeyedCombineFn fn; + private final Coder accumCoder; + private final boolean testSerializability; + + @SuppressWarnings({"unchecked", "rawtypes"}) + public static TestCombineDoFn create( + Combine.GroupedValues transform, + boolean testSerializability) { + return new TestCombineDoFn( + transform.getFn(), transform.getAccumulatorCoder(), testSerializability); + } + + public TestCombineDoFn( + KeyedCombineFn fn, + Coder accumCoder, + boolean testSerializability) { + this.fn = fn; + this.accumCoder = accumCoder; + this.testSerializability = testSerializability; + } + + @Override + public void processElement(ProcessContext c) throws Exception { + K key = c.element().getKey(); + Iterable values = c.element().getValue(); + List groupedPostShuffle = + ensureSerializableByCoder(ListCoder.of(accumCoder), + addInputsRandomly(fn, key, values, new Random()), + "After addInputs of KeyedCombineFn " + fn.toString()); + VA merged = + ensureSerializableByCoder(accumCoder, + fn.mergeAccumulators(key, groupedPostShuffle), + "After mergeAccumulators of KeyedCombineFn " + fn.toString()); + // Note: The serializability of KV is ensured by the + // runner itself, since it's a transform output. + c.output(KV.of(key, fn.extractOutput(key, merged))); + } + + // Create a random list of accumulators from the given list of values + // @VisibleForTesting + public static List addInputsRandomly( + KeyedCombineFn fn, + K key, + Iterable values, + Random random) { + List out = new ArrayList(); + int i = 0; + VA accumulator = fn.createAccumulator(key); + boolean hasInput = false; + + for (VI value : values) { + fn.addInput(key, accumulator, value); + hasInput = true; + + // For each index i, flip a 1/2^i weighted coin for whether to + // create a new accumulator after index i is added, i.e. [0] + // is guaranteed, [1] is an even 1/2, [2] is 1/4, etc. The + // goal is to partition the inputs into accumulators, and make + // the accumulators potentially lumpy. + if (i == 0 || random.nextInt(1 << Math.min(i, 30)) == 0) { + out.add(accumulator); + accumulator = fn.createAccumulator(key); + hasInput = false; + } + i++; + } + if (hasInput) { + out.add(accumulator); + } + + Collections.shuffle(out, random); + return out; + } + + public T ensureSerializableByCoder( + Coder coder, T value, String errorContext) { + if (testSerializability) { + return SerializableUtils.ensureSerializableByCoder( + coder, value, errorContext); + } + return value; + } + } + + @Override + public EvaluationResults run(Pipeline pipeline) { + Evaluator evaluator = new Evaluator(); + evaluator.run(pipeline); + + // Log all counter values for debugging purposes. + for (Counter counter : evaluator.getCounters()) { + LOG.debug("Final aggregator value: {}", counter); + } + + return evaluator; + } + + /** + * An evaluator of a PTransform. + */ + public interface TransformEvaluator { + public void evaluate(PT transform, + EvaluationContext context); + } + + /** + * The interface provided to registered callbacks for interacting + * with the {@code DirectPipelineRunner}, including reading and writing the + * values of {@link PCollection}s and {@link PCollectionView}s. + */ + public interface EvaluationResults extends PipelineResult { + /** + * Retrieves the value of the given PCollection. + * Throws an exception if the PCollection's value hasn't already been set. + */ + List getPCollection(PCollection pc); + + /** + * Retrieves the windowed value of the given PCollection. + * Throws an exception if the PCollection's value hasn't already been set. + */ + List> getPCollectionWindowedValues(PCollection pc); + + /** + * Retrieves the values of each PCollection in the given + * PCollectionList. Throws an exception if the PCollectionList's + * value hasn't already been set. + */ + List> getPCollectionList(PCollectionList pcs); + + /** + * Retrieves the values indicated by the given {@link PCollectionView}. + * Note that within the {@link DoFnContext} a {@link PCollectionView} + * converts from this representation to a suitable side input value. + */ + Iterable> getPCollectionView(PCollectionView view); + } + + /** + * An immutable (value, timestamp) pair, along with other metadata necessary + * for the implementation of {@code DirectPipelineRunner}. + */ + public static class ValueWithMetadata { + /** + * Returns a new {@code ValueWithMetadata} with the {@code WindowedValue}. + * Key is null. + */ + public static ValueWithMetadata of(WindowedValue windowedValue) { + return new ValueWithMetadata<>(windowedValue, null); + } + + /** + * Returns a new {@code ValueWithMetadata} with the implicit key associated + * with this value set. The key is the last key grouped by in the chain of + * productions that produced this element. + * These keys are used internally by {@link DirectPipelineRunner} for keeping + * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.KeyedState} separate + * across keys. + */ + public ValueWithMetadata withKey(Object key) { + return new ValueWithMetadata<>(windowedValue, key); + } + + /** + * Returns a new {@code ValueWithMetadata} that is a copy of this one, but with + * a different value. + */ + public ValueWithMetadata withValue(T value) { + return new ValueWithMetadata(windowedValue.withValue(value), getKey()); + } + + /** + * Returns the {@code WindowedValue} associated with this element. + */ + public WindowedValue getWindowedValue() { + return windowedValue; + } + + /** + * Returns the value associated with this element. + * + * @see #withValue + */ + public V getValue() { + return windowedValue.getValue(); + } + + /** + * Returns the timestamp associated with this element. + */ + public Instant getTimestamp() { + return windowedValue.getTimestamp(); + } + + /** + * Returns the collection of windows this element has been placed into. May + * be null if the {@code PCollection} this element is in has not yet been + * windowed. + * + * @see #getWindows() + */ + public Collection getWindows() { + return windowedValue.getWindows(); + } + + + /** + * Returns the key associated with this element. May be null if the + * {@code PCollection} this element is in is not keyed. + * + * @see #withKey + */ + public Object getKey() { + return key; + } + + //////////////////////////////////////////////////////////////////////////// + + private final Object key; + private final WindowedValue windowedValue; + + private ValueWithMetadata(WindowedValue windowedValue, + Object key) { + this.windowedValue = windowedValue; + this.key = key; + } + } + + /** + * The interface provided to registered callbacks for interacting + * with the {@code DirectPipelineRunner}, including reading and writing the + * values of {@link PCollection}s and {@link PCollectionView}s. + */ + public interface EvaluationContext extends EvaluationResults { + /** + * Returns the configured pipeline options. + */ + DirectPipelineOptions getPipelineOptions(); + + /** + * Sets the value of the given PCollection, where each element also has a timestamp + * and collection of windows. + * Throws an exception if the PCollection's value has already been set. + */ + void setPCollectionValuesWithMetadata( + PCollection pc, List> elements); + + /** + * Shorthand for setting the value of a PCollection where the elements do not have + * timestamps or windows. + * Throws an exception if the PCollection's value has already been set. + */ + void setPCollection(PCollection pc, List elements); + + /** + * Retrieves the value of the given PCollection, along with element metadata + * such as timestamps and windows. + * Throws an exception if the PCollection's value hasn't already been set. + */ + List> getPCollectionValuesWithMetadata(PCollection pc); + + /** + * Sets the value associated with the given {@link PCollectionView}. + * Throws an exception if the {@link PCollectionView}'s value has already been set. + */ + void setPCollectionView( + PCollectionView pc, + Iterable> value); + + /** + * Ensures that the element is encodable and decodable using the + * TypePValue's coder, by encoding it and decoding it, and + * returning the result. + */ + T ensureElementEncodable(TypedPValue pvalue, T element); + + /** + * If the evaluation context is testing unorderedness and + * !isOrdered, randomly permutes the order of the elements, in a + * copy if !inPlaceAllowed, and returns the permuted list, + * otherwise returns the argument unchanged. + */ + List randomizeIfUnordered(boolean isOrdered, + List elements, + boolean inPlaceAllowed); + + /** + * If the evaluation context is testing serializability, ensures + * that the argument function is serializable and deserializable + * by encoding it and then decoding it, and returning the result. + * Otherwise returns the argument unchanged. + */ + Fn ensureSerializable(Fn fn); + + /** + * If the evaluation context is testing serializability, ensures + * that the argument Coder is serializable and deserializable + * by encoding it and then decoding it, and returning the result. + * Otherwise returns the argument unchanged. + */ + Coder ensureCoderSerializable(Coder coder); + + /** + * If the evaluation context is testing serializability, ensures + * that the given data is serializable and deserializable with the + * given Coder by encoding it and then decoding it, and returning + * the result. Otherwise returns the argument unchanged. + * + *

Error context is prefixed to any thrown exceptions. + */ + T ensureSerializableByCoder(Coder coder, + T data, String errorContext); + + /** + * Returns a mutator, which can be used to add additional counters to + * this EvaluationContext. + */ + CounterSet.AddCounterMutator getAddCounterMutator(); + + /** + * Gets the step name for this transform. + */ + public String getStepName(PTransform transform); + } + + + ///////////////////////////////////////////////////////////////////////////// + + class Evaluator implements PipelineVisitor, EvaluationContext { + private final Map stepNames = new HashMap<>(); + private final Map store = new HashMap<>(); + private final CounterSet counters = new CounterSet(); + + // Use a random number generator with a fixed seed, so execution + // using this evaluator is deterministic. (If the user-defined + // functions, transforms, and coders are deterministic.) + Random rand = new Random(0); + + public Evaluator() {} + + public void run(Pipeline pipeline) { + pipeline.traverseTopologically(this); + } + + @Override + public DirectPipelineOptions getPipelineOptions() { + return options; + } + + @Override + public void enterCompositeTransform(TransformTreeNode node) { + } + + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + } + + @SuppressWarnings("unchecked") + @Override + public void visitTransform(TransformTreeNode node) { + PTransform transform = node.getTransform(); + TransformEvaluator evaluator = + getTransformEvaluator(transform.getClass()); + if (evaluator == null) { + throw new IllegalStateException( + "no evaluator registered for " + transform); + } + LOG.debug("Evaluating {}", transform); + evaluator.evaluate(transform, this); + } + + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + LOG.debug("Checking evaluation of {}", value); + if (value.getProducingTransformInternal() == null) { + throw new RuntimeException( + "internal error: expecting a PValue " + + "to have a producingTransform"); + } + if (!producer.isCompositeNode()) { + // Verify that primitive transform outputs are already computed. + getPValue(value); + } + } + + /** + * Sets the value of the given PValue. + * Throws an exception if the PValue's value has already been set. + */ + void setPValue(PValue pvalue, Object contents) { + if (store.containsKey(pvalue)) { + throw new IllegalStateException( + "internal error: setting the value of " + pvalue + + " more than once"); + } + store.put(pvalue, contents); + } + + /** + * Retrieves the value of the given PValue. + * Throws an exception if the PValue's value hasn't already been set. + */ + Object getPValue(PValue pvalue) { + if (!store.containsKey(pvalue)) { + throw new IllegalStateException( + "internal error: getting the value of " + pvalue + + " before it has been computed"); + } + return store.get(pvalue); + } + + /** + * Convert a list of T to a list of {@code ValueWithMetadata}, with a timestamp of 0 + * and null windows. + */ + List> toValuesWithMetadata(List values) { + List> result = new ArrayList<>(values.size()); + for (T value : values) { + result.add(ValueWithMetadata.of(WindowedValue.valueInGlobalWindow(value))); + } + return result; + } + + @Override + public void setPCollection(PCollection pc, List elements) { + setPCollectionValuesWithMetadata(pc, toValuesWithMetadata(elements)); + } + + @Override + public void setPCollectionValuesWithMetadata( + PCollection pc, List> elements) { + LOG.debug("Setting {} = {}", pc, elements); + setPValue(pc, ensurePCollectionEncodable(pc, elements)); + } + + @Override + public void setPCollectionView( + PCollectionView view, + Iterable> value) { + LOG.debug("Setting {} = {}", view, value); + setPValue(view, value); + } + + /** + * Retrieves the value of the given PCollection. + * Throws an exception if the PCollection's value hasn't already been set. + */ + @Override + public List getPCollection(PCollection pc) { + List result = new ArrayList<>(); + for (ValueWithMetadata elem : getPCollectionValuesWithMetadata(pc)) { + result.add(elem.getValue()); + } + return result; + } + + @Override + public List> getPCollectionWindowedValues(PCollection pc) { + return Lists.transform( + getPCollectionValuesWithMetadata(pc), + new Function, WindowedValue>() { + @Override + public WindowedValue apply(ValueWithMetadata input) { + return input.getWindowedValue(); + }}); + } + + @Override + public List> getPCollectionValuesWithMetadata(PCollection pc) { + @SuppressWarnings("unchecked") + List> elements = (List>) getPValue(pc); + elements = randomizeIfUnordered( + pc.isOrdered(), elements, false /* not inPlaceAllowed */); + LOG.debug("Getting {} = {}", pc, elements); + return elements; + } + + @Override + public List> getPCollectionList(PCollectionList pcs) { + List> elementsList = new ArrayList<>(); + for (PCollection pc : pcs.getAll()) { + elementsList.add(getPCollection(pc)); + } + return elementsList; + } + + /** + * Retrieves the value indicated by the given {@link PCollectionView}. + * Note that within the {@link DoFnContext} a {@link PCollectionView} + * converts from this representation to a suitable side input value. + */ + @Override + public Iterable> getPCollectionView(PCollectionView view) { + @SuppressWarnings("unchecked") + Iterable> value = (Iterable>) getPValue(view); + LOG.debug("Getting {} = {}", view, value); + return value; + } + + /** + * If testEncodability, ensures that the PCollection's coder and elements + * are encodable and decodable by encoding them and decoding them, + * and returning the result. Otherwise returns the argument elements. + */ + List> ensurePCollectionEncodable( + PCollection pc, List> elements) { + ensureCoderSerializable(pc.getCoder()); + if (!testEncodability) { + return elements; + } + List> elementsCopy = new ArrayList<>(elements.size()); + for (ValueWithMetadata element : elements) { + elementsCopy.add( + element.withValue(ensureElementEncodable(pc, element.getValue()))); + } + return elementsCopy; + } + + @Override + public T ensureElementEncodable(TypedPValue pvalue, T element) { + return ensureSerializableByCoder( + pvalue.getCoder(), element, "Within " + pvalue.toString()); + } + + @Override + public List randomizeIfUnordered(boolean isOrdered, + List elements, + boolean inPlaceAllowed) { + if (!testUnorderedness || isOrdered) { + return elements; + } + List elementsCopy = new ArrayList<>(elements); + Collections.shuffle(elementsCopy, rand); + return elementsCopy; + } + + @Override + public Fn ensureSerializable(Fn fn) { + if (!testSerializability) { + return fn; + } + return SerializableUtils.ensureSerializable(fn); + } + + @Override + public Coder ensureCoderSerializable(Coder coder) { + if (testSerializability) { + SerializableUtils.ensureSerializable(coder); + } + return coder; + } + + @Override + public T ensureSerializableByCoder( + Coder coder, T value, String errorContext) { + if (testSerializability) { + return SerializableUtils.ensureSerializableByCoder( + coder, value, errorContext); + } + return value; + } + + @Override + public CounterSet.AddCounterMutator getAddCounterMutator() { + return counters.getAddCounterMutator(); + } + + @Override + public String getStepName(PTransform transform) { + String stepName = stepNames.get(transform); + if (stepName == null) { + stepName = "s" + (stepNames.size() + 1); + stepNames.put(transform, stepName); + } + return stepName; + } + + /** + * Returns the CounterSet generated during evaluation, which includes + * user-defined Aggregators and may include system-defined counters. + */ + public CounterSet getCounters() { + return counters; + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + private final DirectPipelineOptions options; + private boolean testSerializability = true; + private boolean testEncodability = true; + private boolean testUnorderedness = true; + + /** Returns a new DirectPipelineRunner. */ + private DirectPipelineRunner(DirectPipelineOptions options) { + this.options = options; + // (Re-)register standard IO factories. Clobbers any prior credentials. + IOChannelUtils.registerStandardIOFactories(options); + } + + public DirectPipelineOptions getPipelineOptions() { + return options; + } + + @Override + public String toString() { return "DirectPipelineRunner#" + hashCode(); } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java new file mode 100644 index 0000000000000..8b134e98601ce --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java @@ -0,0 +1,76 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.PipelineResult; +import com.google.cloud.dataflow.sdk.options.GcsOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.InstanceBuilder; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.POutput; +import com.google.common.base.Preconditions; + +/** + * A PipelineRunner can execute, translate, or otherwise process a + * Pipeline. + * + * @param the type of the result of {@link #run}. + */ +public abstract class PipelineRunner { + + /** + * Constructs a runner from the provided options. + * + * @return The newly created runner. + */ + public static PipelineRunner fromOptions(PipelineOptions options) { + GcsOptions gcsOptions = PipelineOptionsValidator.validate(GcsOptions.class, options); + Preconditions.checkNotNull(options); + + // (Re-)register standard IO factories. Clobbers any prior credentials. + IOChannelUtils.registerStandardIOFactories(gcsOptions); + + @SuppressWarnings("unchecked") + PipelineRunner result = + InstanceBuilder.ofType(PipelineRunner.class) + .fromClass(options.getRunner()) + .fromFactoryMethod("fromOptions") + .withArg(PipelineOptions.class, options) + .build(); + return result; + } + + /** + * Processes the given Pipeline, returning the results. + */ + public abstract Results run(Pipeline pipeline); + + /** + * Applies a transform to the given input, returning the output. + * + *

The default implementation calls PTransform.apply(input), but can be overridden + * to customize behavior for a particular runner. + */ + public Output apply( + PTransform transform, Input input) { + return transform.apply(input); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/RecordingPipelineVisitor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/RecordingPipelineVisitor.java new file mode 100644 index 0000000000000..cb1850d654bf4 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/RecordingPipelineVisitor.java @@ -0,0 +1,53 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.values.PValue; + +import java.util.ArrayList; +import java.util.List; + +/** + * Provides a simple PipelineVisitor which records the transformation tree. + * + *

Provided for internal unit tests. + */ +public class RecordingPipelineVisitor implements Pipeline.PipelineVisitor { + + public final List> transforms = new ArrayList<>(); + public final List values = new ArrayList<>(); + + @Override + public void enterCompositeTransform(TransformTreeNode node) { + } + + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + } + + @Override + public void visitTransform(TransformTreeNode node) { + transforms.add(node.getTransform()); + } + + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + values.add(value); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformHierarchy.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformHierarchy.java new file mode 100644 index 0000000000000..53a90b2b80121 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformHierarchy.java @@ -0,0 +1,111 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.POutput; +import com.google.cloud.dataflow.sdk.values.PValue; +import com.google.common.base.Preconditions; + +import java.util.Deque; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.Map; +import java.util.Set; + +/** + * Captures information about a collection of transformations and their + * associated PValues. + */ +public class TransformHierarchy { + private final Deque transformStack = new LinkedList<>(); + private final Map producingTransformNode = new HashMap<>(); + private final Map, TransformTreeNode> transformToNode = new HashMap<>(); + + public TransformHierarchy() { + // First element in the stack is the root node, holding all child nodes. + transformStack.add(new TransformTreeNode(null, null, "", null)); + } + + /** + * Returns the last TransformTreeNode on the stack. + */ + public TransformTreeNode getCurrent() { + return transformStack.peek(); + } + + /** + * Add a TransformTreeNode to the stack. + */ + public void pushNode(TransformTreeNode current) { + transformStack.push(current); + transformToNode.put(current.getTransform(), current); + } + + /** + * Removes the last TransformTreeNode from the stack. + */ + public void popNode() { + transformStack.pop(); + Preconditions.checkState(!transformStack.isEmpty()); + } + + /** + * Adds an input to the given node. + * + *

This forces the producing node to be finished. + */ + public void addInput(TransformTreeNode node, PInput input) { + for (PValue i : input.expand()) { + TransformTreeNode producer = producingTransformNode.get(i); + if (producer == null) { + throw new IllegalStateException("Producer unknown for input: " + i); + } + + producer.finishSpecifying(); + node.addInputProducer(i, producer); + } + } + + /** + * Sets the output of a transform node. + */ + public void setOutput(TransformTreeNode producer, POutput output) { + producer.setOutput(output); + + for (PValue o : output.expand()) { + producingTransformNode.put(o, producer); + } + } + + /** + * Returns the TransformTreeNode associated with a given transform. + */ + public TransformTreeNode getNode(PTransform transform) { + return transformToNode.get(transform); + } + + /** + * Visits all nodes in the transform hierarchy, in transitive order. + */ + public void visit(Pipeline.PipelineVisitor visitor, + Set visitedNodes) { + transformStack.peekFirst().visit(visitor, visitedNodes); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformTreeNode.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformTreeNode.java new file mode 100644 index 0000000000000..efd28b354f073 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/TransformTreeNode.java @@ -0,0 +1,237 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.POutput; +import com.google.cloud.dataflow.sdk.values.PValue; +import com.google.common.base.Preconditions; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import javax.annotation.Nullable; + +/** + * Provides internal tracking of transform relationships with helper methods + * for initialization and ordered visitation. + */ +public class TransformTreeNode { + private final TransformTreeNode enclosingNode; + + // The transform. If composite.isEmpty(), then this is a + // PrimitivePTransform, otherwise a composite PTransform. + private final PTransform transform; + + private final String fullName; + + // Nodes of a composite transform. + private final Collection parts = new ArrayList<>(); + + // Inputs to the transform, in expanded form and mapped to the producer + // of the input. + private final Map inputs = new HashMap<>(); + + // Input to the transform, in unexpanded form. + private final PInput input; + + // TODO: track which outputs need to be exported to parent. + // Output of the transform, in unexpanded form. + private POutput output; + + private boolean finishedSpecifying = false; + + /** + * Creates a new TransformTreeNode with the given parent and transform. + * + *

EnclosingNode and transform may both be null for a root-level node + * which holds all other nodes. + * + * @param enclosingNode the composite node containing this node + * @param transform the PTransform tracked by this node + * @param fullName the fully qualified name of the transform + * @param input the unexpanded input to the transform + */ + public TransformTreeNode(@Nullable TransformTreeNode enclosingNode, + @Nullable PTransform transform, + String fullName, + @Nullable PInput input) { + this.enclosingNode = enclosingNode; + this.transform = transform; + Preconditions.checkArgument((enclosingNode == null && transform == null) + || (enclosingNode != null && transform != null), + "EnclosingNode and transform must both be specified, or both be null"); + this.fullName = fullName; + this.input = input; + } + + /** + * Returns the transform associated with this transform node. + */ + public PTransform getTransform() { + return transform; + } + + /** + * Returns the enclosing composite transform node, or null if there is none. + */ + public TransformTreeNode getEnclosingNode() { + return enclosingNode; + } + + /** + * Adds a composite operation to the transform node. + * + *

As soon as a node is added, the transform node is considered a + * composite operation instead of a primitive transform. + */ + public void addComposite(TransformTreeNode node) { + parts.add(node); + } + + /** + * Returns true if this node represents a composite transform. + */ + public boolean isCompositeNode() { + return !parts.isEmpty(); + } + + public String getFullName() { + return fullName; + } + + /** + * Adds an input to the transform node. + */ + public void addInputProducer(PValue expandedInput, TransformTreeNode producer) { + Preconditions.checkState(!finishedSpecifying); + inputs.put(expandedInput, producer); + } + + /** + * Returns the transform input, in unexpanded form. + */ + public PInput getInput() { + return input; + } + + /** + * Returns a mapping of inputs to the producing nodes for all inputs to + * the transform. + */ + public Map getInputs() { + return Collections.unmodifiableMap(inputs); + } + + /** + * Adds an output to the transform node. + */ + public void setOutput(POutput output) { + Preconditions.checkState(!finishedSpecifying); + Preconditions.checkState(this.output == null); + this.output = output; + } + + /** + * Returns the transform output, in unexpanded form. + */ + public POutput getOutput() { + return output; + } + + /** + * Returns the transform outputs, in expanded form. + */ + public Collection getExpandedOutputs() { + if (output != null) { + return output.expand(); + } else { + return Collections.emptyList(); + } + } + + /** + * Visit the transform node. + * + *

Provides an ordered visit of the input values, the primitive + * transform (or child nodes for composite transforms), then the + * output values. + */ + public void visit(Pipeline.PipelineVisitor visitor, + Set visitedValues) { + if (!finishedSpecifying) { + finishSpecifying(); + } + + // Visit inputs. + for (Map.Entry entry : inputs.entrySet()) { + if (visitedValues.add(entry.getKey())) { + visitor.visitValue(entry.getKey(), entry.getValue()); + } + } + + if (isCompositeNode()) { + visitor.enterCompositeTransform(this); + for (TransformTreeNode child : parts) { + child.visit(visitor, visitedValues); + } + visitor.leaveCompositeTransform(this); + } else { + visitor.visitTransform(this); + } + + // Visit outputs. + for (PValue pValue : getExpandedOutputs()) { + if (visitedValues.add(pValue)) { + visitor.visitValue(pValue, this); + } + } + } + + /** + * Finish specifying a transform. + * + *

All inputs are finished first, then the transform, then + * all outputs. + */ + public void finishSpecifying() { + if (finishedSpecifying) { + return; + } + finishedSpecifying = true; + + for (TransformTreeNode input : inputs.values()) { + if (input != null) { + input.finishSpecifying(); + } + } + + if (transform != null) { + transform.finishSpecifying(); + } + + if (output != null) { + output.finishSpecifyingOutput(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java new file mode 100644 index 0000000000000..d7e36c54fc050 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java @@ -0,0 +1,113 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.dataflow; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.io.AvroIO; +import com.google.cloud.dataflow.sdk.io.ShardNameTemplate; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; + +/** + * Avro transform support code for the Dataflow backend. + */ +public class AvroIOTranslator { + + /** + * Implements AvroIO Read translation for the Dataflow backend. + */ + public static class ReadTranslator implements TransformTranslator { + + @Override + public void translate( + AvroIO.Read.Bound transform, + TranslationContext context) { + translateReadHelper(transform, context); + } + + private void translateReadHelper( + AvroIO.Read.Bound transform, + TranslationContext context) { + if (context.getPipelineOptions().isStreaming()) { + throw new IllegalArgumentException("AvroIO not supported in streaming mode."); + } + + // Only GCS paths are permitted for filepatterns in the DataflowPipelineRunner. + GcsPath gcsPath = GcsPath.fromUri(transform.getFilepattern()); + context.addStep(transform, "ParallelRead"); + context.addInput(PropertyNames.FORMAT, "avro"); + context.addInput(PropertyNames.FILEPATTERN, gcsPath); + context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); + // TODO: Orderedness? + } + } + + /** + * Implements AvroIO Write translation for the Dataflow backend. + */ + public static class WriteTranslator implements TransformTranslator { + + @Override + public void translate( + AvroIO.Write.Bound transform, + TranslationContext context) { + translateWriteHelper(transform, context); + } + + private void translateWriteHelper( + AvroIO.Write.Bound transform, + TranslationContext context) { + // Only GCS paths are permitted for filepatterns in the DataflowPipelineRunner. + GcsPath gcsPath = GcsPath.fromUri(transform.getFilenamePrefix()); + context.addStep(transform, "ParallelWrite"); + context.addInput(PropertyNames.PARALLEL_INPUT, transform.getInput()); + + // TODO: drop this check when server supports alternative templates. + switch (transform.getShardTemplate()) { + case ShardNameTemplate.INDEX_OF_MAX: + break; // supported by server + case "": + // Empty shard template allowed - forces single output. + Preconditions.checkArgument(transform.getNumShards() <= 1, + "Num shards must be <= 1 when using an empty sharding template"); + break; + default: + throw new UnsupportedOperationException("Shard template " + + transform.getShardTemplate() + + " not yet supported by Dataflow service"); + } + + context.addInput(PropertyNames.FORMAT, "avro"); + context.addInput(PropertyNames.FILENAME_PREFIX, gcsPath); + context.addInput(PropertyNames.SHARD_NAME_TEMPLATE, transform.getShardTemplate()); + context.addInput(PropertyNames.FILENAME_SUFFIX, transform.getFilenameSuffix()); + + long numShards = transform.getNumShards(); + if (numShards > 0) { + context.addInput(PropertyNames.NUM_SHARDS, numShards); + } + + context.addEncodingInput( + WindowedValue.getValueOnlyCoder( + AvroCoder.of(transform.getType(), transform.getSchema()))); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BigQueryIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BigQueryIOTranslator.java new file mode 100644 index 0000000000000..fd2731949c414 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BigQueryIOTranslator.java @@ -0,0 +1,200 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.dataflow; + +import com.google.api.client.json.JsonFactory; +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.TableReference; +import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.options.BigQueryOptions; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator; +import com.google.cloud.dataflow.sdk.util.ApiErrorExtractor; +import com.google.cloud.dataflow.sdk.util.BigQueryTableInserter; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.Transport; +import com.google.cloud.dataflow.sdk.util.WindowedValue; + +import java.io.IOException; + +/** + * BigQuery transform support code for the Dataflow backend. + */ +public class BigQueryIOTranslator { + private static final JsonFactory JSON_FACTORY = Transport.getJsonFactory(); + + /** + * Implements BigQueryIO Read translation for the Dataflow backend. + */ + public static class ReadTranslator + implements DataflowPipelineTranslator.TransformTranslator { + + @Override + public void translate(BigQueryIO.Read.Bound transform, + DataflowPipelineTranslator.TranslationContext context) { + TableReference table = transform.getTable(); + if (table.getProjectId() == null) { + table.setProjectId(context.getPipelineOptions().getProject()); + } + + // Check for source table presence for early failure notification. + // Note that a presence check can fail if the table or dataset are created by earlier stages + // of the pipeline. For these cases the withoutValidation method can be used to disable + // the check. + if (transform.getValidate()) { + verifyDatasetPresence(context.getPipelineOptions(), table); + verifyTablePresence(context.getPipelineOptions(), table); + } + + // Actual translation. + context.addStep(transform, "ParallelRead"); + context.addInput(PropertyNames.FORMAT, "bigquery"); + context.addInput(PropertyNames.BIGQUERY_TABLE, table.getTableId()); + context.addInput(PropertyNames.BIGQUERY_DATASET, table.getDatasetId()); + if (table.getProjectId() != null) { + context.addInput(PropertyNames.BIGQUERY_PROJECT, table.getProjectId()); + } + context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); + } + } + + /** + * Implements BigQueryIO Write translation for the Dataflow backend. + */ + public static class WriteTranslator + implements DataflowPipelineTranslator.TransformTranslator { + + @Override + public void translate(BigQueryIO.Write.Bound transform, + DataflowPipelineTranslator.TranslationContext context) { + if (context.getPipelineOptions().isStreaming()) { + // Streaming is handled by the streaming runner. + throw new AssertionError( + "BigQueryIO is specified to use streaming write in batch mode."); + } + + TableReference table = transform.getTable(); + if (table.getProjectId() == null) { + table.setProjectId(context.getPipelineOptions().getProject()); + } + + // Check for destination table presence and emptiness for early failure notification. + // Note that a presence check can fail if the table or dataset are created by earlier stages + // of the pipeline. For these cases the withoutValidation method can be used to disable + // the check. + if (transform.getValidate()) { + verifyDatasetPresence(context.getPipelineOptions(), table); + if (transform.getCreateDisposition() == BigQueryIO.Write.CreateDisposition.CREATE_NEVER) { + verifyTablePresence(context.getPipelineOptions(), table); + } + if (transform.getWriteDisposition() == BigQueryIO.Write.WriteDisposition.WRITE_EMPTY) { + verifyTableEmpty(context.getPipelineOptions(), table); + } + } + + // Actual translation. + context.addStep(transform, "ParallelWrite"); + context.addInput(PropertyNames.FORMAT, "bigquery"); + context.addInput(PropertyNames.BIGQUERY_TABLE, + table.getTableId()); + context.addInput(PropertyNames.BIGQUERY_DATASET, + table.getDatasetId()); + if (table.getProjectId() != null) { + context.addInput(PropertyNames.BIGQUERY_PROJECT, table.getProjectId()); + } + if (transform.getSchema() != null) { + try { + context.addInput(PropertyNames.BIGQUERY_SCHEMA, + JSON_FACTORY.toString(transform.getSchema())); + } catch (IOException exn) { + throw new IllegalArgumentException("Invalid table schema.", exn); + } + } + context.addInput( + PropertyNames.BIGQUERY_CREATE_DISPOSITION, + transform.getCreateDisposition().name()); + context.addInput( + PropertyNames.BIGQUERY_WRITE_DISPOSITION, + transform.getWriteDisposition().name()); + // Set sink encoding to TableRowJsonCoder. + context.addEncodingInput( + WindowedValue.getValueOnlyCoder(TableRowJsonCoder.of())); + context.addInput(PropertyNames.PARALLEL_INPUT, transform.getInput()); + } + } + + ///////////////////////////////////////////////////////////////////////////// + + private static void verifyDatasetPresence( + BigQueryOptions options, + TableReference table) { + try { + Bigquery client = Transport.newBigQueryClient(options).build(); + client.datasets().get(table.getProjectId(), table.getDatasetId()) + .execute(); + } catch (IOException e) { + ApiErrorExtractor errorExtractor = new ApiErrorExtractor(); + if (errorExtractor.itemNotFound(e)) { + throw new IllegalArgumentException( + "BigQuery dataset not found for table: " + BigQueryIO.toTableSpec(table), e); + } else { + throw new RuntimeException( + "unable to confirm BigQuery dataset presence", e); + } + } + } + + private static void verifyTablePresence( + BigQueryOptions options, + TableReference table) { + try { + Bigquery client = Transport.newBigQueryClient(options).build(); + client.tables().get(table.getProjectId(), table.getDatasetId(), table.getTableId()) + .execute(); + } catch (IOException e) { + ApiErrorExtractor errorExtractor = new ApiErrorExtractor(); + if (errorExtractor.itemNotFound(e)) { + throw new IllegalArgumentException( + "BigQuery table not found: " + BigQueryIO.toTableSpec(table), e); + } else { + throw new RuntimeException( + "unable to confirm BigQuery table presence", e); + } + } + } + + private static void verifyTableEmpty( + BigQueryOptions options, + TableReference table) { + try { + Bigquery client = Transport.newBigQueryClient(options).build(); + BigQueryTableInserter inserter = new BigQueryTableInserter(client, table); + if (!inserter.isEmpty()) { + throw new IllegalArgumentException( + "BigQuery table is not empty: " + BigQueryIO.toTableSpec(table)); + } + } catch (IOException e) { + ApiErrorExtractor errorExtractor = new ApiErrorExtractor(); + if (errorExtractor.itemNotFound(e)) { + // Nothing to do. If the table does not exist, it is considered empty. + } else { + throw new RuntimeException( + "unable to confirm BigQuery table emptiness", e); + } + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DatastoreIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DatastoreIOTranslator.java new file mode 100644 index 0000000000000..4292199174a14 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DatastoreIOTranslator.java @@ -0,0 +1,41 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.dataflow; + +import com.google.cloud.dataflow.sdk.io.DatastoreIO; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext; + +/** + * Datastore transform support code for the Dataflow backend. + */ +public class DatastoreIOTranslator { + + /** + * Implements DatastoreIO Write translation for the Dataflow backend. + */ + public static class WriteTranslator implements TransformTranslator { + @Override + public void translate( + DatastoreIO.Write.Bound transform, + TranslationContext context) { + // TODO: Not implemented yet. + // translateWriteHelper(transform, context); + throw new UnsupportedOperationException("Write only supports direct mode now."); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java new file mode 100644 index 0000000000000..706397bddd37b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java @@ -0,0 +1,91 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.dataflow; + +import com.google.cloud.dataflow.sdk.io.PubsubIO; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.WindowedValue; + +/** + * Pubsub transform support code for the Dataflow backend. + */ +public class PubsubIOTranslator { + + /** + * Implements PubsubIO Read translation for the Dataflow backend. + */ + public static class ReadTranslator implements TransformTranslator { + @Override + public void translate( + PubsubIO.Read.Bound transform, + TranslationContext context) { + translateReadHelper(transform, context); + } + + /* + private static void translateReadHelper( + */ + + private void translateReadHelper( + PubsubIO.Read.Bound transform, + TranslationContext context) { + if (!context.getPipelineOptions().isStreaming()) { + throw new IllegalArgumentException("PubsubIO can only be used in streaming mode."); + } + + context.addStep(transform, "ParallelRead"); + context.addInput(PropertyNames.FORMAT, "pubsub"); + if (transform.getTopic() != null) { + context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic()); + } + if (transform.getSubscription() != null) { + context.addInput(PropertyNames.PUBSUB_SUBSCRIPTION, transform.getSubscription()); + } + context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); + // TODO: Orderedness? + } + } + + /** + * Implements PubsubIO Write translation for the Dataflow backend. + */ + public static class WriteTranslator implements TransformTranslator { + @Override + public void translate( + PubsubIO.Write.Bound transform, + TranslationContext context) { + translateWriteHelper(transform, context); + } + + private void translateWriteHelper( + PubsubIO.Write.Bound transform, + TranslationContext context) { + if (!context.getPipelineOptions().isStreaming()) { + throw new IllegalArgumentException("PubsubIO can only be used in streaming mode."); + } + + context.addStep(transform, "ParallelWrite"); + context.addInput(PropertyNames.FORMAT, "pubsub"); + context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic()); + context.addEncodingInput( + WindowedValue.getValueOnlyCoder(transform.getInput().getCoder())); + context.addInput(PropertyNames.PARALLEL_INPUT, transform.getInput()); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java new file mode 100644 index 0000000000000..05a44648eba95 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java @@ -0,0 +1,129 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.dataflow; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.io.ShardNameTemplate; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext; +import com.google.cloud.dataflow.sdk.util.GcsUtil; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; + +/** + * TextIO transform support code for the Dataflow backend. + */ +public class TextIOTranslator { + + /** + * Implements TextIO Read translation for the Dataflow backend. + */ + public static class ReadTranslator implements TransformTranslator { + @Override + public void translate( + TextIO.Read.Bound transform, + TranslationContext context) { + translateReadHelper(transform, context); + } + + private void translateReadHelper( + TextIO.Read.Bound transform, + TranslationContext context) { + if (context.getPipelineOptions().isStreaming()) { + throw new IllegalArgumentException("TextIO not supported in streaming mode."); + } + + // Only GCS paths are permitted for filepatterns in the DataflowPipelineRunner. + GcsPath gcsPath = GcsPath.fromUri(transform.getFilepattern()); + // Furthermore, on the service there is currently a limitation + // that the first wildcard character must occur after the last + // delimiter, and that the delimiter is fixed to '/' + if (!GcsUtil.GCS_READ_PATTERN.matcher(gcsPath.getObject()).matches()) { + throw new IllegalArgumentException( + "Unsupported wildcard usage in \"" + gcsPath + "\": " + + " all wildcards must occur after the final '/' delimiter."); + } + + context.addStep(transform, "ParallelRead"); + // TODO: How do we want to specify format and + // format-specific properties? + context.addInput(PropertyNames.FORMAT, "text"); + context.addInput(PropertyNames.FILEPATTERN, gcsPath); + context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); + + // TODO: Orderedness? + } + } + + /** + * Implements TextIO Write translation for the Dataflow backend. + */ + public static class WriteTranslator implements TransformTranslator { + @Override + public void translate( + TextIO.Write.Bound transform, + TranslationContext context) { + translateWriteHelper(transform, context); + } + + private void translateWriteHelper( + TextIO.Write.Bound transform, + TranslationContext context) { + if (context.getPipelineOptions().isStreaming()) { + throw new IllegalArgumentException("TextIO not supported in streaming mode."); + } + + // Only GCS paths are permitted for filepatterns in the DataflowPipelineRunner. + GcsPath gcsPath = GcsPath.fromUri(transform.getFilenamePrefix()); + context.addStep(transform, "ParallelWrite"); + context.addInput(PropertyNames.PARALLEL_INPUT, transform.getInput()); + + // TODO: drop this check when server supports alternative templates. + switch (transform.getShardTemplate()) { + case ShardNameTemplate.INDEX_OF_MAX: + break; // supported by server + case "": + // Empty shard template allowed - forces single output. + Preconditions.checkArgument(transform.getNumShards() <= 1, + "Num shards must be <= 1 when using an empty sharding template"); + break; + default: + throw new UnsupportedOperationException("Shard template " + + transform.getShardTemplate() + + " not yet supported by Dataflow service"); + } + + // TODO: How do we want to specify format and + // format-specific properties? + context.addInput(PropertyNames.FORMAT, "text"); + context.addInput(PropertyNames.FILENAME_PREFIX, gcsPath); + context.addInput(PropertyNames.SHARD_NAME_TEMPLATE, + transform.getShardNameTemplate()); + context.addInput(PropertyNames.FILENAME_SUFFIX, transform.getFilenameSuffix()); + + long numShards = transform.getNumShards(); + if (numShards > 0) { + context.addInput(PropertyNames.NUM_SHARDS, numShards); + } + + context.addEncodingInput( + WindowedValue.getValueOnlyCoder(transform.getCoder())); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/package-info.java new file mode 100644 index 0000000000000..c2fcc288cf3c4 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/package-info.java @@ -0,0 +1,20 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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. + */ + +/** + * Implementation of the {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}. + */ +package com.google.cloud.dataflow.sdk.runners.dataflow; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/package-info.java new file mode 100644 index 0000000000000..c75fe2f8348e0 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/package-info.java @@ -0,0 +1,33 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Defines runners for executing Pipelines in different modes, including + * {@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner} and + * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}. + * + *

{@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner} executes a {@code Pipeline} + * locally, without contacting the Dataflow service. + * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner} submits a + * {@code Pipeline} to the Dataflow service, which executes it on Dataflow-managed Compute Engine + * instances. {@code DataflowPipelineRunner} returns + * as soon as the {@code Pipeline} has been submitted. Use + * {@link com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner} to have execution + * updates printed to the console. + * + *

The runner is specified as part {@link com.google.cloud.dataflow.sdk.options.PipelineOptions}. + */ +package com.google.cloud.dataflow.sdk.runners; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ApplianceShuffleReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ApplianceShuffleReader.java new file mode 100644 index 0000000000000..912c570f8efaa --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ApplianceShuffleReader.java @@ -0,0 +1,63 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import java.io.IOException; + +import javax.annotation.concurrent.ThreadSafe; + +/** + * ApplianceShuffleReader reads chunks of data from a shuffle dataset + * for a position range. + * + * It is a JNI wrapper of an equivalent C++ class. + */ +@ThreadSafe +public final class ApplianceShuffleReader implements ShuffleReader { + static { + ShuffleLibrary.load(); + } + + /** + * Pointer to the underlying native shuffle reader object. + */ + private long nativePointer; + + /** + * @param shuffleReaderConfig opaque configuration for creating a + * shuffle reader + */ + public ApplianceShuffleReader(byte[] shuffleReaderConfig) { + this.nativePointer = createFromConfig(shuffleReaderConfig); + } + + @Override + public void finalize() { + destroy(); + } + + /** + * Native methods for interacting with the underlying native shuffle client + * code. + */ + private native long createFromConfig(byte[] shuffleReaderConfig); + private native void destroy(); + + @Override + public native ReadChunkResult readIncludingPosition( + byte[] startPosition, byte[] endPosition) throws IOException; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ApplianceShuffleWriter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ApplianceShuffleWriter.java new file mode 100644 index 0000000000000..d6b3c7518e3e0 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ApplianceShuffleWriter.java @@ -0,0 +1,66 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import java.io.IOException; +import javax.annotation.concurrent.ThreadSafe; + +/** + * ApplianceShuffleWriter writes chunks of data to a shuffle dataset. + * + * 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; + + /** + * @param shuffleWriterConfig opaque configuration for creating a + * shuffle writer + * @param bufferSize the writer buffer size + */ + public ApplianceShuffleWriter(byte[] shuffleWriterConfig, + long bufferSize) { + this.nativePointer = createFromConfig(shuffleWriterConfig, bufferSize); + } + + @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(); + + @Override + public native void write(byte[] chunk) throws IOException; + + @Override + public native void close() throws IOException; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java new file mode 100644 index 0000000000000..f1ae7f11b9374 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java @@ -0,0 +1,86 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getBytes; + +import com.google.api.services.dataflow.model.MultiOutputInfo; +import com.google.api.services.dataflow.model.SideInputInfo; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PTuple; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; + +import java.util.Arrays; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * A wrapper around an AssignWindowsDoFn. This class is the same as + * NormalParDoFn, except that it gets deserialized differently. + */ +class AssignWindowsParDoFn extends NormalParDoFn { + public static AssignWindowsParDoFn create( + PipelineOptions options, + CloudObject cloudUserFn, + String stepName, + @Nullable List sideInputInfos, + @Nullable List multiOutputInfos, + Integer numOutputs, + ExecutionContext executionContext, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler sampler /* unused */) + throws Exception { + Object windowingFn = + SerializableUtils.deserializeFromByteArray( + getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), + "serialized window fn"); + if (!(windowingFn instanceof WindowingFn)) { + throw new Exception( + "unexpected kind of WindowingFn: " + windowingFn.getClass().getName()); + } + + DoFn assignWindowsDoFn = new AssignWindowsDoFn((WindowingFn) windowingFn); + + return new AssignWindowsParDoFn( + options, assignWindowsDoFn, stepName, executionContext, addCounterMutator); + } + + private AssignWindowsParDoFn( + PipelineOptions options, + DoFn fn, + String stepName, + ExecutionContext executionContext, + CounterSet.AddCounterMutator addCounterMutator) { + super( + options, + fn, + PTuple.empty(), + Arrays.asList("output"), + stepName, + executionContext, + addCounterMutator); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSink.java new file mode 100644 index 0000000000000..404b2d261fc98 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSink.java @@ -0,0 +1,83 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumWriter; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * 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 the type of the elements written to the sink + */ +public class AvroByteSink extends Sink { + + final AvroSink avroSink; + final Coder coder; + private final Schema schema = Schema.create(Schema.Type.BYTES); + + public AvroByteSink(String filenamePrefix, Coder coder) { + this(filenamePrefix, "", "", 1, coder); + } + + public AvroByteSink(String filenamePrefix, String shardFormat, String filenameSuffix, + int shardCount, Coder coder) { + this.coder = coder; + avroSink = new AvroSink( + filenamePrefix, shardFormat, filenameSuffix, shardCount, + WindowedValue.getValueOnlyCoder(AvroCoder.of(ByteBuffer.class, schema))); + } + + @Override + public SinkWriter writer() throws IOException { + return new AvroByteFileWriter(); + } + + /** The SinkWriter for an AvroByteSink. */ + class AvroByteFileWriter implements SinkWriter { + + private final SinkWriter> avroFileWriter; + + public AvroByteFileWriter() throws IOException { + avroFileWriter = avroSink.writer(new GenericDatumWriter(schema)); + } + + @Override + public long add(T value) throws IOException { + byte[] encodedElem = CoderUtils.encodeToByteArray(coder, value); + ByteBuffer encodedBuffer = ByteBuffer.wrap(encodedElem); + avroFileWriter.add(WindowedValue.valueInGlobalWindow(encodedBuffer)); + return encodedElem.length; + } + + @Override + public void close() throws IOException { + avroFileWriter.close(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSource.java new file mode 100644 index 0000000000000..b71700a08fcab --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSource.java @@ -0,0 +1,95 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumReader; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +/** + * A source that reads 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 the type of the elements read from the source + */ +public class AvroByteSource extends Source { + + final AvroSource avroSource; + final Coder coder; + private final Schema schema = Schema.create(Schema.Type.BYTES); + + public AvroByteSource(String filename, + @Nullable Long startPosition, + @Nullable Long endPosition, + Coder coder) { + this.coder = coder; + avroSource = new AvroSource( + filename, startPosition, endPosition, + WindowedValue.getValueOnlyCoder(AvroCoder.of(ByteBuffer.class, schema))); + } + + @Override + public SourceIterator iterator() throws IOException { + return new AvroByteFileIterator(); + } + + class AvroByteFileIterator extends AbstractSourceIterator { + + private final SourceIterator> avroFileIterator; + + public AvroByteFileIterator() throws IOException { + avroFileIterator = avroSource.iterator( + new GenericDatumReader(schema)); + } + + @Override + public boolean hasNext() throws IOException { + return avroFileIterator.hasNext(); + } + + @Override + public T next() throws IOException { + if (!hasNext()) { + throw new NoSuchElementException(); + } + ByteBuffer inBuffer = avroFileIterator.next().getValue(); + byte[] encodedElem = new byte[inBuffer.remaining()]; + inBuffer.get(encodedElem); + assert inBuffer.remaining() == 0; + inBuffer.clear(); + notifyElementRead(encodedElem.length); + return CoderUtils.decodeFromByteArray(coder, encodedElem); + } + + @Override + public void close() throws IOException { + avroFileIterator.close(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSink.java new file mode 100644 index 0000000000000..64fe691aa41f9 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSink.java @@ -0,0 +1,140 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.WindowedValue.ValueOnlyWindowedValueCoder; +import static com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; + +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.MimeTypes; +import com.google.cloud.dataflow.sdk.util.ShardingWritableByteChannel; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; + +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.io.DatumWriter; + +import java.io.IOException; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; +import java.util.Random; + +/** + * A sink that writes Avro files. + * + * @param the type of the elements written to the sink + */ +public class AvroSink extends Sink> { + + final String filenamePrefix; + final String shardFormat; + final String filenameSuffix; + final int shardCount; + final AvroCoder avroCoder; + final Schema schema; + + public AvroSink(String filename, WindowedValueCoder coder) { + this(filename, "", "", 1, coder); + } + + public AvroSink(String filenamePrefix, String shardFormat, String filenameSuffix, int shardCount, + WindowedValueCoder coder) { + if (!(coder instanceof ValueOnlyWindowedValueCoder)) { + throw new IllegalArgumentException("Expected ValueOnlyWindowedValueCoder"); + } + + if (!(coder.getValueCoder() instanceof AvroCoder)) { + throw new IllegalArgumentException("AvroSink requires an AvroCoder"); + } + + this.filenamePrefix = filenamePrefix; + this.shardFormat = shardFormat; + this.filenameSuffix = filenameSuffix; + this.shardCount = shardCount; + this.avroCoder = (AvroCoder) coder.getValueCoder(); + this.schema = this.avroCoder.getSchema(); + } + + public SinkWriter> writer(DatumWriter datumWriter) throws IOException { + WritableByteChannel writer = IOChannelUtils.create( + filenamePrefix, shardFormat, filenameSuffix, shardCount, MimeTypes.BINARY); + + if (writer instanceof ShardingWritableByteChannel) { + return new AvroShardingFileWriter(datumWriter, (ShardingWritableByteChannel) writer); + } else { + return new AvroFileWriter(datumWriter, writer); + } + } + + @Override + public SinkWriter> writer() throws IOException { + return writer(avroCoder.createDatumWriter()); + } + + /** The SinkWriter for an AvroSink. */ + class AvroFileWriter implements SinkWriter> { + DataFileWriter fileWriter; + + public AvroFileWriter(DatumWriter datumWriter, WritableByteChannel outputChannel) + throws IOException { + fileWriter = new DataFileWriter<>(datumWriter); + fileWriter.create(schema, Channels.newOutputStream(outputChannel)); + } + + @Override + public long add(WindowedValue value) throws IOException { + fileWriter.append(value.getValue()); + // DataFileWriter doesn't support returning the length written. Use the + // coder instead. + return CoderUtils.encodeToByteArray(avroCoder, value.getValue()).length; + } + + @Override + public void close() throws IOException { + fileWriter.close(); + } + } + + /** The SinkWriter for an AvroSink, which supports sharding. */ + class AvroShardingFileWriter implements SinkWriter> { + private ArrayList fileWriters = new ArrayList<>(); + private final Random random = new Random(); + + public AvroShardingFileWriter( + DatumWriter datumWriter, ShardingWritableByteChannel outputChannel) throws IOException { + for (int i = 0; i < outputChannel.getNumShards(); i++) { + fileWriters.add(new AvroFileWriter(datumWriter, outputChannel.getChannel(i))); + } + } + + @Override + public long add(WindowedValue value) throws IOException { + return fileWriters.get(random.nextInt(fileWriters.size())).add(value); + } + + @Override + public void close() throws IOException { + for (AvroFileWriter fileWriter : fileWriters) { + fileWriter.close(); + } + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java new file mode 100644 index 0000000000000..9a20d17aee220 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; + +/** + * Creates an AvroSink from a CloudObject spec. + */ +public final class AvroSinkFactory { + // Do not instantiate. + private AvroSinkFactory() {} + + public static Sink create(PipelineOptions options, + CloudObject spec, + Coder coder, + ExecutionContext executionContext) + throws Exception { + return create(spec, coder); + } + + static Sink create(CloudObject spec, Coder coder) + throws Exception { + String filename = getString(spec, PropertyNames.FILENAME); + + if (!(coder instanceof WindowedValueCoder)) { + return new AvroByteSink<>(filename, coder); + //throw new IllegalArgumentException("Expected WindowedValueCoder"); + } + + WindowedValueCoder windowedCoder = (WindowedValueCoder) coder; + if (windowedCoder.getValueCoder() instanceof AvroCoder) { + return new AvroSink(filename, windowedCoder); + } else { + return new AvroByteSink<>(filename, windowedCoder); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSource.java new file mode 100644 index 0000000000000..3f071cff2c7a1 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSource.java @@ -0,0 +1,203 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.WindowedValue.ValueOnlyWindowedValueCoder; +import static com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; + +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.IOChannelFactory; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.file.SeekableInput; +import org.apache.avro.io.DatumReader; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.SeekableByteChannel; +import java.util.Collection; +import java.util.Iterator; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +/** + * A source that reads Avro files. + * + * @param the type of the elements read from the source + */ +public class AvroSource extends Source> { + private static final int BUF_SIZE = 200; + final String filename; + @Nullable final Long startPosition; + @Nullable final Long endPosition; + final AvroCoder avroCoder; + private final Schema schema; + + public AvroSource(String filename, + @Nullable Long startPosition, + @Nullable Long endPosition, + WindowedValueCoder coder) { + if (!(coder instanceof ValueOnlyWindowedValueCoder)) { + throw new IllegalArgumentException("Expected ValueOnlyWindowedValueCoder"); + } + + if (!(coder.getValueCoder() instanceof AvroCoder)) { + throw new IllegalArgumentException("AvroSource requires an AvroCoder"); + } + + this.filename = filename; + this.startPosition = startPosition; + this.endPosition = endPosition; + this.avroCoder = (AvroCoder) coder.getValueCoder(); + this.schema = this.avroCoder.getSchema(); + } + + public SourceIterator> iterator(DatumReader datumReader) throws IOException { + IOChannelFactory factory = IOChannelUtils.getFactory(filename); + Collection inputs = factory.match(filename); + + if (inputs.size() == 1) { + String input = inputs.iterator().next(); + ReadableByteChannel reader = factory.open(input); + return new AvroFileIterator(datumReader, input, reader, startPosition, endPosition); + + } else { + if (startPosition != null || endPosition != null) { + throw new UnsupportedOperationException( + "Unable to apply range limits to multiple-input stream: " + + filename); + } + return new AvroFileMultiIterator(datumReader, factory, inputs.iterator()); + } + } + + @Override + public SourceIterator> iterator() throws IOException { + return iterator(avroCoder.createDatumReader()); + } + + class AvroFileMultiIterator extends LazyMultiSourceIterator> { + private final IOChannelFactory factory; + private final DatumReader datumReader; + + public AvroFileMultiIterator(DatumReader datumReader, + IOChannelFactory factory, + Iterator inputs) { + super(inputs); + this.factory = factory; + this.datumReader = datumReader; + } + + @Override + protected SourceIterator> open(String input) throws IOException { + return new AvroFileIterator(datumReader, input, factory.open(input), null, null); + } + } + + class AvroFileIterator extends AbstractSourceIterator> { + final DataFileReader fileReader; + final Long endOffset; + + public AvroFileIterator(DatumReader datumReader, + String filename, + ReadableByteChannel reader, + @Nullable Long startOffset, + @Nullable Long endOffset) + throws IOException { + if (!(reader instanceof SeekableByteChannel)) { + throw new UnsupportedOperationException( + "Unable to seek to offset in stream for " + filename); + } + SeekableByteChannel inChannel = (SeekableByteChannel) reader; + SeekableInput seekableInput = new SeekableByteChannelInput(inChannel); + this.fileReader = new DataFileReader<>(seekableInput, datumReader); + this.endOffset = endOffset; + if (startOffset != null && startOffset > 0) { + // Sync to the first record at or after startOffset. + fileReader.sync(startOffset); + } + } + + @Override + public boolean hasNext() throws IOException { + return fileReader.hasNext() + && (endOffset == null || !fileReader.pastSync(endOffset)); + } + + @Override + public WindowedValue next() throws IOException { + if (!hasNext()) { + throw new NoSuchElementException(); + } + T next = fileReader.next(); + // DataFileReader doesn't seem to support getting the current position. + // The difference between tell() calls seems to be zero. Use the coder + // instead. + notifyElementRead(CoderUtils.encodeToByteArray(avroCoder, next).length); + return WindowedValue.valueInGlobalWindow(next); + } + + @Override + public void close() throws IOException { + fileReader.close(); + } + } + + /** + * An implementation of an Avro SeekableInput wrapping a + * SeekableByteChannel. + */ + static class SeekableByteChannelInput implements SeekableInput { + final SeekableByteChannel channel; + + public SeekableByteChannelInput(SeekableByteChannel channel) { + this.channel = channel; + } + + @Override + public void seek(long position) throws IOException { + channel.position(position); + } + + @Override + public long tell() throws IOException { + return channel.position(); + } + + @Override + public long length() throws IOException { + return channel.size(); + } + + @Override + public int read(byte[] b, int offset, int length) throws IOException { + return channel.read(ByteBuffer.wrap(b, offset, length)); + } + + @Override + public void close() throws IOException { + channel.close(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactory.java new file mode 100644 index 0000000000000..329d8b66e2ee1 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactory.java @@ -0,0 +1,65 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getLong; +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +/** + * Creates an AvroSource from a CloudObject spec. + */ +public class AvroSourceFactory { + // Do not instantiate. + private AvroSourceFactory() {} + + public static Source create(PipelineOptions options, + CloudObject spec, + Coder coder, + ExecutionContext executionContext) + throws Exception { + return create(spec, coder); + } + + static Source create(CloudObject spec, + Coder coder) + throws Exception { + String filename = getString(spec, PropertyNames.FILENAME); + Long startOffset = getLong(spec, PropertyNames.START_OFFSET, null); + Long endOffset = getLong(spec, PropertyNames.END_OFFSET, null); + + if (!(coder instanceof WindowedValueCoder)) { + return new AvroByteSource<>(filename, startOffset, endOffset, coder); + //throw new IllegalArgumentException("Expected WindowedValueCoder"); + } + + WindowedValueCoder windowedCoder = (WindowedValueCoder) coder; + if (windowedCoder.getValueCoder() instanceof AvroCoder) { + return new AvroSource(filename, startOffset, endOffset, windowedCoder); + } else { + return new AvroByteSource<>(filename, startOffset, endOffset, windowedCoder); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySource.java new file mode 100644 index 0000000000000..b43c942b3ed98 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySource.java @@ -0,0 +1,114 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Preconditions.checkNotNull; + +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.dataflow.sdk.options.BigQueryOptions; +import com.google.cloud.dataflow.sdk.util.BigQueryTableRowIterator; +import com.google.cloud.dataflow.sdk.util.Transport; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import java.io.IOException; +import java.util.NoSuchElementException; +import java.util.logging.Logger; + +/** + * A source that reads a BigQuery table and yields TableRow objects. + * + *

The source is a wrapper over the {@code BigQueryTableRowIterator} class which issues a + * query for all rows of a table and then iterates over the result. There is no support for + * progress reporting because the source is used only in situations where the entire table must be + * read by each worker (i.e. the source is used as a side input). + */ +public class BigQuerySource extends Source { + private static final Logger LOG = + Logger.getLogger(BigQuerySource.class.getName()); + + final TableReference tableRef; + final BigQueryOptions bigQueryOptions; + final Bigquery bigQueryClient; + + /** Builds a BigQuery source using pipeline options to instantiate a Bigquery client. */ + public BigQuerySource(BigQueryOptions bigQueryOptions, TableReference tableRef) { + // Save pipeline options so that we can construct the BigQuery client on-demand whenever an + // iterator gets created. + this.bigQueryOptions = bigQueryOptions; + this.tableRef = tableRef; + this.bigQueryClient = null; + } + + /** Builds a BigQuerySource directly using a BigQuery client. */ + public BigQuerySource(Bigquery bigQueryClient, TableReference tableRef) { + this.bigQueryOptions = null; + this.tableRef = tableRef; + this.bigQueryClient = bigQueryClient; + } + + @Override + public SourceIterator iterator() throws IOException { + return new BigQuerySourceIterator( + bigQueryClient != null + ? bigQueryClient + : Transport.newBigQueryClient(bigQueryOptions).build(), + tableRef); + } + + /** + * A SourceIterator that yields TableRow objects for each row of a BigQuery table. + */ + class BigQuerySourceIterator extends AbstractSourceIterator { + + private BigQueryTableRowIterator rowIterator; + + public BigQuerySourceIterator(Bigquery bigQueryClient, TableReference tableRef) { + rowIterator = new BigQueryTableRowIterator(bigQueryClient, tableRef); + } + + @Override + public boolean hasNext() { + return rowIterator.hasNext(); + } + + @Override + public TableRow next() throws IOException { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return rowIterator.next(); + } + + @Override + public Progress getProgress() { + // For now reporting progress is not supported because this source is used only when + // an entire table needs to be read by each worker (used as a side input for instance). + throw new UnsupportedOperationException(); + } + + @Override + public Position updateStopPosition(Progress proposedStopPosition) { + // For now updating the stop position is not supported because this source + // is used only when an entire table needs to be read by each worker (used + // as a side input for instance). + checkNotNull(proposedStopPosition); + throw new UnsupportedOperationException(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactory.java new file mode 100644 index 0000000000000..682b7faa1400b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactory.java @@ -0,0 +1,46 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.api.services.bigquery.model.TableReference; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.BigQueryOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; + +/** + * Creates a BigQuerySource from a {@link CloudObject} spec. + */ +public class BigQuerySourceFactory { + // Do not instantiate. + private BigQuerySourceFactory() {} + + public static BigQuerySource create( + PipelineOptions options, CloudObject spec, Coder coder, + ExecutionContext executionContext) throws Exception { + return new BigQuerySource( + options.as(BigQueryOptions.class), + new TableReference() + .setProjectId(getString(spec, PropertyNames.BIGQUERY_PROJECT)) + .setDatasetId(getString(spec, PropertyNames.BIGQUERY_DATASET)) + .setTableId(getString(spec, PropertyNames.BIGQUERY_TABLE))); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ByteArrayShufflePosition.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ByteArrayShufflePosition.java new file mode 100644 index 0000000000000..881f61b730207 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ByteArrayShufflePosition.java @@ -0,0 +1,95 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Base64.decodeBase64; +import static com.google.api.client.util.Base64.encodeBase64URLSafeString; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.util.common.worker.ShufflePosition; +import com.google.common.primitives.UnsignedBytes; + +import java.util.Arrays; + +/** + * Represents a ShufflePosition as an array of bytes. + */ +public class ByteArrayShufflePosition implements Comparable, ShufflePosition { + private final byte[] position; + + public ByteArrayShufflePosition(byte[] position) { + this.position = position; + } + + public static ByteArrayShufflePosition fromBase64(String position) { + return ByteArrayShufflePosition.of(decodeBase64(position)); + } + + public static ByteArrayShufflePosition of(byte[] position) { + if (position == null) { + return null; + } + return new ByteArrayShufflePosition(position); + } + + public static byte[] getPosition(ShufflePosition shufflePosition) { + if (shufflePosition == null) { + return null; + } + Preconditions.checkArgument( + shufflePosition instanceof ByteArrayShufflePosition); + ByteArrayShufflePosition adapter = (ByteArrayShufflePosition) shufflePosition; + return adapter.getPosition(); + } + + public byte[] getPosition() { return position; } + + public String encodeBase64() { + return encodeBase64URLSafeString(position); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o instanceof ByteArrayShufflePosition) { + ByteArrayShufflePosition that = (ByteArrayShufflePosition) o; + return Arrays.equals(this.position, that.position); + } + return false; + } + + @Override + public int hashCode() { + return Arrays.hashCode(position); + } + + @Override + public String toString() { + return "ShufflePosition(" + (new String(position)) + ")"; + } + + @Override + public int compareTo(Object o) { + if (this == o) { + return 0; + } + return UnsignedBytes.lexicographicalComparator().compare( + position, ((ByteArrayShufflePosition) o).position); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ChunkingShuffleBatchReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ChunkingShuffleBatchReader.java new file mode 100644 index 0000000000000..6f746ffec8c59 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ChunkingShuffleBatchReader.java @@ -0,0 +1,97 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleBatchReader; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; +import com.google.cloud.dataflow.sdk.util.common.worker.ShufflePosition; +import com.google.common.io.ByteStreams; + +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; + +import javax.annotation.Nullable; + +/** + * ChunkingShuffleBatchReader reads data from a shuffle dataset using a + * ShuffleReader. + */ +final class ChunkingShuffleBatchReader implements ShuffleBatchReader { + private ShuffleReader reader; + + /** + * @param reader used to read from a shuffle dataset + */ + public ChunkingShuffleBatchReader(ShuffleReader reader) throws IOException { + this.reader = reader; + } + + @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 = + reader.readIncludingPosition(startPosition, endPosition); + InputStream input = new ByteArrayInputStream(result.chunk); + ArrayList 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(InputStream input) throws IOException { + byte[] position = getFixedLengthPrefixedByteArray(input); + byte[] key = getFixedLengthPrefixedByteArray(input); + byte[] skey = getFixedLengthPrefixedByteArray(input); + byte[] value = getFixedLengthPrefixedByteArray(input); + return new ShuffleEntry(position, key, skey, value); + } + + /** + * Extracts a length-prefix-encoded byte array from a given InputStream. + * + * @param input stream to read from + * @return parsed byte array + */ + static byte[] getFixedLengthPrefixedByteArray(InputStream input) + throws IOException { + DataInputStream dataInputStream = new DataInputStream(input); + int length = dataInputStream.readInt(); + if (length < 0) { + throw new IOException("invalid length: " + length); + } + byte[] data = new byte[(int) length]; + ByteStreams.readFully(dataInputStream, data); + return data; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ChunkingShuffleEntryWriter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ChunkingShuffleEntryWriter.java new file mode 100644 index 0000000000000..9c55c181aebfe --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ChunkingShuffleEntryWriter.java @@ -0,0 +1,87 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Preconditions.checkNotNull; + +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +import javax.annotation.concurrent.NotThreadSafe; + +/** + * ChunkingShuffleEntryWriter buffers ShuffleEntries and writes them + * in batches to a shuffle dataset using a given writer. + */ +@NotThreadSafe +final class ChunkingShuffleEntryWriter implements ShuffleEntryWriter { + // Approximate maximum size of a chunk in bytes. + private static final int MAX_CHUNK_SIZE = 1 << 20; + + private static final byte[] EMPTY_BYTES = new byte[0]; + + private ByteArrayOutputStream chunk = new ByteArrayOutputStream(); + + private final ShuffleWriter writer; + + /** + * @param writer used to write chunks created by this writer + */ + public ChunkingShuffleEntryWriter(ShuffleWriter writer) { + this.writer = checkNotNull(writer); + } + + @Override + public long put(ShuffleEntry entry) throws IOException { + if (chunk.size() >= MAX_CHUNK_SIZE) { + writeChunk(); + } + + DataOutputStream output = new DataOutputStream(chunk); + return putFixedLengthPrefixedByteArray(entry.getKey(), output) + + putFixedLengthPrefixedByteArray(entry.getSecondaryKey(), output) + + putFixedLengthPrefixedByteArray(entry.getValue(), output); + } + + @Override + public void close() throws IOException { + writeChunk(); + writer.close(); + } + + private void writeChunk() throws IOException { + if (chunk.size() > 0) { + writer.write(chunk.toByteArray()); + chunk.reset(); + } + } + + static int putFixedLengthPrefixedByteArray(byte[] data, + DataOutputStream output) + throws IOException { + if (data == null) { + data = EMPTY_BYTES; + } + int bytesWritten = output.size(); + output.writeInt(data.length); + output.write(data, 0, data.length); + return output.size() - bytesWritten; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java new file mode 100644 index 0000000000000..16230571fae12 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java @@ -0,0 +1,219 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getBytes; +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.api.services.dataflow.model.MultiOutputInfo; +import com.google.api.services.dataflow.model.SideInputInfo; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PTuple; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.base.Preconditions; + +import java.util.Arrays; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * A wrapper around a decoded user value combining function. + */ +public class CombineValuesFn extends NormalParDoFn { + /** + * 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 static 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"; + } + + public static CombineValuesFn create( + PipelineOptions options, + CloudObject cloudUserFn, + String stepName, + @Nullable List sideInputInfos, + @Nullable List multiOutputInfos, + Integer numOutputs, + ExecutionContext executionContext, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler /* unused */) + throws Exception { + Object deserializedFn = + SerializableUtils.deserializeFromByteArray( + getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), + "serialized user fn"); + Preconditions.checkArgument( + deserializedFn instanceof Combine.KeyedCombineFn); + Combine.KeyedCombineFn combineFn = (Combine.KeyedCombineFn) deserializedFn; + + // Get the combine phase, default to ALL. (The implementation + // doesn't have to split the combiner). + String phase = getString(cloudUserFn, PropertyNames.PHASE, CombinePhase.ALL); + + Preconditions.checkArgument( + sideInputInfos == null || sideInputInfos.size() == 0, + "unexpected side inputs for CombineValuesFn"); + Preconditions.checkArgument( + numOutputs == 1, "expected exactly one output for CombineValuesFn"); + + DoFn doFn = null; + switch (phase) { + case CombinePhase.ALL: + doFn = new CombineValuesDoFn(combineFn); + break; + case CombinePhase.ADD: + doFn = new AddInputsDoFn(combineFn); + break; + case CombinePhase.MERGE: + doFn = new MergeAccumulatorsDoFn(combineFn); + break; + case CombinePhase.EXTRACT: + doFn = new ExtractOutputDoFn(combineFn); + break; + default: + throw new IllegalArgumentException( + "phase must be one of 'all', 'add', 'merge', 'extract'"); + } + return new CombineValuesFn(options, doFn, stepName, executionContext, addCounterMutator); + } + + private CombineValuesFn( + PipelineOptions options, + DoFn doFn, + String stepName, + ExecutionContext executionContext, + CounterSet.AddCounterMutator addCounterMutator) { + super( + options, + doFn, + PTuple.empty(), + Arrays.asList("output"), + stepName, + executionContext, + addCounterMutator); + } + + /** + * 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 + extends DoFn>, KV>{ + private final Combine.KeyedCombineFn combineFn; + + private CombineValuesDoFn( + Combine.KeyedCombineFn combineFn) { + this.combineFn = combineFn; + } + + @Override + public void processElement(ProcessContext c) { + KV> kv = (KV>) c.element(); + K key = (K) kv.getKey(); + + c.output(KV.of(key, this.combineFn.apply(key, kv.getValue()))); + } + } + + /** + * ADD phase: KV> -> KV + */ + private static class AddInputsDoFn + extends DoFn>, KV>{ + private final Combine.KeyedCombineFn combineFn; + + private AddInputsDoFn( + Combine.KeyedCombineFn combineFn) { + this.combineFn = combineFn; + } + + @Override + public void processElement(ProcessContext c) { + KV> kv = (KV>) c.element(); + K key = kv.getKey(); + VA accum = this.combineFn.createAccumulator(key); + for (VI input : kv.getValue()) { + this.combineFn.addInput(key, accum, input); + } + + c.output(KV.of(key, accum)); + } + } + + /** + * MERGE phase: KV> -> KV + */ + private static class MergeAccumulatorsDoFn + extends DoFn>, KV>{ + private final Combine.KeyedCombineFn combineFn; + + private MergeAccumulatorsDoFn( + Combine.KeyedCombineFn combineFn) { + this.combineFn = combineFn; + } + + @Override + public void processElement(ProcessContext c) { + KV> kv = (KV>) c.element(); + K key = kv.getKey(); + VA accum = this.combineFn.mergeAccumulators(key, kv.getValue()); + + c.output(KV.of(key, accum)); + } + } + + /** + * EXTRACT phase: KV> -> KV + */ + private static class ExtractOutputDoFn + extends DoFn, KV>{ + private final Combine.KeyedCombineFn combineFn; + + private ExtractOutputDoFn( + Combine.KeyedCombineFn combineFn) { + this.combineFn = combineFn; + } + + @Override + public void processElement(ProcessContext c) { + KV kv = (KV) c.element(); + K key = kv.getKey(); + VO output = this.combineFn.extractOutput(key, kv.getValue()); + + c.output(KV.of(key, output)); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CopyableSeekableByteChannel.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CopyableSeekableByteChannel.java new file mode 100644 index 0000000000000..660b374665572 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CopyableSeekableByteChannel.java @@ -0,0 +1,270 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Preconditions.checkNotNull; +import static com.google.api.client.util.Preconditions.checkState; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; +import java.nio.channels.SeekableByteChannel; + +import javax.annotation.concurrent.GuardedBy; + +/** + * A {@link SeekableByteChannel} that adds copy semantics. + * + *

This implementation uses a lock to ensure that only one thread accesses + * the underlying {@code SeekableByteChannel} at any given time. + * + *

{@link SeekableByteChannel#close} is called on the underlying channel once + * all {@code CopyableSeekableByteChannel} objects copied from the initial + * {@code CopyableSeekableByteChannel} are closed. + * + *

The implementation keeps track of the position of each + * {@code CopyableSeekableByteChannel}; on access, it synchronizes with the + * other {@code CopyableSeekableByteChannel} instances accessing the underlying + * channel, seeks to its own position, performs the operation, updates its local + * position, and returns the result. + */ +final class CopyableSeekableByteChannel implements SeekableByteChannel { + /** This particular stream's position in the base stream. */ + private long pos; + + /** + * The synchronization object keeping track of the base + * {@link SeekableByteChannel}, its reference count, and its current position. + * This also doubles as the lock shared by all + * {@link CopyableSeekableByteChannel} instances derived from some original + * instance. + */ + private final Sync sync; + + /** + * Indicates whether this {@link CopyableSeekableByteChannel} is closed. + * + *

Invariant: Unclosed channels own a reference to the base channel, + * allowing us to make {@link #close} idempotent. + * + *

This is only modified under the sync lock. + */ + private boolean closed; + + /** + * Constructs a new {@link CopyableSeekableByteChannel}. The supplied base + * channel will be closed when this channel and all derived channels are + * closed. + */ + public CopyableSeekableByteChannel(SeekableByteChannel base) throws IOException { + this(new Sync(base), 0); + + // Update the position to match the original stream's position. + // + // This doesn't actually need to be synchronized, but it's a little more + // obviously correct to always access sync.position while holding sync's + // internal monitor. + synchronized (sync) { + sync.position = base.position(); + pos = sync.position; + } + } + + /** + * The internal constructor used when deriving a new + * {@link CopyableSeekableByteChannel}. + * + *

N.B. This signature is deliberately incompatible with the public + * constructor. + * + *

Ordinarily, one would implement copy using a copy constructor, and pass + * the object being copied -- but that signature would be compatible with the + * public constructor creating a new set of + * {@code CopyableSeekableByteChannel} objects for some base channel. The + * copy constructor would still be the one called, since its type is more + * specific, but that's fragile; it'd be easy to tweak the signature of the + * constructor used for copies without changing callers, which would silently + * fall back to using the public constructor. So instead, we're careful to + * give this internal constructor its own unique signature. + */ + private CopyableSeekableByteChannel(Sync sync, long pos) { + this.sync = checkNotNull(sync); + checkState(sync.base.isOpen(), + "the base SeekableByteChannel is not open"); + synchronized (sync) { + sync.refCount++; + } + this.pos = pos; + this.closed = false; + } + + /** + * Creates a new {@link CopyableSeekableByteChannel} derived from an existing + * channel, referencing the same base channel. + */ + public CopyableSeekableByteChannel copy() throws IOException { + synchronized (sync) { + if (closed) { + throw new ClosedChannelException(); + } + return new CopyableSeekableByteChannel(sync, pos); + } + } + + // SeekableByteChannel implementation + + @Override + public long position() throws IOException { + synchronized (sync) { + if (closed) { + throw new ClosedChannelException(); + } + return pos; + } + } + + @Override + public CopyableSeekableByteChannel position(long newPosition) + throws IOException { + synchronized (sync) { + if (closed) { + throw new ClosedChannelException(); + } + // Verify that the position is valid for the base channel. + sync.base.position(newPosition); + this.pos = newPosition; + this.sync.position = newPosition; + } + return this; + } + + @Override + public int read(ByteBuffer dst) throws IOException { + synchronized (sync) { + if (closed) { + throw new ClosedChannelException(); + } + reposition(); + int bytesRead = sync.base.read(dst); + notePositionAdded(bytesRead); + return bytesRead; + } + } + + @Override + public long size() throws IOException { + synchronized (sync) { + if (closed) { + throw new ClosedChannelException(); + } + return sync.base.size(); + } + } + + @Override + public CopyableSeekableByteChannel truncate(long size) throws IOException { + synchronized (sync) { + if (closed) { + throw new ClosedChannelException(); + } + sync.base.truncate(size); + return this; + } + } + + @Override + public int write(ByteBuffer src) throws IOException { + synchronized (sync) { + if (closed) { + throw new ClosedChannelException(); + } + reposition(); + int bytesWritten = sync.base.write(src); + notePositionAdded(bytesWritten); + return bytesWritten; + } + } + + @Override + public boolean isOpen() { + synchronized (sync) { + if (closed) { + return false; + } + return sync.base.isOpen(); + } + } + + @Override + public void close() throws IOException { + synchronized (sync) { + if (closed) { + return; + } + closed = true; + sync.refCount--; + if (sync.refCount == 0) { + sync.base.close(); + } + } + } + + /** + * Updates the base stream's position to match the position required by this + * {@link CopyableSeekableByteChannel}. + */ + @GuardedBy("sync") + private void reposition() throws IOException { + if (pos != sync.position) { + sync.base.position(pos); + sync.position = pos; + } + } + + /** + * Notes that the specified amount has been logically added to the current + * stream's position. + */ + @GuardedBy("sync") + private void notePositionAdded(int amount) { + if (amount < 0) { + return; // Handles EOF indicators. + } + pos += amount; + sync.position += amount; + } + + /** + * A simple value type used to synchronize a set of + * {@link CopyableSeekableByteChannel} instances referencing a single + * underlying channel. + */ + private static final class Sync { + // N.B. Another way to do this would be to implement something like a + // RefcountingForwardingSeekableByteChannel. Doing so would have the + // advantage of clearly isolating the mutable state, at the cost of a lot + // more code. + public final SeekableByteChannel base; + @GuardedBy("this") public long refCount = 0; + @GuardedBy("this") public long position = 0; + + public Sync(SeekableByteChannel base) throws IOException { + this.base = checkNotNull(base); + position = base.position(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CustomSourceFormatFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CustomSourceFormatFactory.java new file mode 100644 index 0000000000000..1bb3db228a730 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CustomSourceFormatFactory.java @@ -0,0 +1,47 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.api.services.dataflow.model.Source; +import com.google.cloud.dataflow.sdk.util.InstanceBuilder; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.common.worker.CustomSourceFormat; + +import java.util.Map; + +/** + * Creates {@code CustomSourceFormat} objects from {@code Source}. + */ +public class CustomSourceFormatFactory { + private CustomSourceFormatFactory() {} + + public static CustomSourceFormat create(Source source) throws Exception { + Map spec = source.getSpec(); + + try { + return InstanceBuilder.ofType(CustomSourceFormat.class) + .fromClassName(getString(spec, PropertyNames.OBJECT_TYPE_NAME)) + .build(); + + } catch (ClassNotFoundException exn) { + throw new Exception( + "unable to create a custom source format from " + source, exn); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java new file mode 100644 index 0000000000000..f2d41cfcbc45d --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java @@ -0,0 +1,121 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.runners.worker.DataflowWorker.buildStatus; +import static com.google.cloud.dataflow.sdk.runners.worker.DataflowWorker.uniqueId; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudDuration; +import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime; +import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; + +import com.google.api.services.dataflow.model.ApproximateProgress; +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.cloud.dataflow.sdk.options.DataflowWorkerHarnessOptions; +import com.google.cloud.dataflow.sdk.util.common.worker.WorkExecutor; +import com.google.cloud.dataflow.sdk.util.common.worker.WorkProgressUpdater; + +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.NotThreadSafe; + +/** + * 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); + + /** The Dataflow Worker WorkItem client */ + private final DataflowWorker.WorkUnitClient workUnitClient; + + /** The WorkItem for which work progress updates are sent. */ + private final WorkItem workItem; + + /** Options specifying information about the pipeline run by the worker.*/ + private final DataflowWorkerHarnessOptions options; + + public DataflowWorkProgressUpdater( + WorkItem workItem, + WorkExecutor worker, + DataflowWorker.WorkUnitClient workUnitClient, + DataflowWorkerHarnessOptions options) { + super(worker); + this.workItem = workItem; + this.workUnitClient = workUnitClient; + this.options = options; + } + + @Override + protected String workString() { + return uniqueId(workItem); + } + + @Override + protected long getWorkUnitLeaseExpirationTimestamp() { + return getLeaseExpirationTimestamp(workItem); + } + + @Override + protected void reportProgressHelper() throws Exception { + WorkItemStatus status = buildStatus( + workItem, false /*completed*/, + worker.getOutputCounters(), worker.getOutputMetrics(), options, + worker.getWorkerProgress(), stopPositionToService, + null /*sourceOperationResponse*/, null /*errors*/); + status.setRequestedLeaseDuration(toCloudDuration(Duration.millis(requestedLeaseDurationMs))); + + WorkItemServiceState result = workUnitClient.reportWorkItemStatus(status); + if (result != null) { + // Resets state after a successful progress report. + stopPositionToService = null; + + progressReportIntervalMs = nextProgressReportInterval( + fromCloudDuration(workItem.getReportStatusInterval()).getMillis(), + leaseRemainingTime(getLeaseExpirationTimestamp(result))); + + ApproximateProgress suggestedStopPoint = result.getSuggestedStopPoint(); + if (suggestedStopPoint == null && result.getSuggestedStopPosition() != null) { + suggestedStopPoint = new ApproximateProgress() + .setPosition(result.getSuggestedStopPosition()); + } + + if (suggestedStopPoint != null) { + LOG.info("Proposing stop progress on work unit {} at proposed stopping point {}", + workString(), suggestedStopPoint); + stopPositionToService = + worker.proposeStopPosition( + cloudProgressToSourceProgress(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/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java new file mode 100644 index 0000000000000..5175d15aa882e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java @@ -0,0 +1,330 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudSourceOperationResponseToSourceOperationResponse; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceOperationResponseToCloudSourceOperationResponse; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; + +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.api.services.dataflow.model.Status; +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.cloud.dataflow.sdk.options.DataflowWorkerHarnessOptions; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudCounterUtils; +import com.google.cloud.dataflow.sdk.util.CloudMetricUtils; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.UserCodeException; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.Metric; +import com.google.cloud.dataflow.sdk.util.common.worker.CustomSourceFormat; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.WorkExecutor; +import com.google.cloud.dataflow.sdk.util.common.worker.WorkProgressUpdater; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * This is a semi-abstract harness for executing WorkItem tasks in + * Java workers. Concrete implementations need to implement a + * WorkUnitClient. + * + *

DataflowWorker presents one public interface, + * getAndPerformWork(), which uses the WorkUnitClient to get work, + * execute it, and update the work. + */ +public class DataflowWorker { + + private static final Logger LOG = LoggerFactory.getLogger(DataflowWorker.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; + + public DataflowWorker(WorkUnitClient workUnitClient, + DataflowWorkerHarnessOptions options) { + this.workUnitClient = workUnitClient; + this.options = options; + } + + /** + * Gets WorkItem and performs it; returns true if work was + * successfully completed. + * + * getAndPerformWork may throw if there is a failure of the + * WorkUnitClient. + */ + public boolean getAndPerformWork() throws IOException { + WorkItem work = workUnitClient.getWorkItem(); + if (work == null) { + return false; + } + return doWork(work); + } + + /** + * Performs the given work; returns true if successful. + * + * @throws IOException Only if the WorkUnitClient fails. + */ + private boolean doWork(WorkItem workItem) throws IOException { + LOG.info("Executing: {}", workItem); + + WorkExecutor worker = null; + try { + // Populate PipelineOptions with data from work unit. + options.setProject(workItem.getProjectId()); + + ExecutionContext executionContext = new BatchModeExecutionContext(); + + if (workItem.getMapTask() != null) { + worker = MapTaskExecutorFactory.create(options, + workItem.getMapTask(), + executionContext); + + } else if (workItem.getSourceOperationTask() != null) { + worker = SourceOperationExecutorFactory.create( + workItem.getSourceOperationTask()); + + } else { + throw new RuntimeException("unknown kind of work item: " + workItem.toString()); + } + + WorkProgressUpdater progressUpdater = new DataflowWorkProgressUpdater( + workItem, worker, workUnitClient, options); + progressUpdater.startReportingProgress(); + + // Blocks while executing the work. + // TODO: refactor to allow multiple work unit + // processing threads. + worker.execute(); + + // Log all counter values for debugging purposes. + CounterSet counters = worker.getOutputCounters(); + for (Counter counter : counters) { + LOG.info("COUNTER {}.", counter); + } + + // Log all metrics for debugging purposes. + Collection> metrics = worker.getOutputMetrics(); + for (Metric metric : metrics) { + LOG.info("METRIC {}: {}", metric.getName(), metric.getValue()); + } + + // stopReportingProgress can throw an exception if the final progress + // update fails. For correctness, the task must then be marked as failed. + progressUpdater.stopReportingProgress(); + + // Report job success. + + // TODO: Find out a generic way for the WorkExecutor to report work-specific results + // into the work update. + CustomSourceFormat.SourceOperationResponse sourceOperationResponse = + (worker instanceof SourceOperationExecutor) + ? cloudSourceOperationResponseToSourceOperationResponse( + ((SourceOperationExecutor) worker).getResponse()) + : null; + reportStatus(options, "Success", workItem, counters, metrics, sourceOperationResponse, + null /*errors*/); + + return true; + + } catch (Throwable e) { + handleWorkError(workItem, worker, e); + return false; + + } finally { + if (worker != null) { + try { + worker.close(); + } catch (Exception exn) { + LOG.warn("Uncaught exception occurred during work unit shutdown:", exn); + } + } + } + } + + /** Handles the exception thrown when reading and executing the work. */ + private void handleWorkError( + WorkItem workItem, WorkExecutor worker, Throwable e) + throws IOException { + LOG.warn("Uncaught exception occurred during work unit execution:", e); + + // TODO: Look into moving the stack trace thinning + // into the client. + Throwable t = e instanceof UserCodeException ? e.getCause() : e; + Status error = new Status(); + error.setCode(2); // Code.UNKNOWN. TODO: Replace with a generated definition. + // TODO: Attach the stack trace as exception details, not to the message. + error.setMessage(buildCloudStackTrace(t)); + + reportStatus(options, "Failure", workItem, + worker == null ? null : worker.getOutputCounters(), + worker == null ? null : worker.getOutputMetrics(), + null /*sourceOperationResponse*/, + error == null ? null : Collections.singletonList(error)); + } + + /** + * Recursively goes through an exception, pulling out the stack trace. If the + * exception is a chained exception, it recursively goes through any causes + * and appends them to the stack trace. + */ + private static String buildCloudStackTrace(Throwable t) { + StringWriter result = new StringWriter(); + PrintWriter printResult = new PrintWriter(result); + + printResult.print("Exception: "); + for (;;) { + printResult.println(t.toString()); + for (StackTraceElement frame : t.getStackTrace()) { + printResult.println(frame.toString()); + } + t = t.getCause(); + if (t == null) { + break; + } + printResult.print("Caused by: "); + } + return result.toString(); + } + + private void reportStatus(DataflowWorkerHarnessOptions options, + String status, + WorkItem workItem, + @Nullable CounterSet counters, + @Nullable Collection> metrics, + @Nullable CustomSourceFormat.SourceOperationResponse + sourceOperationResponse, + @Nullable List errors) + throws IOException { + LOG.info("{} processing work item {}", status, uniqueId(workItem)); + WorkItemStatus workItemStatus = buildStatus(workItem, true /*completed*/, + counters, metrics, options, null, null, sourceOperationResponse, errors); + workUnitClient.reportWorkItemStatus(workItemStatus); + } + + static WorkItemStatus buildStatus( + WorkItem workItem, + boolean completed, + @Nullable CounterSet counters, + @Nullable Collection> metrics, + DataflowWorkerHarnessOptions options, + @Nullable Source.Progress progress, + @Nullable Source.Position stopPosition, + @Nullable CustomSourceFormat.SourceOperationResponse sourceOperationResponse, + @Nullable List errors) { + WorkItemStatus status = new WorkItemStatus(); + status.setWorkItemId(Long.toString(workItem.getId())); + status.setCompleted(completed); + + List counterUpdates = null; + List metricUpdates = null; + + if (counters != null) { + // Currently we lack a reliable exactly-once delivery mechanism for + // work updates, i.e. they can be retried or reordered, so sending + // delta updates could lead to double-counted or missed contributions. + // However, delta updates may be beneficial for performance. + // TODO: Implement exactly-once delivery and use deltas, + // if it ever becomes clear that deltas are necessary for performance. + boolean delta = false; + counterUpdates = CloudCounterUtils.extractCounters(counters, delta); + } + if (metrics != null) { + metricUpdates = CloudMetricUtils.extractCloudMetrics(metrics, options.getWorkerId()); + } + List updates = null; + if (counterUpdates == null) { + updates = metricUpdates; + } else if (metrics == null) { + updates = counterUpdates; + } else { + updates = new ArrayList<>(); + updates.addAll(counterUpdates); + updates.addAll(metricUpdates); + } + status.setMetricUpdates(updates); + + // TODO: Provide more structure representation of error, + // e.g., the serialized exception object. + if (errors != null) { + status.setErrors(errors); + } + + if (progress != null) { + status.setProgress(sourceProgressToCloudProgress(progress)); + } + if (stopPosition != null) { + status.setStopPosition(sourcePositionToCloudPosition(stopPosition)); + } + + if (workItem.getSourceOperationTask() != null) { + status.setSourceOperationResponse( + sourceOperationResponseToCloudSourceOperationResponse(sourceOperationResponse)); + } + + return status; + } + + static String uniqueId(WorkItem work) { + return work.getProjectId() + ";" + work.getJobId() + ";" + work.getId(); + } + + /** + * Abstract base class describing a client for WorkItem work units. + */ + public abstract static class WorkUnitClient { + /** + * Returns a new WorkItem unit for this Worker to work on or null + * if no work item is available. + */ + public abstract WorkItem getWorkItem() throws IOException; + + /** + * Reports a {@link WorkItemStatus} for an assigned {@link WorkItem}. + * + * @param workItemStatus the status to report + * @return a {@link WorkServiceState} (e.g. a new stop position) + */ + public abstract WorkItemServiceState reportWorkItemStatus( + WorkItemStatus workItemStatus) + throws IOException; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java new file mode 100644 index 0000000000000..fa17cf67390d0 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java @@ -0,0 +1,231 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; +import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudTime; + +import com.google.api.client.util.Preconditions; +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.cloud.dataflow.sdk.options.DataflowWorkerHarnessOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingFormatter; +import com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingInitializer; +import com.google.cloud.dataflow.sdk.util.Credentials; +import com.google.cloud.dataflow.sdk.util.GcsIOChannelFactory; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.Transport; +import com.google.common.collect.ImmutableList; + +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.slf4j.MDC; + +import java.io.IOException; +import java.lang.Thread.UncaughtExceptionHandler; +import java.util.Collections; +import java.util.List; + +import javax.annotation.concurrent.ThreadSafe; + +/** + * This is a harness for executing WorkItem tasks in Java workers. + *

+ * The worker fetches WorkItem units from the Dataflow Service. + * When the work is complete, the program sends results via the worker service API. + *

+ * Returns status code 0 on successful completion, 1 on any uncaught failures. + *

+ * TODO: add support for VM initialization via config. + * During initialization, we should take a configuration which specifies + * an initialization function, allowing user code to run on VM startup. + */ +public class DataflowWorkerHarness { + private static final Logger LOG = LoggerFactory.getLogger(DataflowWorkerHarness.class); + + private static final String APPLICATION_NAME = "DataflowWorkerHarness"; + + /** + * This uncaught exception handler logs the {@link Throwable} to the logger, {@link System#err} + * and exits the application with status code 1. + */ + static class WorkerUncaughtExceptionHandler implements UncaughtExceptionHandler { + static final WorkerUncaughtExceptionHandler INSTANCE = new WorkerUncaughtExceptionHandler(); + + @Override + public void uncaughtException(Thread t, Throwable e) { + LOG.error("Uncaught exception in main thread. Exiting with status code 1.", e); + System.err.println("Uncaught exception in main thread. Exiting with status code 1."); + e.printStackTrace(); + System.exit(1); + } + } + + /** + * Fetches and processes work units from the Dataflow service. + */ + public static void main(String[] args) throws Exception { + Thread.currentThread().setUncaughtExceptionHandler(WorkerUncaughtExceptionHandler.INSTANCE); + new DataflowWorkerLoggingInitializer().initialize(); + + DataflowWorker worker = createFromSystemProperties(); + processWork(worker); + } + + // Visible for testing. + static void processWork(DataflowWorker worker) throws IOException { + worker.getAndPerformWork(); + } + + static DataflowWorker createFromSystemProperties() { + return create(PipelineOptionsFactory.createFromSystemProperties()); + } + + static DataflowWorker create(DataflowWorkerHarnessOptions options) { + MDC.put(DataflowWorkerLoggingFormatter.MDC_DATAFLOW_JOB_ID, options.getJobId()); + MDC.put(DataflowWorkerLoggingFormatter.MDC_DATAFLOW_WORKER_ID, options.getWorkerId()); + options.setAppName(APPLICATION_NAME); + + if (options.getGcpCredential() == null) { + try { + // Load the worker credential, otherwise the default is to load user + // credentials. + options.setGcpCredential(Credentials.getWorkerCredential(options)); + Preconditions.checkState(options.getGcpCredential() != null, + "Failed to obtain worker credential"); + } catch (Throwable e) { + LOG.warn("Unable to obtain any valid credentials. Worker inoperable.", e); + return null; + } + } + + // Configure standard IO factories. + IOChannelUtils.setIOFactory("gs", new GcsIOChannelFactory(options)); + + DataflowWorkUnitClient client = DataflowWorkUnitClient.fromOptions(options); + return new DataflowWorker(client, options); + } + + /** + * A Dataflow WorkUnit client that fetches WorkItems from the Dataflow service. + */ + @ThreadSafe + static class DataflowWorkUnitClient extends DataflowWorker.WorkUnitClient { + private final Dataflow dataflow; + private final DataflowWorkerHarnessOptions options; + + /** + * Creates a client that fetches WorkItems from the Dataflow service. + * + * @param options The pipeline options. + * @return A WorkItemClient that fetches WorkItems from the Dataflow service. + */ + static DataflowWorkUnitClient fromOptions(DataflowWorkerHarnessOptions options) { + return new DataflowWorkUnitClient( + Transport.newDataflowClient(options).build(), + options); + } + + /** + * Package private constructor for testing. + */ + DataflowWorkUnitClient(Dataflow dataflow, DataflowWorkerHarnessOptions options) { + this.dataflow = dataflow; + this.options = options; + } + + /** + * Gets a WorkItem from the Dataflow service. + */ + @Override + public WorkItem getWorkItem() throws IOException { + LeaseWorkItemRequest request = new LeaseWorkItemRequest(); + request.setFactory(Transport.getJsonFactory()); + request.setWorkItemTypes(ImmutableList.of( + "map_task", "seq_map_task", "remote_source_task")); + // All remote sources require the "remote_source" capability. Dataflow's + // custom sources are further tagged with the format "custom_source". + request.setWorkerCapabilities(ImmutableList.of( + options.getWorkerId(), "remote_source", PropertyNames.CUSTOM_SOURCE_FORMAT)); + 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.standardSeconds(60))); + + LOG.debug("Leasing work: {}", request); + + LeaseWorkItemResponse response = dataflow.v1b3().projects().jobs().workItems().lease( + options.getProject(), options.getJobId(), request).execute(); + LOG.debug("Lease work response: {}", response); + + List workItems = response.getWorkItems(); + if (workItems == null || workItems.isEmpty()) { + // We didn't lease any work + return null; + } 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); + if (work == null || work.getId() == null) { + return null; + } + + MDC.put(DataflowWorkerLoggingFormatter.MDC_DATAFLOW_WORK_ID, Long.toString(work.getId())); + // Looks like the work's a'ight. + return work; + } + + @Override + public WorkItemServiceState reportWorkItemStatus(WorkItemStatus workItemStatus) + throws IOException { + workItemStatus.setFactory(Transport.getJsonFactory()); + LOG.debug("Reporting work status: {}", workItemStatus); + ReportWorkItemStatusResponse result = + dataflow.v1b3().projects().jobs().workItems().reportStatus( + options.getProject(), options.getJobId(), + new ReportWorkItemStatusRequest() + .setWorkerId(options.getWorkerId()) + .setWorkItemStatuses(Collections.singletonList(workItemStatus)) + .setCurrentWorkerTime(toCloudTime(DateTime.now()))) + .execute(); + if (result == null || result.getWorkItemServiceStates() == null + || result.getWorkItemServiceStates().size() != 1) { + throw new IOException( + "This version of the SDK expects exactly one work item service state from the service"); + } + WorkItemServiceState state = result.getWorkItemServiceStates().get(0); + LOG.debug("ReportWorkItemStatus result: {}", state); + return state; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedSource.java new file mode 100644 index 0000000000000..beea88747c1cf --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedSource.java @@ -0,0 +1,259 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Preconditions.checkNotNull; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToSourcePosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; + +import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.IOChannelFactory; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.ProgressTracker; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.PushbackInputStream; +import java.nio.channels.Channels; +import java.util.Collection; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +/** + * Abstract base class for sources that read from files. + * + * @param the type of the elements read from the source + */ +public abstract class FileBasedSource extends Source { + protected static final int BUF_SIZE = 200; + protected final String filename; + @Nullable protected final Long startPosition; + @Nullable protected final Long endPosition; + protected final Coder coder; + protected final boolean useDefaultBufferSize; + + private static final Logger LOG = LoggerFactory.getLogger(FileBasedSource.class); + + protected FileBasedSource(String filename, + @Nullable Long startPosition, + @Nullable Long endPosition, + Coder coder, + boolean useDefaultBufferSize) { + this.filename = filename; + this.startPosition = startPosition; + this.endPosition = endPosition; + this.coder = coder; + this.useDefaultBufferSize = useDefaultBufferSize; + } + + /** + * Returns a new iterator for elements in the given range in the + * given file. If the range starts in the middle an element, this + * element is skipped as it is considered part of the previous + * range; if the last element that starts in the range finishes + * beyond the end position, it is still considered part of this + * range. In other words, the start position and the end position + * are "rounded up" to element boundaries. + * + * @param endPosition offset of the end position; null means end-of-file + */ + protected abstract SourceIterator newSourceIteratorForRangeInFile( + IOChannelFactory factory, String oneFile, long startPosition, + @Nullable Long endPosition) + throws IOException; + + /** + * Returns a new iterator for elements in the given files. Caller + * must ensure that the file collection is not empty. + */ + protected abstract SourceIterator newSourceIteratorForFiles( + IOChannelFactory factory, Collection files) throws IOException; + + @Override + public SourceIterator iterator() throws IOException { + IOChannelFactory factory = IOChannelUtils.getFactory(filename); + Collection inputs = factory.match(filename); + if (inputs.isEmpty()) { + throw new IOException("No match for file pattern '" + filename + "'"); + } + + if (startPosition != null || endPosition != null) { + if (inputs.size() != 1) { + throw new UnsupportedOperationException( + "Unable to apply range limits to multiple-input stream: " + + filename); + } + + return newSourceIteratorForRangeInFile( + factory, inputs.iterator().next(), + startPosition == null ? 0 : startPosition, endPosition); + } else { + return newSourceIteratorForFiles(factory, inputs); + } + } + + /** + * Abstract base class for file-based source iterators. + */ + protected abstract class FileBasedIterator extends AbstractSourceIterator { + protected final CopyableSeekableByteChannel seeker; + protected final PushbackInputStream stream; + protected final Long startOffset; + protected Long endOffset; + protected final ProgressTracker tracker; + protected ByteArrayOutputStream nextElement; + protected boolean nextElementComputed = false; + protected long offset; + + FileBasedIterator(CopyableSeekableByteChannel seeker, + long startOffset, + long offset, + @Nullable Long endOffset, + ProgressTracker tracker) throws IOException { + this.seeker = checkNotNull(seeker); + this.seeker.position(startOffset); + BufferedInputStream bufferedStream = useDefaultBufferSize + ? new BufferedInputStream(Channels.newInputStream(seeker)) + : new BufferedInputStream(Channels.newInputStream(seeker), BUF_SIZE); + this.stream = new PushbackInputStream(bufferedStream, BUF_SIZE); + this.startOffset = startOffset; + this.offset = offset; + this.endOffset = endOffset; + this.tracker = checkNotNull(tracker); + } + + /** + * Reads the next element. + * + * @return a {@code ByteArrayOutputStream} containing the contents + * of the element, or {@code null} if the end of the stream + * has been reached. + * @throws IOException if an I/O error occurs + */ + protected abstract ByteArrayOutputStream readElement() + throws IOException; + + @Override + public boolean hasNext() throws IOException { + computeNextElement(); + return nextElement != null; + } + + @Override + public T next() throws IOException { + advance(); + return CoderUtils.decodeFromByteArray(coder, nextElement.toByteArray()); + } + + void advance() throws IOException { + computeNextElement(); + if (nextElement == null) { + throw new NoSuchElementException(); + } + nextElementComputed = false; + } + + @Override + public Progress getProgress() { + // Currently we assume that only a offset position is reported as + // current progress. Source writer can override this method to update + // other metrics, e.g. completion percentage or remaining time. + com.google.api.services.dataflow.model.Position currentPosition = + new com.google.api.services.dataflow.model.Position(); + currentPosition.setByteOffset(offset); + + ApproximateProgress progress = new ApproximateProgress(); + progress.setPosition(currentPosition); + + return cloudProgressToSourceProgress(progress); + } + + @Override + public Position updateStopPosition(Progress proposedStopPosition) { + checkNotNull(proposedStopPosition); + + // Currently we only support stop position in byte offset of + // CloudPosition in a file-based Source. If stop position in + // other types is proposed, the end position in iterator will + // not be updated, and return null. + com.google.api.services.dataflow.model.ApproximateProgress stopPosition = + sourceProgressToCloudProgress(proposedStopPosition); + if (stopPosition == null) { + LOG.warn( + "A stop position other than CloudPosition is not supported now."); + return null; + } + + Long byteOffset = stopPosition.getPosition().getByteOffset(); + if (byteOffset == null) { + LOG.warn( + "A stop position other than byte offset is not supported in a " + + "file-based Source."); + return null; + } + if (byteOffset <= offset) { + // Proposed stop position is not after the current position: + // No stop position update. + return null; + } + + if (endOffset != null && byteOffset >= endOffset) { + // Proposed stop position is after the current stop (end) position: No + // stop position update. + return null; + } + + this.endOffset = byteOffset; + return cloudPositionToSourcePosition(stopPosition.getPosition()); + } + + /** + * Returns the end offset of the iterator. + * The method is called for test ONLY. + */ + Long getEndOffset() { + return this.endOffset; + } + + @Override + public void close() throws IOException { + stream.close(); + } + + private void computeNextElement() throws IOException { + if (nextElementComputed) { + return; + } + + if (endOffset == null || offset < endOffset) { + nextElement = readElement(); + } else { + nextElement = null; + } + nextElementComputed = true; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java new file mode 100644 index 0000000000000..adf0435e6e981 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java @@ -0,0 +1,119 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getBytes; +import static com.google.cloud.dataflow.sdk.util.Structs.getObject; + +import com.google.api.services.dataflow.model.MultiOutputInfo; +import com.google.api.services.dataflow.model.SideInputInfo; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PTuple; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.util.Serializer; +import com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn; +import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import javax.annotation.Nullable; + +/** + * A wrapper around a GroupAlsoByWindowsDoFn. This class is the same as + * NormalParDoFn, except that it gets deserialized differently. + */ +class GroupAlsoByWindowsParDoFn extends NormalParDoFn { + public static GroupAlsoByWindowsParDoFn create( + PipelineOptions options, + CloudObject cloudUserFn, + String stepName, + @Nullable List sideInputInfos, + @Nullable List multiOutputInfos, + Integer numOutputs, + ExecutionContext executionContext, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler sampler /* unused */) + throws Exception { + Object windowingFn = + SerializableUtils.deserializeFromByteArray( + getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), + "serialized window fn"); + if (!(windowingFn instanceof WindowingFn)) { + throw new Exception( + "unexpected kind of WindowingFn: " + windowingFn.getClass().getName()); + } + + byte[] serializedCombineFn = getBytes(cloudUserFn, PropertyNames.COMBINE_FN, null); + Object combineFn = null; + if (serializedCombineFn != null) { + combineFn = + SerializableUtils.deserializeFromByteArray(serializedCombineFn, "serialized combine fn"); + if (!(combineFn instanceof KeyedCombineFn)) { + throw new Exception("unexpected kind of KeyedCombineFn: " + combineFn.getClass().getName()); + } + } + + Map inputCoderObject = getObject(cloudUserFn, PropertyNames.INPUT_CODER); + + Coder inputCoder = Serializer.deserialize(inputCoderObject, Coder.class); + if (!(inputCoder instanceof WindowedValueCoder)) { + throw new Exception( + "Expected WindowedValueCoder for inputCoder, got: " + + inputCoder.getClass().getName()); + } + Coder elemCoder = ((WindowedValueCoder) inputCoder).getValueCoder(); + if (!(elemCoder instanceof KvCoder)) { + throw new Exception( + "Expected KvCoder for inputCoder, got: " + elemCoder.getClass().getName()); + } + + DoFn windowingDoFn = StreamingGroupAlsoByWindowsDoFn.create( + (WindowingFn) windowingFn, + ((KvCoder) elemCoder).getValueCoder()); + + return new GroupAlsoByWindowsParDoFn( + options, windowingDoFn, stepName, executionContext, addCounterMutator); + } + + private GroupAlsoByWindowsParDoFn( + PipelineOptions options, + DoFn fn, + String stepName, + ExecutionContext executionContext, + CounterSet.AddCounterMutator addCounterMutator) { + super( + options, + fn, + PTuple.empty(), + Arrays.asList("output"), + stepName, + executionContext, + addCounterMutator); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java new file mode 100644 index 0000000000000..2d168879a21b7 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java @@ -0,0 +1,368 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Preconditions.checkNotNull; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToSourcePosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; + +import com.google.api.client.util.Preconditions; + +import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.Reiterable; +import com.google.cloud.dataflow.sdk.util.common.Reiterator; +import com.google.cloud.dataflow.sdk.util.common.worker.BatchingShuffleEntryReader; +import com.google.cloud.dataflow.sdk.util.common.worker.GroupingShuffleEntryIterator; +import com.google.cloud.dataflow.sdk.util.common.worker.KeyGroupedShuffleEntries; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntryReader; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.values.KV; + +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Iterator; +import java.util.NoSuchElementException; + +/** + * A source that reads from a shuffled dataset and yields key-grouped data. + * + * @param the type of the keys read from the shuffle + * @param the type of the values read from the shuffle + */ +public class GroupingShuffleSource + extends Source>>> { + private static final Logger LOG = + LoggerFactory.getLogger(GroupingShuffleSource.class); + + final byte[] shuffleReaderConfig; + final String startShufflePosition; + final String stopShufflePosition; + final BatchModeExecutionContext executionContext; + + Coder keyCoder; + Coder valueCoder; + + public GroupingShuffleSource(PipelineOptions options, + byte[] shuffleReaderConfig, + String startShufflePosition, + String stopShufflePosition, + Coder>>> coder, + BatchModeExecutionContext executionContext) + throws Exception { + this.shuffleReaderConfig = shuffleReaderConfig; + this.startShufflePosition = startShufflePosition; + this.stopShufflePosition = stopShufflePosition; + this.executionContext = executionContext; + initCoder(coder); + } + + @Override + public SourceIterator>>> iterator() + throws IOException { + Preconditions.checkArgument(shuffleReaderConfig != null); + return iterator(new BatchingShuffleEntryReader( + new ChunkingShuffleBatchReader(new ApplianceShuffleReader( + shuffleReaderConfig)))); + } + + private void initCoder(Coder>>> coder) throws Exception { + if (!(coder instanceof WindowedValueCoder)) { + throw new Exception( + "unexpected kind of coder for WindowedValue: " + coder); + } + Coder>> elemCoder = + ((WindowedValueCoder>>) coder).getValueCoder(); + if (!(elemCoder instanceof KvCoder)) { + throw new Exception( + "unexpected kind of coder for elements read from " + + "a key-grouping shuffle: " + elemCoder); + } + KvCoder> kvCoder = (KvCoder>) elemCoder; + this.keyCoder = kvCoder.getKeyCoder(); + Coder> kvValueCoder = kvCoder.getValueCoder(); + if (!(kvValueCoder instanceof IterableCoder)) { + throw new Exception( + "unexpected kind of coder for values of KVs read from " + + "a key-grouping shuffle"); + } + IterableCoder iterCoder = (IterableCoder) kvValueCoder; + this.valueCoder = iterCoder.getElemCoder(); + } + + final SourceIterator>>> iterator(ShuffleEntryReader reader) + throws IOException { + return new GroupingShuffleSourceIterator(reader); + } + + /** + * A SourceIterator that reads from a ShuffleEntryReader and groups + * all the values with the same key. + * + *

A key limitation of this implementation is that all iterator accesses + * must by externally synchronized (the iterator objects are not individually + * thread-safe, and the iterators derived from a single original iterator + * access shared state which is not thread-safe). + * + *

To access the current position, the iterator must advance + * on-demand and cache the next batch of key grouped shuffle + * entries. The iterator does not advance a second time in @next() + * to avoid asking the underlying iterator to advance to the next + * key before the caller/user iterates over the values corresponding + * to the current key -- which would introduce a performance + * penalty. + */ + private final class GroupingShuffleSourceIterator + extends AbstractSourceIterator>>> { + // N.B. This class is *not* static; it uses the keyCoder, valueCoder, and + // executionContext from its enclosing GroupingShuffleSource. + + /** The iterator over shuffle entries, grouped by common key. */ + private final Iterator groups; + + /** The stop position. No records with a position at or after + * @stopPosition will be returned. Initialized + * to @AbstractShuffleSource.stopShufflePosition but can be + * dynamically updated via @updateStopPosition() (note that such + * updates can only decrease @stopPosition). + * + *

The granularity of the stop position is such that it can + * only refer to records at the boundary of a key. + */ + private ByteArrayShufflePosition stopPosition = null; + + /** The next group to be consumed, if available */ + private KeyGroupedShuffleEntries nextGroup = null; + + public GroupingShuffleSourceIterator(ShuffleEntryReader reader) { + stopPosition = ByteArrayShufflePosition.fromBase64(stopShufflePosition); + this.groups = + new GroupingShuffleEntryIterator(reader.read( + ByteArrayShufflePosition.fromBase64(startShufflePosition), + stopPosition)) { + @Override + protected void notifyElementRead(long byteSize) { + GroupingShuffleSource.this.notifyElementRead(byteSize); + } + }; + } + + private void advanceIfNecessary() { + if (nextGroup == null && groups.hasNext()) { + nextGroup = groups.next(); + } + } + + @Override + public boolean hasNext() throws IOException { + return hasNextInternal(); + } + + /** + * Returns false if the next group does not exist (i.e., no more + * records available) or the group is beyond @stopPosition. + */ + private boolean hasNextInternal() { + advanceIfNecessary(); + if (nextGroup == null) { + return false; + } + ByteArrayShufflePosition current = + ByteArrayShufflePosition.of(nextGroup.position); + return stopPosition == null || current.compareTo(stopPosition) < 0; + } + + @Override + public WindowedValue>> next() throws IOException { + if (!hasNext()) { + throw new NoSuchElementException(); + } + KeyGroupedShuffleEntries group = nextGroup; + nextGroup = null; + + K key = CoderUtils.decodeFromByteArray(keyCoder, group.key); + if (executionContext != null) { + executionContext.setKey(key); + } + + return WindowedValue.valueInEmptyWindows( + KV.>of(key, new ValuesIterable(group.values))); + } + + /** + * Returns the position before the next {@code KV>} to be returned by the + * {@link GroupingShuffleSourceIterator}. Returns null if the + * {@link GroupingShuffleSourceIterator} is finished. + */ + @Override + public Progress getProgress() { + com.google.api.services.dataflow.model.Position currentPosition = + new com.google.api.services.dataflow.model.Position(); + ApproximateProgress progress = new ApproximateProgress(); + if (hasNextInternal()) { + ByteArrayShufflePosition current = + ByteArrayShufflePosition.of(nextGroup.position); + currentPosition.setShufflePosition(current.encodeBase64()); + } else { + if (stopPosition != null) { + currentPosition.setShufflePosition(stopPosition.encodeBase64()); + } else { + // The original stop position described the end of the + // shuffle-position-space (or infinity) and all records have + // been consumed. + progress.setPercentComplete((float) 1.0); + progress.setRemainingTime(toCloudDuration(Duration.ZERO)); + return cloudProgressToSourceProgress(progress); + } + } + + progress.setPosition(currentPosition); + return cloudProgressToSourceProgress(progress); + } + + /** + * Updates the stop position of the shuffle source to the position proposed. Ignores the + * proposed stop position if it is smaller than or equal to the position before the next + * {@code KV>} to be returned by the {@link GroupingShuffleSourceIterator}. + */ + @Override + public Position updateStopPosition(Progress proposedStopPosition) { + checkNotNull(proposedStopPosition); + com.google.api.services.dataflow.model.Position stopCloudPosition = + sourceProgressToCloudProgress(proposedStopPosition).getPosition(); + if (stopCloudPosition == null) { + LOG.warn( + "A stop position other than a Position is not supported now."); + return null; + } + + if (stopCloudPosition.getShufflePosition() == null) { + LOG.warn( + "A stop position other than shuffle position is not supported in " + + "a grouping shuffle source: " + stopCloudPosition.toString()); + return null; + } + ByteArrayShufflePosition newStopPosition = + ByteArrayShufflePosition.fromBase64(stopCloudPosition.getShufflePosition()); + + if (!hasNextInternal()) { + LOG.warn("Cannot update stop position to " + + stopCloudPosition.getShufflePosition() + + " since all input was consumed."); + return null; + } + ByteArrayShufflePosition current = + ByteArrayShufflePosition.of(nextGroup.position); + if (newStopPosition.compareTo(current) <= 0) { + LOG.warn("Proposed stop position: " + + stopCloudPosition.getShufflePosition() + " <= current position: " + + current.encodeBase64()); + return null; + } + + if (this.stopPosition != null + && newStopPosition.compareTo(this.stopPosition) >= 0) { + LOG.warn("Proposed stop position: " + + stopCloudPosition.getShufflePosition() + + " >= current stop position: " + + this.stopPosition.encodeBase64()); + return null; + } + + this.stopPosition = newStopPosition; + LOG.info("Updated the stop position to " + + stopCloudPosition.getShufflePosition()); + + return cloudPositionToSourcePosition(stopCloudPosition); + } + + /** + * Provides the {@link Reiterable} used to iterate through the values part + * of a {@code KV>} entry produced by a + * {@link GroupingShuffleSource}. + */ + private final class ValuesIterable implements Reiterable { + // N.B. This class is *not* static; it uses the valueCoder from + // its enclosing GroupingShuffleSource. + + private final Reiterable base; + + public ValuesIterable(Reiterable base) { + this.base = checkNotNull(base); + } + + @Override + public ValuesIterator iterator() { + return new ValuesIterator(base.iterator()); + } + } + + /** + * Provides the {@link Reiterator} used to iterate through the values part + * of a {@code KV>} entry produced by a + * {@link GroupingShuffleSource}. + */ + private final class ValuesIterator implements Reiterator { + // N.B. This class is *not* static; it uses the valueCoder from + // its enclosing GroupingShuffleSource. + + private final Reiterator base; + + public ValuesIterator(Reiterator base) { + this.base = checkNotNull(base); + } + + @Override + public boolean hasNext() { + return base.hasNext(); + } + + @Override + public V next() { + ShuffleEntry entry = base.next(); + try { + return CoderUtils.decodeFromByteArray(valueCoder, entry.getValue()); + } catch (IOException exn) { + throw new RuntimeException(exn); + } + } + + @Override + public void remove() { + base.remove(); + } + + @Override + public ValuesIterator copy() { + return new ValuesIterator(base.copy()); + } + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceFactory.java new file mode 100644 index 0000000000000..2229a77ddc10b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceFactory.java @@ -0,0 +1,62 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Base64.decodeBase64; +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.values.KV; + +/** + * Creates a GroupingShuffleSource from a CloudObject spec. + */ +public class GroupingShuffleSourceFactory { + // Do not instantiate. + private GroupingShuffleSourceFactory() {} + + public static GroupingShuffleSource create( + PipelineOptions options, + CloudObject spec, + Coder>>> coder, + ExecutionContext executionContext) + throws Exception { + return create(options, spec, coder, + (BatchModeExecutionContext) executionContext); + } + + static GroupingShuffleSource create( + PipelineOptions options, + CloudObject spec, + Coder>>> coder, + BatchModeExecutionContext executionContext) + throws Exception { + return new GroupingShuffleSource<>( + options, + decodeBase64(getString(spec, PropertyNames.SHUFFLE_READER_CONFIG)), + getString(spec, PropertyNames.START_SHUFFLE_POSITION, null), + getString(spec, PropertyNames.END_SHUFFLE_POSITION, null), + coder, + executionContext); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySource.java new file mode 100644 index 0000000000000..a0a524ee0c9b9 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySource.java @@ -0,0 +1,163 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Preconditions.checkNotNull; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToSourcePosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static java.lang.Math.min; + +import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.StringUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +/** + * A source that yields a set of precomputed elements. + * + * @param the type of the elements read from the source + */ +public class InMemorySource extends Source { + private static final Logger LOG = LoggerFactory.getLogger(InMemorySource.class); + + final List encodedElements; + final int startIndex; + final int endIndex; + final Coder coder; + + public InMemorySource(List encodedElements, + @Nullable Long startIndex, + @Nullable Long endIndex, + Coder coder) { + this.encodedElements = encodedElements; + int maxIndex = encodedElements.size(); + if (startIndex == null) { + this.startIndex = 0; + } else { + if (startIndex < 0) { + throw new IllegalArgumentException("start index should be >= 0"); + } + this.startIndex = (int) min(startIndex, maxIndex); + } + if (endIndex == null) { + this.endIndex = maxIndex; + } else { + if (endIndex < this.startIndex) { + throw new IllegalArgumentException( + "end index should be >= start index"); + } + this.endIndex = (int) min(endIndex, maxIndex); + } + this.coder = coder; + } + + @Override + public SourceIterator iterator() throws IOException { + return new InMemorySourceIterator(); + } + + /** + * A SourceIterator that yields an in-memory list of elements. + */ + class InMemorySourceIterator extends AbstractSourceIterator { + int index; + int endPosition; + + public InMemorySourceIterator() { + index = startIndex; + endPosition = endIndex; + } + + @Override + public boolean hasNext() { + return index < endPosition; + } + + @Override + public T next() throws IOException { + if (!hasNext()) { + throw new NoSuchElementException(); + } + String encodedElementString = encodedElements.get(index++); + // TODO: Replace with the real encoding used by the + // front end, when we know what it is. + byte[] encodedElement = + StringUtils.jsonStringToByteArray(encodedElementString); + notifyElementRead(encodedElement.length); + return CoderUtils.decodeFromByteArray(coder, encodedElement); + } + + @Override + public Progress getProgress() { + // Currently we assume that only a record index position is reported as + // current progress. Source writer can override this method to update + // other metrics, e.g. completion percentage or remaining time. + com.google.api.services.dataflow.model.Position currentPosition = + new com.google.api.services.dataflow.model.Position(); + currentPosition.setRecordIndex((long) index); + + ApproximateProgress progress = new ApproximateProgress(); + progress.setPosition(currentPosition); + + return cloudProgressToSourceProgress(progress); + } + + @Override + public Position updateStopPosition(Progress proposedStopPosition) { + checkNotNull(proposedStopPosition); + + // Currently we only support stop position in record index of + // an API Position in InMemorySource. If stop position in other types is + // proposed, the end position in iterator will not be updated, + // and return null. + com.google.api.services.dataflow.model.Position stopPosition = + sourceProgressToCloudProgress(proposedStopPosition).getPosition(); + if (stopPosition == null) { + LOG.warn( + "A stop position other than a Dataflow API Position is not currently supported."); + return null; + } + + Long recordIndex = stopPosition.getRecordIndex(); + if (recordIndex == null) { + LOG.warn( + "A stop position other than record index is not supported in InMemorySource."); + return null; + } + if (recordIndex <= index || recordIndex >= endPosition) { + // Proposed stop position is not after the current position or proposed + // stop position is after the current stop (end) position: No stop + // position update. + return null; + } + + this.endPosition = recordIndex.intValue(); + return cloudPositionToSourcePosition(stopPosition); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactory.java new file mode 100644 index 0000000000000..3f2cd9c9a1dba --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactory.java @@ -0,0 +1,54 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getLong; +import static com.google.cloud.dataflow.sdk.util.Structs.getStrings; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; + +import java.util.Collections; + +/** + * Creates an InMemorySource from a CloudObject spec. + */ +public class InMemorySourceFactory { + // Do not instantiate. + private InMemorySourceFactory() {} + + public static InMemorySource create(PipelineOptions options, + CloudObject spec, + Coder coder, + ExecutionContext executionContext) + throws Exception { + return create(spec, coder); + } + + static InMemorySource create(CloudObject spec, + Coder coder) throws Exception { + return new InMemorySource<>( + getStrings(spec, + PropertyNames.ELEMENTS, Collections.emptyList()), + getLong(spec, PropertyNames.START_INDEX, null), + getLong(spec, PropertyNames.END_INDEX, null), + coder); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/LazyMultiSourceIterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/LazyMultiSourceIterator.java new file mode 100644 index 0000000000000..3ccebd5617565 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/LazyMultiSourceIterator.java @@ -0,0 +1,87 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import java.io.IOException; +import java.util.Iterator; +import java.util.NoSuchElementException; + +/** + * Implements a SourceIterator over a collection of inputs. + * + * The sources are used sequentially, each consumed entirely before moving + * to the next source. + * + * The input is lazily constructed by using the abstract method {@code open} to + * create a source iterator for inputs on demand. This allows the resources to + * be produced lazily, as an open source iterator may consume process resources + * such as file descriptors. + */ +abstract class LazyMultiSourceIterator + extends Source.AbstractSourceIterator { + private final Iterator inputs; + Source.SourceIterator current; + + public LazyMultiSourceIterator(Iterator inputs) { + this.inputs = inputs; + } + + @Override + public boolean hasNext() throws IOException { + while (selectSource()) { + if (!current.hasNext()) { + current.close(); + current = null; + } else { + return true; + } + } + return false; + } + + @Override + public T next() throws IOException { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return current.next(); + } + + @Override + public void close() throws IOException { + while (selectSource()) { + current.close(); + current = null; + } + } + + protected abstract Source.SourceIterator open(String input) + throws IOException; + + boolean selectSource() throws IOException { + if (current != null) { + return true; + } + if (inputs.hasNext()) { + current = open(inputs.next()); + return true; + } + return false; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java new file mode 100644 index 0000000000000..095aa0876ee8e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java @@ -0,0 +1,413 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.api.services.dataflow.model.FlattenInstruction; +import com.google.api.services.dataflow.model.InstructionInput; +import com.google.api.services.dataflow.model.InstructionOutput; +import com.google.api.services.dataflow.model.MapTask; +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.WriteInstruction; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.Serializer; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObservable; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; +import com.google.cloud.dataflow.sdk.util.common.worker.FlattenOperation; +import com.google.cloud.dataflow.sdk.util.common.worker.MapTaskExecutor; +import com.google.cloud.dataflow.sdk.util.common.worker.Operation; +import com.google.cloud.dataflow.sdk.util.common.worker.OutputReceiver; +import com.google.cloud.dataflow.sdk.util.common.worker.ParDoFn; +import com.google.cloud.dataflow.sdk.util.common.worker.ParDoOperation; +import com.google.cloud.dataflow.sdk.util.common.worker.PartialGroupByKeyOperation; +import com.google.cloud.dataflow.sdk.util.common.worker.ReadOperation; +import com.google.cloud.dataflow.sdk.util.common.worker.ReceivingOperation; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; +import com.google.cloud.dataflow.sdk.util.common.worker.WriteOperation; +import com.google.cloud.dataflow.sdk.values.KV; + +import java.util.ArrayList; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * Creates a MapTaskExecutor from a MapTask definition. + */ +public class MapTaskExecutorFactory { + /** + * Creates a new MapTaskExecutor from the given MapTask definition. + */ + public static MapTaskExecutor create(PipelineOptions options, + MapTask mapTask, + ExecutionContext context) + throws Exception { + List operations = new ArrayList<>(); + CounterSet counters = new CounterSet(); + String counterPrefix = mapTask.getStageName() + "-"; + StateSampler stateSampler = new StateSampler( + counterPrefix, counters.getAddCounterMutator()); + // Open-ended state. + stateSampler.setState("other"); + + // Instantiate operations for each instruction in the graph. + for (ParallelInstruction instruction : mapTask.getInstructions()) { + operations.add( + createOperation(options, instruction, context, operations, + counterPrefix, counters.getAddCounterMutator(), + stateSampler)); + } + + return new MapTaskExecutor(operations, counters, stateSampler); + } + + /** + * Creates an Operation from the given ParallelInstruction definition. + */ + static Operation createOperation( + PipelineOptions options, + ParallelInstruction instruction, + ExecutionContext executionContext, + List priorOperations, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) + throws Exception { + if (instruction.getRead() != null) { + return createReadOperation( + options, instruction, executionContext, priorOperations, + counterPrefix, addCounterMutator, stateSampler); + } else if (instruction.getWrite() != null) { + return createWriteOperation( + options, instruction, executionContext, priorOperations, + counterPrefix, addCounterMutator, stateSampler); + } else if (instruction.getParDo() != null) { + return createParDoOperation( + options, instruction, executionContext, priorOperations, + counterPrefix, addCounterMutator, stateSampler); + } else if (instruction.getPartialGroupByKey() != null) { + return createPartialGroupByKeyOperation( + options, instruction, executionContext, priorOperations, + counterPrefix, addCounterMutator, stateSampler); + } else if (instruction.getFlatten() != null) { + return createFlattenOperation( + options, instruction, executionContext, priorOperations, + counterPrefix, addCounterMutator, stateSampler); + } else { + throw new Exception("Unexpected instruction: " + instruction); + } + } + + static ReadOperation createReadOperation( + PipelineOptions options, + ParallelInstruction instruction, + ExecutionContext executionContext, + List priorOperations, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) + throws Exception { + ReadInstruction read = instruction.getRead(); + + Source source = + SourceFactory.create(options, read.getSource(), executionContext); + + OutputReceiver[] receivers = createOutputReceivers( + instruction, counterPrefix, addCounterMutator, stateSampler, 1); + + return new ReadOperation(instruction.getSystemName(), source, receivers, + counterPrefix, addCounterMutator, stateSampler); + } + + static WriteOperation createWriteOperation( + PipelineOptions options, + ParallelInstruction instruction, + ExecutionContext executionContext, + List priorOperations, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) + throws Exception { + WriteInstruction write = instruction.getWrite(); + + Sink sink = SinkFactory.create(options, write.getSink(), executionContext); + + OutputReceiver[] receivers = createOutputReceivers( + instruction, counterPrefix, addCounterMutator, stateSampler, 0); + + WriteOperation operation = + new WriteOperation(instruction.getSystemName(), sink, receivers, + counterPrefix, addCounterMutator, stateSampler); + + attachInput(operation, write.getInput(), priorOperations); + + return operation; + } + + static ParDoOperation createParDoOperation( + PipelineOptions options, + ParallelInstruction instruction, + ExecutionContext executionContext, + List priorOperations, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) + throws Exception { + ParDoInstruction parDo = instruction.getParDo(); + + ParDoFn fn = ParDoFnFactory.create( + options, + CloudObject.fromSpec(parDo.getUserFn()), + instruction.getSystemName(), + parDo.getSideInputs(), + parDo.getMultiOutputInfos(), + parDo.getNumOutputs(), + executionContext, + addCounterMutator, + stateSampler); + + OutputReceiver[] receivers = + createOutputReceivers(instruction, counterPrefix, addCounterMutator, + stateSampler, parDo.getNumOutputs()); + + ParDoOperation operation = + new ParDoOperation(instruction.getSystemName(), fn, receivers, + counterPrefix, addCounterMutator, stateSampler); + + attachInput(operation, parDo.getInput(), priorOperations); + + return operation; + } + + static PartialGroupByKeyOperation createPartialGroupByKeyOperation( + PipelineOptions options, + ParallelInstruction instruction, + ExecutionContext executionContext, + List priorOperations, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) + throws Exception { + PartialGroupByKeyInstruction pgbk = instruction.getPartialGroupByKey(); + + Coder coder = Serializer.deserialize(pgbk.getInputElementCodec(), Coder.class); + if (!(coder instanceof WindowedValueCoder)) { + throw new Exception( + "unexpected kind of input coder for PartialGroupByKeyOperation: " + coder); + } + Coder elemCoder = ((WindowedValueCoder) coder).getValueCoder(); + if (!(elemCoder instanceof KvCoder)) { + throw new Exception( + "unexpected kind of input element coder for PartialGroupByKeyOperation: " + elemCoder); + } + KvCoder kvCoder = (KvCoder) elemCoder; + Coder keyCoder = kvCoder.getKeyCoder(); + Coder valueCoder = kvCoder.getValueCoder(); + + OutputReceiver[] receivers = createOutputReceivers( + instruction, counterPrefix, addCounterMutator, stateSampler, 1); + + PartialGroupByKeyOperation operation = + new PartialGroupByKeyOperation(instruction.getSystemName(), + new CoderGroupingKeyCreator(keyCoder), + new CoderSizeEstimator(keyCoder), + new CoderSizeEstimator(valueCoder), + 0.001 /*sizeEstimatorSampleRate*/, + PairInfo.create(), + receivers, + counterPrefix, addCounterMutator, + stateSampler); + + attachInput(operation, pgbk.getInput(), priorOperations); + + return operation; + } + + /** + * Implements PGBKOp.PairInfo via KVs. + */ + public static class PairInfo implements PartialGroupByKeyOperation.PairInfo { + private static PairInfo theInstance = new PairInfo(); + public static PairInfo create() { return theInstance; } + private PairInfo() {} + @Override + public Object getKeyFromInputPair(Object pair) { + WindowedValue> windowedKv = (WindowedValue>) pair; + return windowedKv.getValue().getKey(); + } + @Override + public Object getValueFromInputPair(Object pair) { + WindowedValue> windowedKv = (WindowedValue>) pair; + return windowedKv.getValue().getValue(); + } + @Override + public Object makeOutputPair(Object key, Object values) { + return WindowedValue.valueInEmptyWindows(KV.of(key, values)); + } + } + + /** + * Implements PGBKOp.GroupingKeyCreator via Coder. + */ + public static class CoderGroupingKeyCreator + implements PartialGroupByKeyOperation.GroupingKeyCreator { + final Coder coder; + + public CoderGroupingKeyCreator(Coder coder) { + this.coder = coder; + } + + @Override + public Object createGroupingKey(Object value) throws Exception { + return new PartialGroupByKeyOperation.StructuralByteArray( + CoderUtils.encodeToByteArray(coder, value)); + } + } + + /** + * Implements PGBKOp.SizeEstimator via Coder. + */ + public static class CoderSizeEstimator + implements PartialGroupByKeyOperation.SizeEstimator { + final Coder coder; + + public CoderSizeEstimator(Coder coder) { + this.coder = coder; + } + + @Override + public long estimateSize(Object value) throws Exception { + return CoderUtils.encodeToByteArray(coder, value).length; + } + } + + static FlattenOperation createFlattenOperation( + PipelineOptions options, + ParallelInstruction instruction, + ExecutionContext executionContext, + List priorOperations, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) + throws Exception { + FlattenInstruction flatten = instruction.getFlatten(); + + OutputReceiver[] receivers = + createOutputReceivers(instruction, counterPrefix, addCounterMutator, + stateSampler, 1); + + FlattenOperation operation = + new FlattenOperation(instruction.getSystemName(), receivers, + counterPrefix, addCounterMutator, stateSampler); + + for (InstructionInput input : flatten.getInputs()) { + attachInput(operation, input, priorOperations); + } + + return operation; + } + + /** + * Returns an array of OutputReceivers for the given + * ParallelInstruction definition. + */ + static OutputReceiver[] createOutputReceivers( + ParallelInstruction instruction, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler, + int expectedNumOutputs) + throws Exception { + int numOutputs = 0; + if (instruction.getOutputs() != null) { + numOutputs = instruction.getOutputs().size(); + } + if (numOutputs != expectedNumOutputs) { + throw new AssertionError( + "ParallelInstruction.Outputs has an unexpected length"); + } + OutputReceiver[] receivers = new OutputReceiver[numOutputs]; + for (int i = 0; i < numOutputs; i++) { + InstructionOutput cloudOutput = instruction.getOutputs().get(i); + receivers[i] = new OutputReceiver( + cloudOutput.getName(), + new ElementByteSizeObservableCoder( + Serializer.deserialize(cloudOutput.getCodec(), Coder.class)), + counterPrefix, + addCounterMutator); + } + return receivers; + } + + /** + * Adapts a Coder to the ElementByteSizeObservable interface. + */ + public static class ElementByteSizeObservableCoder + implements ElementByteSizeObservable { + final Coder coder; + + public ElementByteSizeObservableCoder(Coder coder) { + this.coder = coder; + } + + @Override + public boolean isRegisterByteSizeObserverCheap(T value) { + return coder.isRegisterByteSizeObserverCheap(value, Coder.Context.OUTER); + } + + @Override + public void registerByteSizeObserver(T value, + ElementByteSizeObserver observer) + throws Exception { + coder.registerByteSizeObserver(value, observer, Coder.Context.OUTER); + } + } + + /** + * Adds an input to the given Operation, coming from the given + * producer instruction output. + */ + static void attachInput(ReceivingOperation operation, + @Nullable InstructionInput input, + List priorOperations) { + Integer producerInstructionIndex = 0; + Integer outputNum = 0; + if (input != null) { + if (input.getProducerInstructionIndex() != null) { + producerInstructionIndex = input.getProducerInstructionIndex(); + } + if (input.getOutputNum() != null) { + outputNum = input.getOutputNum(); + } + } + // Input id must refer to an operation that has already been seen. + Operation source = priorOperations.get(producerInstructionIndex); + operation.attachInput(source, outputNum); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java new file mode 100644 index 0000000000000..c6e5f9f163e35 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java @@ -0,0 +1,214 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getBytes; + +import com.google.api.services.dataflow.model.MultiOutputInfo; +import com.google.api.services.dataflow.model.SideInputInfo; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.DoFnRunner; +import com.google.cloud.dataflow.sdk.util.DoFnRunner.OutputManager; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.ExecutionContext.StepContext; +import com.google.cloud.dataflow.sdk.util.PTuple; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.worker.OutputReceiver; +import com.google.cloud.dataflow.sdk.util.common.worker.ParDoFn; +import com.google.cloud.dataflow.sdk.util.common.worker.Receiver; +import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import javax.annotation.Nullable; + +/** + * A wrapper around a decoded user DoFn. + */ +public class NormalParDoFn extends ParDoFn { + public static NormalParDoFn create( + PipelineOptions options, + CloudObject cloudUserFn, + String stepName, + @Nullable List sideInputInfos, + @Nullable List multiOutputInfos, + Integer numOutputs, + ExecutionContext executionContext, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler /* ignored */) + throws Exception { + Object deserializedFn = + SerializableUtils.deserializeFromByteArray( + getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), + "serialized user fn"); + if (!(deserializedFn instanceof DoFn)) { + throw new Exception("unexpected kind of DoFn: " + deserializedFn.getClass().getName()); + } + DoFn fn = (DoFn) deserializedFn; + + PTuple sideInputValues = PTuple.empty(); + if (sideInputInfos != null) { + for (SideInputInfo sideInputInfo : sideInputInfos) { + Object sideInputValue = SideInputUtils.readSideInput( + options, sideInputInfo, executionContext); + TupleTag tag = new TupleTag(sideInputInfo.getTag()); + sideInputValues = sideInputValues.and(tag, sideInputValue); + } + } + + List outputTags = new ArrayList<>(); + if (multiOutputInfos != null) { + for (MultiOutputInfo multiOutputInfo : multiOutputInfos) { + outputTags.add(multiOutputInfo.getTag()); + } + } + if (outputTags.isEmpty()) { + // Legacy support: assume there's a single output tag named "output". + // (The output tag name will be ignored, for the main output.) + outputTags.add("output"); + } + if (numOutputs != outputTags.size()) { + throw new AssertionError( + "unexpected number of outputTags for DoFn"); + } + + return new NormalParDoFn(options, fn, sideInputValues, outputTags, + stepName, executionContext, addCounterMutator); + } + + public final PipelineOptions options; + public final DoFn fn; + public final PTuple sideInputValues; + public final TupleTag mainOutputTag; + public final List> sideOutputTags; + public final String stepName; + public final ExecutionContext executionContext; + private final CounterSet.AddCounterMutator addCounterMutator; + + /** The DoFnRunner executing a batch. Null between batches. */ + DoFnRunner fnRunner; + + public NormalParDoFn(PipelineOptions options, + DoFn fn, + PTuple sideInputValues, + List outputTags, + String stepName, + ExecutionContext executionContext, + CounterSet.AddCounterMutator addCounterMutator) { + this.options = options; + this.fn = fn; + this.sideInputValues = sideInputValues; + if (outputTags.size() < 1) { + throw new AssertionError("expected at least one output"); + } + this.mainOutputTag = new TupleTag(outputTags.get(0)); + this.sideOutputTags = new ArrayList<>(); + if (outputTags.size() > 1) { + for (String tag : outputTags.subList(1, outputTags.size())) { + this.sideOutputTags.add(new TupleTag(tag)); + } + } + this.stepName = stepName; + this.executionContext = executionContext; + this.addCounterMutator = addCounterMutator; + } + + @Override + public void startBundle(final Receiver... receivers) throws Exception { + if (receivers.length != sideOutputTags.size() + 1) { + throw new AssertionError( + "unexpected number of receivers for DoFn"); + } + + StepContext stepContext = null; + if (executionContext != null) { + stepContext = executionContext.getStepContext(stepName); + } + + fnRunner = DoFnRunner.create( + options, + fn, + sideInputValues, + new OutputManager() { + final Map, OutputReceiver> undeclaredOutputs = + new HashMap<>(); + + @Override + public Receiver initialize(TupleTag tag) { + // Declared outputs. + if (tag.equals(mainOutputTag)) { + return receivers[0]; + } else if (sideOutputTags.contains(tag)) { + return receivers[sideOutputTags.indexOf(tag) + 1]; + } + + // Undeclared outputs. + OutputReceiver receiver = undeclaredOutputs.get(tag); + if (receiver == null) { + // A new undeclared output. + // TODO: plumb through the operationName, so that we can + // name implicit outputs after it. + String outputName = "implicit-" + tag.getId(); + // TODO: plumb through the counter prefix, so we can + // make it available to the OutputReceiver class in case + // it wants to use it in naming output counters. (It + // doesn't today.) + String counterPrefix = ""; + receiver = new OutputReceiver( + outputName, counterPrefix, addCounterMutator); + undeclaredOutputs.put(tag, receiver); + } + return receiver; + } + + @Override + public void output(Receiver receiver, WindowedValue output) { + try { + receiver.process(output); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + }, + mainOutputTag, + sideOutputTags, + stepContext, + addCounterMutator); + + fnRunner.startBundle(); + } + + @Override + public void processElement(Object elem) throws Exception { + fnRunner.processElement((WindowedValue) elem); + } + + @Override + public void finishBundle() throws Exception { + fnRunner.finishBundle(); + fnRunner = null; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/OrderedCode.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/OrderedCode.java new file mode 100644 index 0000000000000..487420ce39342 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/OrderedCode.java @@ -0,0 +1,678 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.common.math.LongMath; +import com.google.common.primitives.Longs; + +import java.math.RoundingMode; +import java.util.ArrayList; +import java.util.Arrays; + +/** + * This module provides routines for encoding a sequence of typed + * entities into a byte array. The resulting byte arrays can be + * lexicographically compared to yield the same comparison value that + * would have been generated if the encoded items had been compared + * one by one according to their type. + * + * More precisely, suppose: + * 1. byte array A is generated by encoding the sequence of items [A_1..A_n] + * 2. byte array B is generated by encoding the sequence of items [B_1..B_n] + * 3. The types match; i.e., for all i: A_i was encoded using + * the same routine as B_i + * Then: + * Comparing A vs. B lexicographically is the same as comparing + * the vectors [A_1..A_n] and [B_1..B_n] lexicographically. + * + *

+ * This class is NOT thread safe. + */ +public class OrderedCode { + // We want to encode a few extra symbols in strings: + // Separator between items + // Infinite string + // + // Therefore we need an alphabet with at least 258 characters. We + // achieve this by using two-letter sequences starting with '\0' and '\xff' + // as extra symbols: + // encoded as => \0\1 + // \0 encoded as => \0\xff + // \xff encoded as => \xff\x00 + // encoded as => \xff\xff + // + // The remaining two letter sequences starting with '\0' and '\xff' + // are currently unused. + + public static final byte ESCAPE1 = 0x00; + public static final byte NULL_CHARACTER = + (byte) 0xff; // Combined with ESCAPE1 + public static final byte SEPARATOR = 0x01; // Combined with ESCAPE1 + + public static final byte ESCAPE2 = (byte) 0xff; + public static final byte INFINITY = + (byte) 0xff; // Combined with ESCAPE2 + public static final byte FF_CHARACTER = 0x00; // Combined with ESCAPE2 + + public static final byte[] ESCAPE1_SEPARATOR = { ESCAPE1, SEPARATOR }; + + public static final byte[] INFINITY_ENCODED = { ESCAPE2, INFINITY }; + + /** + * This array maps encoding length to header bits in the first two bytes for + * SignedNumIncreasing encoding. + */ + private static final byte[][] LENGTH_TO_HEADER_BITS = { + { 0, 0 }, + { (byte) 0x80, 0 }, + { (byte) 0xc0, 0 }, + { (byte) 0xe0, 0 }, + { (byte) 0xf0, 0 }, + { (byte) 0xf8, 0 }, + { (byte) 0xfc, 0 }, + { (byte) 0xfe, 0 }, + { (byte) 0xff, 0 }, + { (byte) 0xff, (byte) 0x80 }, + { (byte) 0xff, (byte) 0xc0 } + }; + + /** + * This array maps encoding lengths to the header bits that overlap with + * the payload and need fixing during readSignedNumIncreasing. + */ + private static final long[] LENGTH_TO_MASK = { + 0L, + 0x80L, + 0xc000L, + 0xe00000L, + 0xf0000000L, + 0xf800000000L, + 0xfc0000000000L, + 0xfe000000000000L, + 0xff00000000000000L, + 0x8000000000000000L, + 0L + }; + + /** + * This array maps the number of bits in a number to the encoding + * length produced by WriteSignedNumIncreasing. + * For positive numbers, the number of bits is 1 plus the most significant + * bit position (the highest bit position in a positive long is 63). + * For a negative number n, we count the bits in ~n. + * That is, length = BITS_TO_LENGTH[log2Floor(n < 0 ? ~n : n) + 1]. + */ + private static final short[] BITS_TO_LENGTH = { + 1, 1, 1, 1, 1, 1, 1, + 2, 2, 2, 2, 2, 2, 2, + 3, 3, 3, 3, 3, 3, 3, + 4, 4, 4, 4, 4, 4, 4, + 5, 5, 5, 5, 5, 5, 5, + 6, 6, 6, 6, 6, 6, 6, + 7, 7, 7, 7, 7, 7, 7, + 8, 8, 8, 8, 8, 8, 8, + 9, 9, 9, 9, 9, 9, 9, + 10 + }; + + /** + * stores the current encoded value as a list of byte arrays. Note that this + * is manipulated as we read/write items. + * Note that every item will fit on at most one array. One array may + * have more than one item (eg when used for decoding). While encoding, + * one array will have exactly one item. While returning the encoded array + * we will merge all the arrays in this list. + */ + private final ArrayList encodedArrays = new ArrayList<>(); + + /** + * This is the current position on the first array. Will be non-zero + * only if the ordered code was created using encoded byte array. + */ + private int firstArrayPosition = 0; + + /** + * Creates OrderedCode from scractch. Typically used at encoding time. + */ + public OrderedCode(){ + } + + /** + * Creates OrderedCode from a given encoded byte array. Typically used at + * decoding time. + * + *

+ * For better performance, it uses the input array provided (not a copy). + * Therefore the input array should not be modified. + */ + public OrderedCode(byte[] encodedByteArray) { + encodedArrays.add(encodedByteArray); + } + + /** + * Adds the given byte array item to the OrderedCode. It encodes the input + * byte array, followed by a separator and appends the result to its + * internal encoded byte array store. + * + *

+ * It works with the input array, + * so the input array 'value' should not be modified till the method returns. + * + * @param value bytes to be written. + * @see #readBytes() + */ + public void writeBytes(byte[] value) { + // Determine the length of the encoded array + int encodedLength = 2; // for separator + for (byte b : value) { + if ((b == ESCAPE1) || (b == ESCAPE2)) { + encodedLength += 2; + } else { + encodedLength++; + } + } + + byte[] encodedArray = new byte[encodedLength]; + int copyStart = 0; + int outIndex = 0; + for (int i = 0; i < value.length; i++) { + byte b = value[i]; + if (b == ESCAPE1) { + System.arraycopy(value, copyStart, encodedArray, outIndex, + i - copyStart); + outIndex += i - copyStart; + encodedArray[outIndex++] = ESCAPE1; + encodedArray[outIndex++] = NULL_CHARACTER; + copyStart = i + 1; + } else if (b == ESCAPE2) { + System.arraycopy(value, copyStart, encodedArray, outIndex, + i - copyStart); + outIndex += i - copyStart; + encodedArray[outIndex++] = ESCAPE2; + encodedArray[outIndex++] = FF_CHARACTER; + copyStart = i + 1; + } + } + if (copyStart < value.length) { + System.arraycopy(value, copyStart, encodedArray, outIndex, + value.length - copyStart); + outIndex += value.length - copyStart; + } + encodedArray[outIndex++] = ESCAPE1; + encodedArray[outIndex] = SEPARATOR; + + encodedArrays.add(encodedArray); + } + + /** + * Encodes the long item, in big-endian format, and appends the result to its + * internal encoded byte array store. + * + * Note that the specified long is treated like a uint64, e.g. + * {@code new OrderedCode().writeNumIncreasing(-1L).getEncodedBytes() > + * new OrderedCode().writeNumIncreasing(Long.MAX_VALUE).getEncodedBytes()}. + * + * @see #readNumIncreasing() + */ + public void writeNumIncreasing(long value) { + // Values are encoded with a single byte length prefix, followed + // by the actual value in big-endian format with leading 0 bytes + // dropped. + byte[] bufer = new byte[9]; // 8 bytes for value plus one byte for length + int len = 0; + while (value != 0) { + len++; + bufer[9 - len] = (byte) (value & 0xff); + value >>>= 8; + } + bufer[9 - len - 1] = (byte) len; + len++; + byte[] encodedArray = new byte[len]; + System.arraycopy(bufer, 9 - len, encodedArray, 0, len); + encodedArrays.add(encodedArray); + } + + /** + * Return floor(log2(n)) for positive integer n. Returns -1 iff n == 0. + */ + int log2Floor(long n) { + if (n < 0) { + throw new IllegalArgumentException("must be non-negative"); + } + return n == 0 ? -1 : LongMath.log2(n, RoundingMode.FLOOR); + } + + /** + * Calculates the encoding length in bytes of the signed number n. + */ + int getSignedEncodingLength(long n) { + return BITS_TO_LENGTH[log2Floor(n < 0 ? ~n : n) + 1]; + } + + /** + * Encodes the long item, in big-endian format, and appends the result to its + * internal encoded byte array store. + * + * Note that the specified long is treated like an int64, i.e. + * {@code new OrderedCode().writeNumIncreasing(-1L).getEncodedBytes() < + * new OrderedCode().writeNumIncreasing(0L).getEncodedBytes()}. + * + * @see #readSignedNumIncreasing() + */ + public void writeSignedNumIncreasing(long val) { + long x = val < 0 ? ~val : val; + if (x < 64) { // Fast path for encoding length == 1. + byte[] encodedArray = + new byte[] { (byte) (LENGTH_TO_HEADER_BITS[1][0] ^ val) }; + encodedArrays.add(encodedArray); + return; + } + // buf = val in network byte order, sign extended to 10 bytes. + byte signByte = val < 0 ? (byte) 0xff : 0; + byte[] buf = new byte[2 + Longs.BYTES]; + buf[0] = buf[1] = signByte; + System.arraycopy(Longs.toByteArray(val), 0, buf, 2, Longs.BYTES); + int len = getSignedEncodingLength(x); + if (len < 2) { + throw new IllegalStateException( + "Invalid length (" + len + ")" + + " returned by getSignedEncodingLength(" + x + ")"); + } + int beginIndex = buf.length - len; + buf[beginIndex] ^= LENGTH_TO_HEADER_BITS[len][0]; + buf[beginIndex + 1] ^= LENGTH_TO_HEADER_BITS[len][1]; + + byte[] encodedArray = new byte[len]; + System.arraycopy(buf, beginIndex, encodedArray, 0, len); + encodedArrays.add(encodedArray); + } + + /** + * Encodes and appends INFINITY item to its internal encoded byte array + * store. + * + * @see #readInfinity() + */ + public void writeInfinity() { + writeTrailingBytes(INFINITY_ENCODED); + } + + /** + * Appends the byte array item to its internal encoded byte array + * store. This is used for the last item and is not encoded. It + * also can be used to write a fixed number of bytes which will be + * read back using {@link #readBytes(int)}. + * + *

+ * It stores the input array in the store, + * so the input array 'value' should not be modified. + * + * @param value bytes to be written. + * @see #readTrailingBytes() + * @see #readBytes(int) + */ + public void writeTrailingBytes(byte[] value) { + if ((value == null) || (value.length == 0)) { + throw new IllegalArgumentException( + "Value cannot be null or have 0 elements"); + } + + encodedArrays.add(value); + } + + /** + * Returns the next byte array item from its encoded byte array store and + * removes the item from the store. + * + * @see #writeBytes(byte[]) + */ + public byte[] readBytes() { + if ((encodedArrays == null) || (encodedArrays.size() == 0) || + ((encodedArrays.get(0)).length - firstArrayPosition <= 0)) { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + + // Determine the length of the decoded array + // We only scan up to "length-2" since a valid string must end with + // a two character terminator: 'ESCAPE1 SEPARATOR' + byte[] store = encodedArrays.get(0); + int decodedLength = 0; + boolean valid = false; + int i = firstArrayPosition; + while (i < store.length - 1) { + byte b = store[i++]; + if (b == ESCAPE1) { + b = store[i++]; + if (b == SEPARATOR) { + valid = true; + break; + } else if (b == NULL_CHARACTER) { + decodedLength++; + } else { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + } else if (b == ESCAPE2) { + b = store[i++]; + if (b == FF_CHARACTER) { + decodedLength++; + } else { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + } else { + decodedLength++; + } + } + if (!valid) { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + + byte[] decodedArray = new byte[decodedLength]; + int copyStart = firstArrayPosition; + int outIndex = 0; + int j = firstArrayPosition; + while (j < store.length - 1) { + byte b = store[j++]; // note that j has been incremented + if (b == ESCAPE1) { + System.arraycopy(store, copyStart, decodedArray, outIndex, + j - copyStart - 1); + outIndex += j - copyStart - 1; + // ESCAPE1 SEPARATOR ends component + // ESCAPE1 NULL_CHARACTER represents '\0' + b = store[j++]; + if (b == SEPARATOR) { + if ((store.length - j) == 0) { + // we are done with the first array + encodedArrays.remove(0); + firstArrayPosition = 0; + } else { + firstArrayPosition = j; + } + return decodedArray; + } else if (b == NULL_CHARACTER) { + decodedArray[outIndex++] = 0x00; + } // else not required - handled during length determination + copyStart = j; + } else if (b == ESCAPE2) { + System.arraycopy(store, copyStart, decodedArray, outIndex, + j - copyStart - 1); + outIndex += j - copyStart - 1; + // ESCAPE2 FF_CHARACTER represents '\xff' + // ESCAPE2 INFINITY is an error + b = store[j++]; + if (b == FF_CHARACTER) { + decodedArray[outIndex++] = (byte) 0xff; + } // else not required - handled during length determination + copyStart = j; + } + } + // not required due to the first phase, but need to entertain the compiler + throw new IllegalArgumentException("Invalid encoded byte array"); + } + + /** + * Returns the next long item (encoded in big-endian format via + * {@code writeNumIncreasing(long)}) from its internal encoded byte array + * store and removes the item from the store. + * + * @see #writeNumIncreasing(long) + */ + public long readNumIncreasing() { + if ((encodedArrays == null) || (encodedArrays.size() == 0) || + ((encodedArrays.get(0)).length - firstArrayPosition < 1)) { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + + byte[] store = encodedArrays.get(0); + // Decode length byte + int len = store[firstArrayPosition]; + if ((firstArrayPosition + len + 1 > store.length) || len > 8) { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + + long result = 0; + for (int i = 0; i < len; i++) { + result <<= 8; + result |= (store[firstArrayPosition + i + 1] & 0xff); + } + + if ((store.length - firstArrayPosition - len - 1) == 0) { + // we are done with the first array + encodedArrays.remove(0); + firstArrayPosition = 0; + } else { + firstArrayPosition = firstArrayPosition + len + 1; + } + + return result; + } + + /** + * Returns the next long item (encoded via + * {@code writeSignedNumIncreasing(long)}) from its internal encoded byte + * array store and removes the item from the store. + * + * @see #writeSignedNumIncreasing(long) + */ + public long readSignedNumIncreasing() { + if ((encodedArrays == null) || (encodedArrays.size() == 0) || + ((encodedArrays.get(0)).length - firstArrayPosition < 1)) { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + + byte[] store = encodedArrays.get(0); + + long xorMask = ((store[firstArrayPosition] & 0x80) == 0) ? ~0L : 0L; + // Store first byte as an int rather than a (signed) byte -- to avoid + // accidental byte-to-int promotion later which would extend the byte's + // sign bit (if any). + int firstByte = + (store[firstArrayPosition] & 0xff) ^ (int) (xorMask & 0xff); + + // Now calculate and test length, and set x to raw (unmasked) result. + int len; + long x; + if (firstByte != 0xff) { + len = 7 - log2Floor(firstByte ^ 0xff); + if (store.length - firstArrayPosition < len) { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + x = xorMask; // Sign extend using xorMask. + for (int i = firstArrayPosition; i < firstArrayPosition + len; i++) { + x = (x << 8) | (store[i] & 0xff); + } + } else { + len = 8; + if (store.length - firstArrayPosition < len) { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + int secondByte = + (store[firstArrayPosition + 1] & 0xff) ^ (int) (xorMask & 0xff); + if (secondByte >= 0x80) { + if (secondByte < 0xc0) { + len = 9; + } else { + int thirdByte = + (store[firstArrayPosition + 2] & 0xff) ^ (int) (xorMask & 0xff); + if (secondByte == 0xc0 && thirdByte < 0x80) { + len = 10; + } else { + // Either len > 10 or len == 10 and #bits > 63. + throw new IllegalArgumentException("Invalid encoded byte array"); + } + } + if (store.length - firstArrayPosition < len) { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + } + x = Longs.fromByteArray(Arrays.copyOfRange( + store, firstArrayPosition + len - 8, firstArrayPosition + len)); + } + + x ^= LENGTH_TO_MASK[len]; // Remove spurious header bits. + + if (len != getSignedEncodingLength(x)) { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + + if ((store.length - firstArrayPosition - len) == 0) { + // We are done with the first array. + encodedArrays.remove(0); + firstArrayPosition = 0; + } else { + firstArrayPosition = firstArrayPosition + len; + } + + return x; + } + + /** + * Removes INFINITY item from its internal encoded byte array store + * if present. Returns whether INFINITY was present. + * + * @see #writeInfinity() + */ + public boolean readInfinity() { + if ((encodedArrays == null) || (encodedArrays.size() == 0) || + ((encodedArrays.get(0)).length - firstArrayPosition < 1)) { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + byte[] store = encodedArrays.get(0); + if (store.length - firstArrayPosition < 2) { + return false; + } + if ((store[firstArrayPosition] == ESCAPE2) && + (store[firstArrayPosition + 1] == INFINITY)) { + if ((store.length - firstArrayPosition - 2) == 0) { + // we are done with the first array + encodedArrays.remove(0); + firstArrayPosition = 0; + } else { + firstArrayPosition = firstArrayPosition + 2; + } + return true; + } else { + return false; + } + } + + /** + * Returns the trailing byte array item from its internal encoded byte array + * store and removes the item from the store. + * + * @see #writeTrailingBytes(byte[]) + */ + public byte[] readTrailingBytes() { + // one item is contained within one byte array + if ((encodedArrays == null) || (encodedArrays.size() != 1)) { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + + byte[] store = encodedArrays.get(0); + encodedArrays.remove(0); + assert encodedArrays.size() == 0; + return Arrays.copyOfRange(store, firstArrayPosition, store.length); + } + + /** + * Reads (unencoded) {@code len} bytes. + * + * @see #writeTrailingBytes(byte[]) + */ + public byte[] readBytes(int len) { + if ((encodedArrays == null) || (encodedArrays.size() == 0) || + ((encodedArrays.get(0)).length - firstArrayPosition < len)) { + throw new IllegalArgumentException("Invalid encoded byte array"); + } + + byte[] store = encodedArrays.get(0); + + byte[] result; + if (store.length - firstArrayPosition == len) { + // We are done with the first array. + result = encodedArrays.remove(0); + firstArrayPosition = 0; + } else { + result = new byte[len]; + System.arraycopy(store, firstArrayPosition, result, 0, len); + firstArrayPosition = firstArrayPosition + len; + } + return result; + } + + /** + * Returns the encoded bytes that represent the current state of the + * OrderedCode. + * + *

+ * NOTE: This method returns OrederedCode's internal array (not a + * copy) for better performance. Therefore the returned array should not be + * modified. + */ + public byte[] getEncodedBytes() { + if (encodedArrays.size() == 0) { + return new byte[0]; + } + if ((encodedArrays.size() == 1) && (firstArrayPosition == 0)) { + return encodedArrays.get(0); + } + + int totalLength = 0; + + for (int i = 0; i < encodedArrays.size(); i++) { + byte[] bytes = encodedArrays.get(i); + if (i == 0) { + totalLength += bytes.length - firstArrayPosition; + } else { + totalLength += bytes.length; + } + } + + byte[] encodedBytes = new byte[totalLength]; + int destPos = 0; + for (int i = 0; i < encodedArrays.size(); i++) { + byte[] bytes = encodedArrays.get(i); + if (i == 0) { + System.arraycopy(bytes, firstArrayPosition, encodedBytes, destPos, + bytes.length - firstArrayPosition); + destPos += bytes.length - firstArrayPosition; + } else { + System.arraycopy(bytes, 0, encodedBytes, destPos, bytes.length); + destPos += bytes.length; + } + } + + // replace the store with merged array, so that repeated calls + // don't need to merge. The reads can handle both the versions. + encodedArrays.clear(); + encodedArrays.add(encodedBytes); + firstArrayPosition = 0; + + return encodedBytes; + } + + /** + * Returns true if this has more encoded bytes that haven't been read, + * false otherwise. Return value of true doesn't imply anything about + * validity of remaining data. + * @return true if it has more encoded bytes that haven't been read, + * false otherwise. + */ + public boolean hasRemainingEncodedBytes() { + // We delete an array after fully consuming it. + return encodedArrays != null && encodedArrays.size() != 0; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactory.java new file mode 100644 index 0000000000000..23d4040685bfd --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactory.java @@ -0,0 +1,115 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.api.services.dataflow.model.MultiOutputInfo; +import com.google.api.services.dataflow.model.SideInputInfo; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.InstanceBuilder; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.worker.ParDoFn; +import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Creates a ParDoFn from a CloudObject spec. + * + * A ParDoFnFactory concrete "subclass" should define a method with + * the following signature: + *

 {@code
+ * static SomeParDoFnSubclass create(
+ *     CloudObject spec,
+ *     List sideInputInfos,
+ *     List multiOutputInfos,
+ *     int numOutputs,
+ *     ExecutionContext executionContext);
+ * } 
+ */ +public class ParDoFnFactory { + // Do not instantiate. + private ParDoFnFactory() {} + + /** + * A map from the short names of predefined ParDoFnFactories to their full + * class names. + */ + static Map predefinedParDoFnFactories = new HashMap<>(); + + static { + predefinedParDoFnFactories.put("DoFn", + NormalParDoFn.class.getName()); + predefinedParDoFnFactories.put("CombineValuesFn", + CombineValuesFn.class.getName()); + // TODO: Remove outdated bindings once the services produces the right ones + predefinedParDoFnFactories.put("MergeBucketsDoFn", + GroupAlsoByWindowsParDoFn.class.getName()); + predefinedParDoFnFactories.put("AssignBucketsDoFn", + AssignWindowsParDoFn.class.getName()); + predefinedParDoFnFactories.put("MergeWindowsDoFn", + GroupAlsoByWindowsParDoFn.class.getName()); + predefinedParDoFnFactories.put("AssignWindowsDoFn", + AssignWindowsParDoFn.class.getName()); + } + + /** + * Creates a ParDoFn from a CloudObject spec. + * + * @throws Exception if the CloudObject spec could not be + * decoded and constructed. + */ + public static ParDoFn create(PipelineOptions options, + CloudObject cloudUserFn, + String stepName, + List sideInputInfos, + List multiOutputInfos, + int numOutputs, + ExecutionContext executionContext, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) + throws Exception { + String className = cloudUserFn.getClassName(); + String parDoFnFactoryClassName = predefinedParDoFnFactories.get(className); + if (parDoFnFactoryClassName == null) { + parDoFnFactoryClassName = className; + } + + try { + return InstanceBuilder.ofType(ParDoFn.class) + .fromClassName(parDoFnFactoryClassName) + .fromFactoryMethod("create") + .withArg(PipelineOptions.class, options) + .withArg(CloudObject.class, cloudUserFn) + .withArg(String.class, stepName) + .withArg(List.class, sideInputInfos) + .withArg(List.class, multiOutputInfos) + .withArg(Integer.class, numOutputs) + .withArg(ExecutionContext.class, executionContext) + .withArg(CounterSet.AddCounterMutator.class, addCounterMutator) + .withArg(StateSampler.class, stateSampler) + .build(); + + } catch (ClassNotFoundException exn) { + throw new Exception( + "unable to create a ParDoFn from " + cloudUserFn, exn); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSource.java new file mode 100644 index 0000000000000..5394a26cc47fc --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSource.java @@ -0,0 +1,128 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.worker.BatchingShuffleEntryReader; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntryReader; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.values.KV; + +import java.io.IOException; +import java.util.Iterator; + +/** + * A source that reads from a key-sharded dataset, and returns KVs without + * any values grouping. + * + * @param the type of the keys read from the shuffle + * @param the type of the values read from the shuffle + */ +public class PartitioningShuffleSource extends Source>> { + + final byte[] shuffleReaderConfig; + final String startShufflePosition; + final String stopShufflePosition; + Coder keyCoder; + WindowedValueCoder windowedValueCoder; + + public PartitioningShuffleSource(PipelineOptions options, + byte[] shuffleReaderConfig, + String startShufflePosition, + String stopShufflePosition, + Coder>> coder) + throws Exception { + this.shuffleReaderConfig = shuffleReaderConfig; + this.startShufflePosition = startShufflePosition; + this.stopShufflePosition = stopShufflePosition; + initCoder(coder); + } + + /** + * Given a {@code WindowedValueCoder>}, splits it into a coder for K + * and a {@code WindowedValueCoder} with the same kind of windows. + */ + private void initCoder(Coder>> coder) throws Exception { + if (!(coder instanceof WindowedValueCoder)) { + throw new Exception( + "unexpected kind of coder for WindowedValue: " + coder); + } + WindowedValueCoder> windowedElemCoder = ((WindowedValueCoder>) coder); + Coder> elemCoder = windowedElemCoder.getValueCoder(); + if (!(elemCoder instanceof KvCoder)) { + throw new Exception( + "unexpected kind of coder for elements read from " + + "a key-partitioning shuffle: " + elemCoder); + } + KvCoder kvCoder = (KvCoder) elemCoder; + this.keyCoder = kvCoder.getKeyCoder(); + windowedValueCoder = windowedElemCoder.withValueCoder(kvCoder.getValueCoder()); + } + + @Override + public com.google.cloud.dataflow.sdk.util.common.worker.Source.SourceIterator< + WindowedValue>> iterator() throws IOException { + Preconditions.checkArgument(shuffleReaderConfig != null); + return iterator(new BatchingShuffleEntryReader( + new ChunkingShuffleBatchReader(new ApplianceShuffleReader( + shuffleReaderConfig)))); + } + + SourceIterator>> iterator(ShuffleEntryReader reader) throws IOException { + return new PartitioningShuffleSourceIterator(reader); + } + + /** + * A SourceIterator that reads from a ShuffleEntryReader, + * extracts K and {@code WindowedValue}, and returns a constructed + * {@code WindowedValue}. + */ + class PartitioningShuffleSourceIterator + extends AbstractSourceIterator>> { + Iterator iterator; + + PartitioningShuffleSourceIterator(ShuffleEntryReader reader) { + this.iterator = reader.read( + ByteArrayShufflePosition.fromBase64(startShufflePosition), + ByteArrayShufflePosition.fromBase64(stopShufflePosition)); + } + + @Override + public boolean hasNext() throws IOException { + return iterator.hasNext(); + } + + @Override + public WindowedValue> next() throws IOException { + ShuffleEntry record = iterator.next(); + K key = CoderUtils.decodeFromByteArray(keyCoder, record.getKey()); + WindowedValue windowedValue = + CoderUtils.decodeFromByteArray(windowedValueCoder, record.getValue()); + notifyElementRead(record.length()); + return WindowedValue.of(KV.of(key, windowedValue.getValue()), + windowedValue.getTimestamp(), + windowedValue.getWindows()); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceFactory.java new file mode 100644 index 0000000000000..f97d1d5b82988 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceFactory.java @@ -0,0 +1,50 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Base64.decodeBase64; +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.values.KV; + +/** + * Creates a PartitioningShuffleSource from a CloudObject spec. + */ +public class PartitioningShuffleSourceFactory { + // Do not instantiate. + private PartitioningShuffleSourceFactory() {} + + public static PartitioningShuffleSource create( + PipelineOptions options, + CloudObject spec, + Coder>> coder, + ExecutionContext executionContext) + throws Exception { + return new PartitioningShuffleSource( + options, + decodeBase64(getString(spec, PropertyNames.SHUFFLE_READER_CONFIG)), + getString(spec, PropertyNames.START_SHUFFLE_POSITION, null), + getString(spec, PropertyNames.END_SHUFFLE_POSITION, null), + coder); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleEntryWriter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleEntryWriter.java new file mode 100644 index 0000000000000..4fd44230421d5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleEntryWriter.java @@ -0,0 +1,39 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; + +import java.io.IOException; + +import javax.annotation.concurrent.NotThreadSafe; + +/** + * ShuffleEntryWriter provides an interface for writing key/value + * entries to a shuffle dataset. + */ +@NotThreadSafe +interface ShuffleEntryWriter extends AutoCloseable { + /** + * Writes an entry to a shuffle dataset. Returns the size + * in bytes of the data written. + */ + public long put(ShuffleEntry entry) throws IOException; + + @Override + public void close() throws IOException; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleLibrary.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleLibrary.java new file mode 100644 index 0000000000000..8863436d2e1d6 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleLibrary.java @@ -0,0 +1,44 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.StandardCopyOption; + +/** + * Native library used to read from and write to a shuffle dataset. + */ +class ShuffleLibrary { + /** + * Loads the native shuffle library. + */ + static void load() { + try { + File tempfile = File.createTempFile("libshuffle_client_jni", ".so"); + InputStream input = ClassLoader.getSystemResourceAsStream( + "libshuffle_client_jni.so.stripped"); + Files.copy(input, tempfile.toPath(), StandardCopyOption.REPLACE_EXISTING); + System.load(tempfile.getAbsolutePath()); + } catch (IOException e) { + throw new RuntimeException("Loading shuffle_client failed:", e); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleReader.java new file mode 100644 index 0000000000000..8a1018b237ee7 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleReader.java @@ -0,0 +1,48 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import java.io.IOException; + +/** + * ShuffleReader reads chunks of data from a shuffle dataset for + * a given position range. + */ +interface ShuffleReader { + /** Represents a chunk of data read from a shuffle dataset. */ + public static class ReadChunkResult { + public final byte[] chunk; + public final byte[] nextStartPosition; + public ReadChunkResult(byte[] chunk, byte[] nextStartPosition) { + this.chunk = chunk; + this.nextStartPosition = nextStartPosition; + } + } + + /** + * Reads a chunk of data for keys in the given position range. + * The chunk is a sequence of pairs encoded as: + * {@code + } + * where the sizes are 4-byte big-endian integers. + * + * @param startPosition the start of the requested range (inclusive) + * @param endPosition the end of the requested range (exclusive) + */ + public ReadChunkResult readIncludingPosition( + byte[] startPosition, byte[] endPosition) throws IOException; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSink.java new file mode 100644 index 0000000000000..72ea16fc99b43 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSink.java @@ -0,0 +1,248 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.InstantCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; +import com.google.cloud.dataflow.sdk.values.KV; + +import java.io.IOException; + +/** + * A sink that writes to a shuffle dataset. + * + * @param the type of the elements written to the sink + */ +public class ShuffleSink extends Sink> { + + enum ShuffleKind { UNGROUPED, PARTITION_KEYS, GROUP_KEYS, GROUP_KEYS_AND_SORT_VALUES } + + static final long SHUFFLE_WRITER_BUFFER_SIZE = 128 << 20; + + final byte[] shuffleWriterConfig; + + final ShuffleKind shuffleKind; + + boolean shardByKey; + boolean groupValues; + boolean sortValues; + + WindowedValueCoder windowedElemCoder; + WindowedValueCoder windowedValueCoder; + Coder elemCoder; + Coder keyCoder; + Coder valueCoder; + Coder sortKeyCoder; + Coder sortValueCoder; + + public static ShuffleKind parseShuffleKind(String shuffleKind) + throws Exception { + try { + return Enum.valueOf(ShuffleKind.class, shuffleKind.trim().toUpperCase()); + } catch (IllegalArgumentException e) { + throw new Exception("unexpected shuffle_kind", e); + } + } + + public ShuffleSink(PipelineOptions options, + byte[] shuffleWriterConfig, + ShuffleKind shuffleKind, + Coder> coder) + throws Exception { + this.shuffleWriterConfig = shuffleWriterConfig; + this.shuffleKind = shuffleKind; + initCoder(coder); + } + + private void initCoder(Coder> coder) throws Exception { + switch (shuffleKind) { + case UNGROUPED: + this.shardByKey = false; + this.groupValues = false; + this.sortValues = false; + break; + case PARTITION_KEYS: + this.shardByKey = true; + this.groupValues = false; + this.sortValues = false; + break; + case GROUP_KEYS: + this.shardByKey = true; + this.groupValues = true; + this.sortValues = false; + break; + case GROUP_KEYS_AND_SORT_VALUES: + this.shardByKey = true; + this.groupValues = true; + this.sortValues = true; + break; + default: + throw new AssertionError("unexpected shuffle kind"); + } + + this.windowedElemCoder = (WindowedValueCoder) coder; + this.elemCoder = windowedElemCoder.getValueCoder(); + if (shardByKey) { + if (!(elemCoder instanceof KvCoder)) { + throw new Exception( + "unexpected kind of coder for elements written to " + + "a key-grouping shuffle"); + } + KvCoder kvCoder = (KvCoder) elemCoder; + this.keyCoder = kvCoder.getKeyCoder(); + this.valueCoder = kvCoder.getValueCoder(); + if (sortValues) { + // TODO: Decide the representation of sort-keyed values. + // For now, we'll just use KVs. + if (!(valueCoder instanceof KvCoder)) { + throw new Exception( + "unexpected kind of coder for values written to " + + "a value-sorting shuffle"); + } + KvCoder kvValueCoder = (KvCoder) valueCoder; + this.sortKeyCoder = kvValueCoder.getKeyCoder(); + this.sortValueCoder = kvValueCoder.getValueCoder(); + } else { + this.sortKeyCoder = null; + this.sortValueCoder = null; + } + if (groupValues) { + this.windowedValueCoder = null; + } else { + this.windowedValueCoder = this.windowedElemCoder.withValueCoder(this.valueCoder); + } + } else { + this.keyCoder = null; + this.valueCoder = null; + this.sortKeyCoder = null; + this.sortValueCoder = null; + this.windowedValueCoder = null; + } + } + + /** + * Returns a SinkWriter that allows writing to this ShuffleSink, + * using the given ShuffleEntryWriter. + */ + public SinkWriter> writer(ShuffleEntryWriter writer) throws IOException { + return new ShuffleSinkWriter(writer); + } + + /** The SinkWriter for a ShuffleSink. */ + class ShuffleSinkWriter implements SinkWriter> { + ShuffleEntryWriter writer; + long seqNum = 0; + + ShuffleSinkWriter(ShuffleEntryWriter writer) throws IOException { + this.writer = writer; + } + + @Override + public long add(WindowedValue windowedElem) throws IOException { + byte[] keyBytes; + byte[] secondaryKeyBytes; + byte[] valueBytes; + T elem = windowedElem.getValue(); + if (shardByKey) { + if (!(elem instanceof KV)) { + throw new AssertionError( + "expecting the values written to a key-grouping shuffle " + + "to be KVs"); + } + KV kv = (KV) elem; + Object key = kv.getKey(); + Object value = kv.getValue(); + + keyBytes = CoderUtils.encodeToByteArray(keyCoder, key); + + if (sortValues) { + if (!(value instanceof KV)) { + throw new AssertionError( + "expecting the value parts of the KVs written to " + + "a value-sorting shuffle to also be KVs"); + } + KV kvValue = (KV) value; + Object sortKey = kvValue.getKey(); + Object sortValue = kvValue.getValue(); + + // TODO: Need to coordinate with the + // GroupingShuffleSource, to make sure it knows how to + // reconstruct the value from the sortKeyBytes and + // sortValueBytes. Right now, it doesn't know between + // sorting and non-sorting GBKs. + secondaryKeyBytes = + CoderUtils.encodeToByteArray(sortKeyCoder, sortKey); + valueBytes = CoderUtils.encodeToByteArray(sortValueCoder, sortValue); + + } else if (groupValues) { + // Sort values by timestamp so that GroupAlsoByWindows can run efficiently. + if (windowedElem.getTimestamp().getMillis() == Long.MIN_VALUE) { + // Empty secondary keys sort before all other secondary keys, so we + // can omit this common value here for efficiency. + secondaryKeyBytes = null; + } else { + secondaryKeyBytes = + CoderUtils.encodeToByteArray(InstantCoder.of(), windowedElem.getTimestamp()); + } + valueBytes = CoderUtils.encodeToByteArray(valueCoder, value); + } else { + secondaryKeyBytes = null; + valueBytes = CoderUtils.encodeToByteArray( + windowedValueCoder, + WindowedValue.of(value, windowedElem.getTimestamp(), windowedElem.getWindows())); + } + + } else { + // Not partitioning or grouping by key, just resharding values. + // is ignored, except by the shuffle splitter. Use a seq# + // as the key, so we can split records anywhere. This also works + // for writing a single-sharded ordered PCollection through a + // shuffle, since the order of elements in the input will be + // preserved in the output. + keyBytes = + CoderUtils.encodeToByteArray(BigEndianLongCoder.of(), seqNum++); + + secondaryKeyBytes = null; + valueBytes = CoderUtils.encodeToByteArray(windowedElemCoder, windowedElem); + } + + return writer.put(new ShuffleEntry( + keyBytes, secondaryKeyBytes, valueBytes)); + } + + @Override + public void close() throws IOException { + writer.close(); + } + } + + @Override + public SinkWriter> writer() throws IOException { + Preconditions.checkArgument(shuffleWriterConfig != null); + return writer(new ChunkingShuffleEntryWriter(new ApplianceShuffleWriter( + shuffleWriterConfig, SHUFFLE_WRITER_BUFFER_SIZE))); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkFactory.java new file mode 100644 index 0000000000000..6db9945eb6135 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkFactory.java @@ -0,0 +1,55 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Base64.decodeBase64; +import static com.google.cloud.dataflow.sdk.runners.worker.ShuffleSink.parseShuffleKind; +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.WindowedValue; + +/** + * Creates a ShuffleSink from a CloudObject spec. + */ +public class ShuffleSinkFactory { + // Do not instantiate. + private ShuffleSinkFactory() {} + + public static ShuffleSink create(PipelineOptions options, + CloudObject spec, + Coder> coder, + ExecutionContext executionContext) + throws Exception { + return create(options, spec, coder); + } + + static ShuffleSink create(PipelineOptions options, + CloudObject spec, + Coder> coder) + throws Exception { + return new ShuffleSink<>( + options, + decodeBase64(getString(spec, PropertyNames.SHUFFLE_WRITER_CONFIG, null)), + parseShuffleKind(getString(spec, PropertyNames.SHUFFLE_KIND)), + coder); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleWriter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleWriter.java new file mode 100644 index 0000000000000..ff880fd13c4c1 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleWriter.java @@ -0,0 +1,37 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import java.io.IOException; + +/** + * ShuffleWriter writes chunks of records to a shuffle dataset. + */ +interface ShuffleWriter extends AutoCloseable { + /** + * Writes a chunk of records. The chunk is a sequence of pairs encoded as: + * + * where the sizes are 4-byte big-endian integers. + */ + public void write(byte[] chunk) throws IOException; + + /** + * Flushes written records and closes this writer. + */ + @Override + public void close() throws IOException; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtils.java new file mode 100644 index 0000000000000..f3fc1cf3f3ef3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtils.java @@ -0,0 +1,211 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.api.services.dataflow.model.SideInputInfo; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; + +/** + * Utilities for working with side inputs. + */ +public class SideInputUtils { + static final String SINGLETON_KIND = "singleton"; + static final String COLLECTION_KIND = "collection"; + + /** + * Reads the given side input, producing the contents associated + * with a a {@link PCollectionView}. + */ + public static Object readSideInput(PipelineOptions options, + SideInputInfo sideInputInfo, + ExecutionContext executionContext) + throws Exception { + Iterable elements = + readSideInputSources(options, sideInputInfo.getSources(), executionContext); + return readSideInputValue(sideInputInfo.getKind(), elements); + } + + static Iterable readSideInputSources( + PipelineOptions options, + List sideInputSources, + ExecutionContext executionContext) + throws Exception { + int numSideInputSources = sideInputSources.size(); + if (numSideInputSources == 0) { + throw new Exception("expecting at least one side input Source"); + } else if (numSideInputSources == 1) { + return readSideInputSource(options, sideInputSources.get(0), executionContext); + } else { + List> shards = new ArrayList<>(); + for (com.google.api.services.dataflow.model.Source sideInputSource + : sideInputSources) { + shards.add(readSideInputSource(options, sideInputSource, executionContext)); + } + return new ShardedIterable<>(shards); + } + } + + static Iterable readSideInputSource( + PipelineOptions options, + com.google.api.services.dataflow.model.Source sideInputSource, + ExecutionContext executionContext) + throws Exception { + return new SourceIterable<>( + SourceFactory.create(options, sideInputSource, executionContext)); + } + + static Object readSideInputValue(Map sideInputKind, + Iterable elements) + throws Exception { + String className = getString(sideInputKind, PropertyNames.OBJECT_TYPE_NAME); + if (SINGLETON_KIND.equals(className)) { + Iterator iter = elements.iterator(); + if (iter.hasNext()) { + Object elem = iter.next(); + if (!iter.hasNext()) { + return elem; + } + } + throw new Exception( + "expecting a singleton side input to have a single value"); + + } else if (COLLECTION_KIND.equals(className)) { + return elements; + + } else { + throw new Exception("unexpected kind of side input: " + className); + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + + static class SourceIterable implements Iterable { + final Source source; + + public SourceIterable(Source source) { + this.source = source; + } + + @Override + public Iterator iterator() { + try { + return new SourceIterator<>(source.iterator()); + } catch (Exception exn) { + throw new RuntimeException(exn); + } + } + } + + static class SourceIterator implements Iterator { + final Source.SourceIterator iterator; + + public SourceIterator(Source.SourceIterator iterator) { + this.iterator = iterator; + } + + @Override + public boolean hasNext() { + try { + return iterator.hasNext(); + } catch (Exception exn) { + throw new RuntimeException(exn); + } + } + + @Override + public T next() { + try { + return iterator.next(); + } catch (Exception exn) { + throw new RuntimeException(exn); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + static class ShardedIterable implements Iterable { + final List> shards; + + public ShardedIterable(List> shards) { + this.shards = shards; + } + + @Override + public Iterator iterator() { + return new ShardedIterator<>(shards.iterator()); + } + } + + static class ShardedIterator implements Iterator { + final Iterator> shards; + Iterator shard; + + public ShardedIterator(Iterator> shards) { + this.shards = shards; + this.shard = null; + } + + @Override + public boolean hasNext() { + boolean shardHasNext; + for (;;) { + shardHasNext = (shard != null && shard.hasNext()); + if (shardHasNext) { + break; + } + if (!shards.hasNext()) { + break; + } + shard = shards.next().iterator(); + } + return shardHasNext; + } + + @Override + public T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return shard.next(); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactory.java new file mode 100644 index 0000000000000..df2d5ac754281 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactory.java @@ -0,0 +1,94 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.InstanceBuilder; +import com.google.cloud.dataflow.sdk.util.Serializer; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; +import com.google.common.reflect.TypeToken; + +import java.util.HashMap; +import java.util.Map; + +/** + * Constructs a Sink from a Dataflow service protocol Sink definition. + * + * A SinkFactory concrete "subclass" should define a method with the following + * signature: + *
 {@code
+ * static SomeSinkSubclass create(PipelineOptions, CloudObject,
+ *                                   Coder, ExecutionContext);
+ * } 
+ */ +public final class SinkFactory { + // Do not instantiate. + private SinkFactory() {} + + /** + * A map from the short names of predefined sinks to their full + * factory class names. + */ + static Map predefinedSinkFactories = new HashMap<>(); + + static { + predefinedSinkFactories.put("TextSink", + TextSinkFactory.class.getName()); + predefinedSinkFactories.put("AvroSink", + AvroSinkFactory.class.getName()); + predefinedSinkFactories.put("ShuffleSink", + ShuffleSinkFactory.class.getName()); + } + + /** + * Creates a {@link Sink} from a Dataflow API Sink definition. + * + * @throws Exception if the sink could not be decoded and + * constructed + */ + public static Sink create( + PipelineOptions options, + com.google.api.services.dataflow.model.Sink cloudSink, + ExecutionContext executionContext) + throws Exception { + Coder coder = Serializer.deserialize(cloudSink.getCodec(), Coder.class); + CloudObject object = CloudObject.fromSpec(cloudSink.getSpec()); + + String className = predefinedSinkFactories.get(object.getClassName()); + if (className == null) { + className = object.getClassName(); + } + + try { + return InstanceBuilder.ofType(new TypeToken>() {}) + .fromClassName(className) + .fromFactoryMethod("create") + .withArg(PipelineOptions.class, options) + .withArg(CloudObject.class, object) + .withArg(Coder.class, coder) + .withArg(ExecutionContext.class, executionContext) + .build(); + + } catch (ClassNotFoundException exn) { + throw new Exception( + "unable to create a sink from " + cloudSink, exn); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactory.java new file mode 100644 index 0000000000000..d4726094a3ea6 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactory.java @@ -0,0 +1,113 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.CloudSourceUtils; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.InstanceBuilder; +import com.google.cloud.dataflow.sdk.util.Serializer; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.common.reflect.TypeToken; + +import java.util.HashMap; +import java.util.Map; + +import javax.annotation.Nullable; + +/** + * Constructs a Source from a Dataflow API Source definition. + * + * A SourceFactory concrete "subclass" should define a method with the following + * signature: + *
 {@code
+ * static SomeSourceSubclass create(PipelineOptions, CloudObject,
+ *                                     Coder, ExecutionContext);
+ * } 
+ */ +public final class SourceFactory { + // Do not instantiate. + private SourceFactory() {} + + /** + * A map from the short names of predefined sources to + * their full factory class names. + */ + static Map predefinedSourceFactories = new HashMap<>(); + + static { + predefinedSourceFactories.put( + "TextSource", + TextSourceFactory.class.getName()); + predefinedSourceFactories.put( + "AvroSource", + AvroSourceFactory.class.getName()); + predefinedSourceFactories.put( + "UngroupedShuffleSource", + UngroupedShuffleSourceFactory.class.getName()); + predefinedSourceFactories.put( + "PartitioningShuffleSource", + PartitioningShuffleSourceFactory.class.getName()); + predefinedSourceFactories.put( + "GroupingShuffleSource", + GroupingShuffleSourceFactory.class.getName()); + predefinedSourceFactories.put( + "InMemorySource", + InMemorySourceFactory.class.getName()); + predefinedSourceFactories.put( + "BigQuerySource", + BigQuerySourceFactory.class.getName()); + } + + /** + * Creates a Source from a Dataflow API Source definition. + * + * @throws Exception if the source could not be decoded and + * constructed + */ + public static Source create( + @Nullable PipelineOptions options, + com.google.api.services.dataflow.model.Source cloudSource, + @Nullable ExecutionContext executionContext) + throws Exception { + cloudSource = CloudSourceUtils.flattenBaseSpecs(cloudSource); + Coder coder = Serializer.deserialize(cloudSource.getCodec(), Coder.class); + CloudObject object = CloudObject.fromSpec(cloudSource.getSpec()); + + String sourceFactoryClassName = predefinedSourceFactories.get(object.getClassName()); + if (sourceFactoryClassName == null) { + sourceFactoryClassName = object.getClassName(); + } + + try { + return InstanceBuilder.ofType(new TypeToken>() {}) + .fromClassName(sourceFactoryClassName) + .fromFactoryMethod("create") + .withArg(PipelineOptions.class, options) + .withArg(CloudObject.class, object) + .withArg(Coder.class, coder) + .withArg(ExecutionContext.class, executionContext) + .build(); + + } catch (ClassNotFoundException exn) { + throw new Exception( + "unable to create a source from " + cloudSource, exn); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java new file mode 100644 index 0000000000000..2db18b2724740 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java @@ -0,0 +1,72 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudSourceOperationRequestToSourceOperationRequest; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceOperationResponseToCloudSourceOperationResponse; + +import com.google.api.services.dataflow.model.Source; +import com.google.api.services.dataflow.model.SourceOperationRequest; +import com.google.api.services.dataflow.model.SourceOperationResponse; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.worker.MapTaskExecutor; +import com.google.cloud.dataflow.sdk.util.common.worker.WorkExecutor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An executor for a source operation, defined by a {@code SourceOperationRequest}. + */ +public class SourceOperationExecutor extends WorkExecutor { + private static final Logger LOG = LoggerFactory.getLogger(MapTaskExecutor.class); + + private final SourceOperationRequest request; + private SourceOperationResponse response; + + public SourceOperationExecutor(SourceOperationRequest request, + CounterSet counters) { + super(counters); + this.request = request; + } + + @Override + public void execute() throws Exception { + LOG.debug("Executing source operation"); + + Source sourceSpec; + if (request.getGetMetadata() != null) { + sourceSpec = request.getGetMetadata().getSource(); + } else if (request.getSplit() != null) { + sourceSpec = request.getSplit().getSource(); + } else { + throw new UnsupportedOperationException("Unknown source operation"); + } + + this.response = + sourceOperationResponseToCloudSourceOperationResponse( + CustomSourceFormatFactory.create(sourceSpec) + .performSourceOperation( + cloudSourceOperationRequestToSourceOperationRequest(request))); + + LOG.debug("Source operation execution complete"); + } + + public SourceOperationResponse getResponse() { + return response; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutorFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutorFactory.java new file mode 100644 index 0000000000000..10c862e464875 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutorFactory.java @@ -0,0 +1,31 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.api.services.dataflow.model.SourceOperationRequest; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +/** + * Creates a SourceOperationExecutor from a SourceOperation. + */ +public class SourceOperationExecutorFactory { + public static SourceOperationExecutor create(SourceOperationRequest request) + throws Exception { + CounterSet counters = new CounterSet(); + return new SourceOperationExecutor(request, counters); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java new file mode 100644 index 0000000000000..1e0c8aa234918 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java @@ -0,0 +1,189 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean; +import static com.google.cloud.dataflow.sdk.util.Structs.addDictionary; +import static com.google.cloud.dataflow.sdk.util.Structs.addLong; +import static com.google.cloud.dataflow.sdk.util.Structs.getDictionary; + +import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.Position; +import com.google.api.services.dataflow.model.SourceMetadata; +import com.google.api.services.dataflow.model.SourceOperationRequest; +import com.google.api.services.dataflow.model.SourceOperationResponse; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.common.worker.CustomSourceFormat; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import java.util.HashMap; +import java.util.Map; + +import javax.annotation.Nullable; + +/** + * Utilities for representing Source-specific objects + * using Dataflow model protos. + */ +public class SourceTranslationUtils { + public static Source.Progress cloudProgressToSourceProgress( + @Nullable ApproximateProgress cloudProgress) { + return cloudProgress == null ? null + : new DataflowSourceProgress(cloudProgress); + } + + public static Source.Position cloudPositionToSourcePosition( + @Nullable Position cloudPosition) { + return cloudPosition == null ? null + : new DataflowSourcePosition(cloudPosition); + } + + public static CustomSourceFormat.SourceOperationRequest + cloudSourceOperationRequestToSourceOperationRequest( + @Nullable SourceOperationRequest request) { + return request == null ? null + : new DataflowSourceOperationRequest(request); + } + + public static CustomSourceFormat.SourceOperationResponse + cloudSourceOperationResponseToSourceOperationResponse( + @Nullable SourceOperationResponse response) { + return response == null ? null + : new DataflowSourceOperationResponse(response); + } + + public static CustomSourceFormat.SourceSpec cloudSourceToSourceSpec( + @Nullable com.google.api.services.dataflow.model.Source cloudSource) { + return cloudSource == null ? null + : new DataflowSourceSpec(cloudSource); + } + + public static ApproximateProgress sourceProgressToCloudProgress( + @Nullable Source.Progress sourceProgress) { + return sourceProgress == null ? null + : ((DataflowSourceProgress) sourceProgress).cloudProgress; + } + + public static Position sourcePositionToCloudPosition( + @Nullable Source.Position sourcePosition) { + return sourcePosition == null ? null + : ((DataflowSourcePosition) sourcePosition).cloudPosition; + } + + public static SourceOperationRequest + sourceOperationRequestToCloudSourceOperationRequest( + @Nullable CustomSourceFormat.SourceOperationRequest request) { + return (request == null) ? null + : ((DataflowSourceOperationRequest) request).cloudRequest; + } + + public static SourceOperationResponse + sourceOperationResponseToCloudSourceOperationResponse( + @Nullable CustomSourceFormat.SourceOperationResponse response) { + return (response == null) ? null + : ((DataflowSourceOperationResponse) response).cloudResponse; + } + + public static com.google.api.services.dataflow.model.Source sourceSpecToCloudSource( + @Nullable CustomSourceFormat.SourceSpec spec) { + return (spec == null) ? null + : ((DataflowSourceSpec) spec).cloudSource; + } + + static class DataflowSourceProgress implements Source.Progress { + public final ApproximateProgress cloudProgress; + public DataflowSourceProgress(ApproximateProgress cloudProgress) { + this.cloudProgress = cloudProgress; + } + } + + static class DataflowSourcePosition implements Source.Position { + public final Position cloudPosition; + public DataflowSourcePosition(Position cloudPosition) { + this.cloudPosition = cloudPosition; + } + } + + static class DataflowSourceOperationRequest implements CustomSourceFormat.SourceOperationRequest { + public final SourceOperationRequest cloudRequest; + public DataflowSourceOperationRequest(SourceOperationRequest cloudRequest) { + this.cloudRequest = cloudRequest; + } + } + + static class DataflowSourceOperationResponse + implements CustomSourceFormat.SourceOperationResponse { + public final SourceOperationResponse cloudResponse; + public DataflowSourceOperationResponse(SourceOperationResponse cloudResponse) { + this.cloudResponse = cloudResponse; + } + } + + static class DataflowSourceSpec implements CustomSourceFormat.SourceSpec { + public final com.google.api.services.dataflow.model.Source cloudSource; + public DataflowSourceSpec(com.google.api.services.dataflow.model.Source cloudSource) { + this.cloudSource = cloudSource; + } + } + + // Represents a cloud Source as a dictionary for encoding inside the CUSTOM_SOURCE + // property of CloudWorkflowStep.input. + public static Map cloudSourceToDictionary( + com.google.api.services.dataflow.model.Source source) { + // Do not translate encoding - the source's encoding is translated elsewhere + // to the step's output info. + Map res = new HashMap<>(); + addDictionary(res, PropertyNames.CUSTOM_SOURCE_SPEC, source.getSpec()); + if (source.getMetadata() != null) { + addDictionary(res, PropertyNames.CUSTOM_SOURCE_METADATA, + cloudSourceMetadataToDictionary(source.getMetadata())); + } + if (source.getDoesNotNeedSplitting() != null) { + addBoolean(res, PropertyNames.CUSTOM_SOURCE_DOES_NOT_NEED_SPLITTING, + source.getDoesNotNeedSplitting()); + } + return res; + } + + private static Map cloudSourceMetadataToDictionary( + SourceMetadata metadata) { + Map res = new HashMap<>(); + if (metadata.getProducesSortedKeys() != null) { + addBoolean(res, PropertyNames.CUSTOM_SOURCE_PRODUCES_SORTED_KEYS, + metadata.getProducesSortedKeys()); + } + if (metadata.getEstimatedSizeBytes() != null) { + addLong(res, PropertyNames.CUSTOM_SOURCE_ESTIMATED_SIZE_BYTES, + metadata.getEstimatedSizeBytes()); + } + if (metadata.getInfinite() != null) { + addBoolean(res, PropertyNames.CUSTOM_SOURCE_IS_INFINITE, + metadata.getInfinite()); + } + return res; + } + + public static com.google.api.services.dataflow.model.Source dictionaryToCloudSource( + Map params) throws Exception { + com.google.api.services.dataflow.model.Source res = + new com.google.api.services.dataflow.model.Source(); + res.setSpec(getDictionary(params, PropertyNames.CUSTOM_SOURCE_SPEC)); + // CUSTOM_SOURCE_METADATA and CUSTOM_SOURCE_DOES_NOT_NEED_SPLITTING do not have to be + // translated, because they only make sense in cloud Source objects produced by the user. + return res; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSink.java new file mode 100644 index 0000000000000..5fef80f725131 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSink.java @@ -0,0 +1,285 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.MimeTypes; +import com.google.cloud.dataflow.sdk.util.ShardingWritableByteChannel; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.Random; + +import javax.annotation.Nullable; + +/** + * A sink that writes text files. + * + * @param the type of the elements written to the sink + */ +public class TextSink extends Sink { + + static final byte[] NEWLINE = getNewline(); + + private static byte[] getNewline() { + String newline = "\n"; + try { + return newline.getBytes("UTF-8"); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException("UTF-8 not supported", e); + } + } + + final String namePrefix; + final String shardFormat; + final String nameSuffix; + final int shardCount; + final boolean appendTrailingNewlines; + final String header; + final String footer; + final Coder coder; + + /** + * For testing only. + * + *

Used by simple tests which write to a single unsharded file. + */ + public static TextSink> createForTest( + String filename, + boolean appendTrailingNewlines, + @Nullable String header, + @Nullable String footer, + Coder coder) { + return create(filename, + "", + "", + 1, + appendTrailingNewlines, + header, + footer, + WindowedValue.getValueOnlyCoder(coder)); + } + + /** + * For DirectPipelineRunner only. + * It wraps the coder with {@code WindowedValue.ValueOnlyCoder}. + */ + public static TextSink> createForDirectPipelineRunner( + String filenamePrefix, + String shardFormat, + String filenameSuffix, + int shardCount, + boolean appendTrailingNewlines, + @Nullable String header, + @Nullable String footer, + Coder coder) { + return create(filenamePrefix, + shardFormat, + filenameSuffix, + shardCount, + appendTrailingNewlines, + header, + footer, + WindowedValue.getValueOnlyCoder(coder)); + } + + /** + * Constructs a new TextSink. + * + * @param filenamePrefix the prefix of output filenames. + * @param shardFormat the shard name template to use for output filenames. + * @param filenameSuffix the suffix of output filenames. + * @param shardCount the number of outupt shards to produce. + * @param appendTrailingNewlines true to append newlines to each output line. + * @param header text to place at the beginning of each output file. + * @param footer text to place at the end of each output file. + * @param coder the code used to encode elements for output. + */ + public static TextSink create(String filenamePrefix, + String shardFormat, + String filenameSuffix, + int shardCount, + boolean appendTrailingNewlines, + @Nullable String header, + @Nullable String footer, + Coder coder) { + return new TextSink<>(filenamePrefix, + shardFormat, + filenameSuffix, + shardCount, + appendTrailingNewlines, + header, + footer, + coder); + } + + private TextSink(String filenamePrefix, + String shardFormat, + String filenameSuffix, + int shardCount, + boolean appendTrailingNewlines, + @Nullable String header, + @Nullable String footer, + Coder coder) { + this.namePrefix = filenamePrefix; + this.shardFormat = shardFormat; + this.nameSuffix = filenameSuffix; + this.shardCount = shardCount; + this.appendTrailingNewlines = appendTrailingNewlines; + this.header = header; + this.footer = footer; + this.coder = coder; + } + + @Override + public SinkWriter writer() throws IOException { + String mimeType; + + if (!(coder instanceof WindowedValueCoder)) { + throw new IOException( + "Expected WindowedValueCoder for inputCoder, got: " + + coder.getClass().getName()); + } + Coder valueCoder = ((WindowedValueCoder) coder).getValueCoder(); + if (valueCoder.equals(StringUtf8Coder.of())) { + mimeType = MimeTypes.TEXT; + } else { + mimeType = MimeTypes.BINARY; + } + + WritableByteChannel writer = IOChannelUtils.create(namePrefix, shardFormat, + nameSuffix, shardCount, mimeType); + + if (writer instanceof ShardingWritableByteChannel) { + return new ShardingTextFileWriter((ShardingWritableByteChannel) writer); + } else { + return new TextFileWriter(writer); + } + } + + /** + * Abstract SinkWriter base class shared by sharded and unsharded Text + * writer implementations. + */ + abstract class AbstractTextFileWriter implements SinkWriter { + protected void init() throws IOException { + if (header != null) { + printLine(ShardingWritableByteChannel.ALL_SHARDS, + CoderUtils.encodeToByteArray(StringUtf8Coder.of(), header)); + } + } + + /** + * Adds a value to the sink. Returns the size in bytes of the data written. + * The return value does -not- include header/footer size. + */ + @Override + public long add(T value) throws IOException { + return printLine(getShardNum(value), + CoderUtils.encodeToByteArray(coder, value)); + } + + @Override + public void close() throws IOException { + if (footer != null) { + printLine(ShardingWritableByteChannel.ALL_SHARDS, + CoderUtils.encodeToByteArray(StringUtf8Coder.of(), footer)); + } + } + + protected long printLine(int shardNum, byte[] line) throws IOException { + long length = line.length; + write(shardNum, ByteBuffer.wrap(line)); + + if (appendTrailingNewlines) { + write(shardNum, ByteBuffer.wrap(NEWLINE)); + length += NEWLINE.length; + } + + return length; + } + + protected abstract void write(int shardNum, ByteBuffer buf) + throws IOException; + protected abstract int getShardNum(T value); + } + + /** An unsharded SinkWriter for a TextSink. */ + class TextFileWriter extends AbstractTextFileWriter { + private final WritableByteChannel outputChannel; + + TextFileWriter(WritableByteChannel outputChannel) throws IOException { + this.outputChannel = outputChannel; + init(); + } + + @Override + public void close() throws IOException { + super.close(); + outputChannel.close(); + } + + @Override + protected void write(int shardNum, ByteBuffer buf) throws IOException { + outputChannel.write(buf); + } + + @Override + protected int getShardNum(T value) { + return 0; + } + } + + /** A sharding SinkWriter for a TextSink. */ + class ShardingTextFileWriter extends AbstractTextFileWriter { + private final Random rng = new Random(); + private final int numShards; + private final ShardingWritableByteChannel outputChannel; + + // TODO: add support for user-defined sharding function. + ShardingTextFileWriter(ShardingWritableByteChannel outputChannel) + throws IOException { + this.outputChannel = outputChannel; + numShards = outputChannel.getNumShards(); + init(); + } + + @Override + public void close() throws IOException { + super.close(); + outputChannel.close(); + } + + @Override + protected void write(int shardNum, ByteBuffer buf) throws IOException { + outputChannel.writeToShard(shardNum, buf); + } + + @Override + protected int getShardNum(T value) { + return rng.nextInt(numShards); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkFactory.java new file mode 100644 index 0000000000000..bac663dea2da5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkFactory.java @@ -0,0 +1,55 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getBoolean; +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; + +/** + * Creates a TextSink from a CloudObject spec. + */ +public final class TextSinkFactory { + // Do not instantiate. + private TextSinkFactory() {} + + public static TextSink create(PipelineOptions options, + CloudObject spec, + Coder coder, + ExecutionContext executionContext) + throws Exception { + return create(spec, coder); + } + + static TextSink create(CloudObject spec, Coder coder) + throws Exception { + return TextSink.create( + getString(spec, PropertyNames.FILENAME), + "", // No shard template + "", // No suffix + 1, // Exactly one output file + getBoolean(spec, PropertyNames.APPEND_TRAILING_NEWLINES, true), + getString(spec, PropertyNames.HEADER, null), + getString(spec, PropertyNames.FOOTER, null), + coder); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSource.java new file mode 100644 index 0000000000000..5bbcba0e6b91d --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSource.java @@ -0,0 +1,383 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.util.IOChannelFactory; +import com.google.cloud.dataflow.sdk.util.common.worker.ProgressTracker; +import com.google.cloud.dataflow.sdk.util.common.worker.ProgressTrackerGroup; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.PushbackInputStream; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.SeekableByteChannel; +import java.util.Collection; +import java.util.Iterator; + +import javax.annotation.Nullable; + +/** + * A source that reads text files. + * + * @param the type of the elements read from the source + */ +public class TextSource extends FileBasedSource { + final boolean stripTrailingNewlines; + + public TextSource(String filename, + boolean stripTrailingNewlines, + @Nullable Long startPosition, + @Nullable Long endPosition, + Coder coder) { + this(filename, stripTrailingNewlines, + startPosition, endPosition, coder, true); + } + + protected TextSource(String filename, + boolean stripTrailingNewlines, + @Nullable Long startPosition, + @Nullable Long endPosition, + Coder coder, + boolean useDefaultBufferSize) { + super(filename, startPosition, endPosition, coder, useDefaultBufferSize); + this.stripTrailingNewlines = stripTrailingNewlines; + } + + @Override + protected SourceIterator newSourceIteratorForRangeInFile( + IOChannelFactory factory, String oneFile, long startPosition, + @Nullable Long endPosition) + throws IOException { + // Position before the first record, so we can find the record beginning. + final long start = startPosition > 0 ? startPosition - 1 : 0; + + TextFileIterator iterator = newSourceIteratorForRangeWithStrictStart( + factory, oneFile, stripTrailingNewlines, start, endPosition); + + // Skip the initial record if start position was set. + if (startPosition > 0 && iterator.hasNext()) { + iterator.advance(); + } + + return iterator; + } + + @Override + protected SourceIterator newSourceIteratorForFiles( + IOChannelFactory factory, Collection files) throws IOException { + if (files.size() == 1) { + return newSourceIteratorForFile( + factory, files.iterator().next(), stripTrailingNewlines); + } + + return new TextFileMultiIterator( + factory, files.iterator(), stripTrailingNewlines); + } + + private TextFileIterator newSourceIteratorForFile( + IOChannelFactory factory, String input, boolean stripTrailingNewlines) + throws IOException { + return newSourceIteratorForRangeWithStrictStart( + factory, input, stripTrailingNewlines, 0, null); + } + + /** + * Returns a new iterator for lines in the given range in the given + * file. Does NOT skip the first line if the range starts in the + * middle of a line (instead, the latter half that starts at + * startOffset will be returned as the first element). + */ + private TextFileIterator newSourceIteratorForRangeWithStrictStart( + IOChannelFactory factory, String input, boolean stripTrailingNewlines, + long startOffset, @Nullable Long endOffset) throws IOException { + ReadableByteChannel reader = factory.open(input); + if (!(reader instanceof SeekableByteChannel)) { + throw new UnsupportedOperationException( + "Unable to seek in stream for " + input); + } + + SeekableByteChannel seeker = (SeekableByteChannel) reader; + + return new TextFileIterator( + new CopyableSeekableByteChannel(seeker), + stripTrailingNewlines, startOffset, endOffset); + } + + class TextFileMultiIterator extends LazyMultiSourceIterator { + private final IOChannelFactory factory; + private final boolean stripTrailingNewlines; + + public TextFileMultiIterator(IOChannelFactory factory, + Iterator inputs, boolean stripTrailingNewlines) { + super(inputs); + this.factory = factory; + this.stripTrailingNewlines = stripTrailingNewlines; + } + + @Override + protected SourceIterator open(String input) throws IOException { + return newSourceIteratorForFile(factory, input, stripTrailingNewlines); + } + } + + class TextFileIterator extends FileBasedIterator { + private final boolean stripTrailingNewlines; + private ScanState state; + + TextFileIterator(CopyableSeekableByteChannel seeker, + boolean stripTrailingNewlines, + long startOffset, + @Nullable Long endOffset) throws IOException { + this(seeker, stripTrailingNewlines, startOffset, startOffset, endOffset, + new ProgressTrackerGroup() { + @Override + protected void report(Integer lineLength) { + notifyElementRead(lineLength.longValue()); + } + }.start(), new ScanState(BUF_SIZE, !stripTrailingNewlines)); + } + + private TextFileIterator(CopyableSeekableByteChannel seeker, + boolean stripTrailingNewlines, + long startOffset, + long offset, + @Nullable Long endOffset, + ProgressTracker tracker, + ScanState state) throws IOException { + super(seeker, startOffset, offset, endOffset, tracker); + + this.stripTrailingNewlines = stripTrailingNewlines; + this.state = state; + } + + private TextFileIterator(TextFileIterator it) throws IOException { + this(it.seeker.copy(), it.stripTrailingNewlines, + /* Correctly adjust the start position of the seeker given + * that it may hold bytes that have been read and now reside + * in the read buffer (that is copied during cloning) */ + it.startOffset + it.state.totalBytesRead, + it.offset, + it.endOffset, it.tracker.copy(), it.state.copy()); + } + + @Override + public SourceIterator copy() throws IOException { + return new TextFileIterator(this); + } + + /** + * Reads a line of text. A line is considered to be terminated by any + * one of a line feed ({@code '\n'}), a carriage return + * ({@code '\r'}), or a carriage return followed immediately by a linefeed + * ({@code "\r\n"}). + * + * @return a {@code ByteArrayOutputStream} containing the contents of the + * line, with any line-termination characters stripped if + * keepNewlines==false, or {@code null} if the end of the stream has + * been reached. + * @throws IOException if an I/O error occurs + */ + @Override + protected ByteArrayOutputStream readElement() + throws IOException { + ByteArrayOutputStream buffer = new ByteArrayOutputStream(BUF_SIZE); + + int charsConsumed = 0; + while (true) { + // Attempt to read blocks of data at a time + // until a separator is found. + if (!state.readBytes(stream)) { + break; + } + + int consumed = state.consumeUntilSeparator(buffer); + charsConsumed += consumed; + if (consumed > 0 && state.separatorFound()) { + if (state.lastByteRead() == '\r') { + charsConsumed += state.copyCharIfLinefeed(buffer, stream); + } + break; + } + } + + if (charsConsumed == 0) { + // Note that charsConsumed includes the size of any separators that may + // have been stripped off -- so if we didn't get anything, we're at the + // end of the file. + return null; + } + + offset += charsConsumed; + tracker.saw(charsConsumed); + return buffer; + } + } + + /** + * ScanState encapsulates the state for the current buffer of text + * being scanned. + */ + private static class ScanState { + private int start; // Valid bytes in buf start at this index + private int pos; // Where the separator is in the buf (if one was found) + private int end; // the index of the end of bytes in buf + private byte[] buf; + private boolean keepNewlines; + private byte lastByteRead; + private long totalBytesRead; + + public ScanState(int size, boolean keepNewlines) { + this.start = 0; + this.pos = 0; + this.end = 0; + this.buf = new byte[size]; + this.keepNewlines = keepNewlines; + totalBytesRead = 0; + } + + public ScanState copy() { + byte[] bufCopy = new byte[buf.length]; // copy :( + System.arraycopy(buf, start, bufCopy, start, end - start); + return new ScanState( + this.keepNewlines, this.start, this.pos, this.end, + bufCopy, this.lastByteRead, 0); + } + + private ScanState( + boolean keepNewlines, int start, int pos, int end, + byte[] buf, byte lastByteRead, long totalBytesRead) { + this.start = start; + this.pos = pos; + this.end = end; + this.buf = buf; + this.keepNewlines = keepNewlines; + this.lastByteRead = lastByteRead; + this.totalBytesRead = totalBytesRead; + } + + public boolean readBytes(PushbackInputStream stream) throws IOException { + if (start < end) { + return true; + } + assert end <= buf.length : end + " > " + buf.length; + int bytesRead = stream.read(buf, end, buf.length - end); + if (bytesRead == -1) { + return false; + } + totalBytesRead += bytesRead; + end += bytesRead; + return true; + } + + /** + * Consumes characters until a separator character is found or the + * end of buffer is reached. + * + * Updates the state to indicate the position of the separator + * character. If pos==len, no separator was found. + * + * @return the number of characters consumed. + */ + public int consumeUntilSeparator(ByteArrayOutputStream out) { + for (pos = start; pos < end; ++pos) { + lastByteRead = buf[pos]; + if (separatorFound()) { + int charsConsumed = (pos - start + 1); // The separator is consumed + copyToOutputBuffer(out); + start = pos + 1; // skip the separator + return charsConsumed; + } + } + // No separator found + assert pos == end; + int charsConsumed = (pos - start); + out.write(buf, start, charsConsumed); + start = 0; + end = 0; + pos = 0; + return charsConsumed; + } + + public boolean separatorFound() { + return lastByteRead == '\n' || lastByteRead == '\r'; + } + + public byte lastByteRead() { + return buf[pos]; + } + + public int bytesBuffered() { + assert end >= start : end + " must be >= " + start; + return end - start; + } + + /** + * Copies data from the input buffer to the output buffer. + * + * If keepNewlines==true, line-termination characters are included in the copy. + */ + private void copyToOutputBuffer(ByteArrayOutputStream out) { + int charsCopied = pos - start; + if (keepNewlines && separatorFound()) { + charsCopied++; + } + out.write(buf, start, charsCopied); + } + + /** + * Scans the input buffer to determine if a matched carriage return + * has an accompanying linefeed and process the input buffer accordingly. + * + * If keepNewlines==true and a linefeed character is detected, + * it is included in the copy. + * + * @return the number of characters consumed + */ + private int copyCharIfLinefeed(ByteArrayOutputStream out, PushbackInputStream stream) + throws IOException { + int charsConsumed = 0; + // Check to make sure we don't go off the end of the buffer + if ((pos + 1) < end) { + if (buf[pos + 1] == '\n') { + charsConsumed++; + pos++; + start++; + if (keepNewlines) { + out.write('\n'); + } + } + } else { + // We are at the end of the buffer and need one more + // byte. Get it the slow but safe way. + int b = stream.read(); + if (b == '\n') { + charsConsumed++; + totalBytesRead++; + if (keepNewlines) { + out.write(b); + } + } else if (b != -1) { + // Consider replacing unread() since it may be slow if + // iterators are cloned frequently. + stream.unread(b); + } + } + return charsConsumed; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactory.java new file mode 100644 index 0000000000000..a15c2d505c47f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactory.java @@ -0,0 +1,74 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getBoolean; +import static com.google.cloud.dataflow.sdk.util.Structs.getLong; +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.api.services.dataflow.model.Source; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.Serializer; + +/** + * Creates a TextSource from a CloudObject spec. + */ +public class TextSourceFactory { + // Do not instantiate. + private TextSourceFactory() {} + + public static TextSource create(PipelineOptions options, + CloudObject spec, + Coder coder, + ExecutionContext executionContext) + throws Exception { + return create(spec, coder); + } + + static TextSource create(CloudObject spec, + Coder coder) + throws Exception { + return create(spec, coder, true); + } + + public static TextSource create(Source spec) + throws Exception { + return create( + CloudObject.fromSpec(spec.getSpec()), + Serializer.deserialize(spec.getCodec(), Coder.class)); + } + + static TextSource create(CloudObject spec, + Coder coder, + boolean useDefaultBufferSize) throws Exception { + String filenameOrPattern = getString(spec, PropertyNames.FILENAME, null); + if (filenameOrPattern == null) { + filenameOrPattern = getString(spec, PropertyNames.FILEPATTERN, null); + } + return new TextSource<>( + filenameOrPattern, + getBoolean(spec, PropertyNames.STRIP_TRAILING_NEWLINES, true), + getLong(spec, PropertyNames.START_OFFSET, null), + getLong(spec, PropertyNames.END_OFFSET, null), + coder, + useDefaultBufferSize); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSource.java new file mode 100644 index 0000000000000..d7d0cf7cf841e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSource.java @@ -0,0 +1,96 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.BatchingShuffleEntryReader; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntryReader; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import java.io.IOException; +import java.util.Iterator; + +import javax.annotation.Nullable; + +/** + * A source that reads from a shuffled dataset, without any key grouping. + * Returns just the values. (This reader is for an UNGROUPED shuffle session.) + * + * @param the type of the elements read from the source + */ +public class UngroupedShuffleSource extends Source { + final byte[] shuffleReaderConfig; + final String startShufflePosition; + final String stopShufflePosition; + final Coder coder; + + public UngroupedShuffleSource(PipelineOptions options, + byte[] shuffleReaderConfig, + @Nullable String startShufflePosition, + @Nullable String stopShufflePosition, + Coder coder) { + this.shuffleReaderConfig = shuffleReaderConfig; + this.startShufflePosition = startShufflePosition; + this.stopShufflePosition = stopShufflePosition; + this.coder = coder; + } + + @Override + public SourceIterator iterator() throws IOException { + Preconditions.checkArgument(shuffleReaderConfig != null); + return iterator(new BatchingShuffleEntryReader( + new ChunkingShuffleBatchReader(new ApplianceShuffleReader( + shuffleReaderConfig)))); + } + + SourceIterator iterator(ShuffleEntryReader reader) throws IOException { + return new UngroupedShuffleSourceIterator(reader); + } + + /** + * A SourceIterator that reads from a ShuffleEntryReader and extracts + * just the values. + */ + class UngroupedShuffleSourceIterator extends AbstractSourceIterator { + Iterator iterator; + + UngroupedShuffleSourceIterator(ShuffleEntryReader reader) + throws IOException { + this.iterator = reader.read( + ByteArrayShufflePosition.fromBase64(startShufflePosition), + ByteArrayShufflePosition.fromBase64(stopShufflePosition)); + } + + @Override + public boolean hasNext() throws IOException { + return iterator.hasNext(); + } + + @Override + public T next() throws IOException { + ShuffleEntry record = iterator.next(); + // Throw away the primary and the secondary keys. + byte[] value = record.getValue(); + notifyElementRead(record.length()); + return CoderUtils.decodeFromByteArray(coder, value); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceFactory.java new file mode 100644 index 0000000000000..adff71226d6b8 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceFactory.java @@ -0,0 +1,56 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Base64.decodeBase64; +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; + +/** + * Creates an UngroupedShuffleSource from a CloudObject spec. + */ +public class UngroupedShuffleSourceFactory { + // Do not instantiate. + private UngroupedShuffleSourceFactory() {} + + public static UngroupedShuffleSource create( + PipelineOptions options, + CloudObject spec, + Coder coder, + ExecutionContext executionContext) + throws Exception { + return create(options, spec, coder); + } + + static UngroupedShuffleSource create( + PipelineOptions options, + CloudObject spec, + Coder coder) + throws Exception { + return new UngroupedShuffleSource<>( + options, + decodeBase64(getString(spec, PropertyNames.SHUFFLE_READER_CONFIG)), + getString(spec, PropertyNames.START_SHUFFLE_POSITION, null), + getString(spec, PropertyNames.END_SHUFFLE_POSITION, null), + coder); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java new file mode 100644 index 0000000000000..85805773c7060 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java @@ -0,0 +1,77 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker.logging; + +import static com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingInitializer.LEVELS; + +import com.google.common.base.MoreObjects; + +import org.joda.time.format.DateTimeFormatter; +import org.joda.time.format.ISODateTimeFormat; +import org.slf4j.MDC; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.logging.Formatter; +import java.util.logging.LogRecord; + +/** + * Formats {@link LogRecord} into the following format: + * ISO8601Date LogLevel JobId WorkerId WorkId ThreadId LoggerName LogMessage + * with one or more additional lines for any {@link Throwable} associated with + * the {@link LogRecord}. The exception is output using + * {@link Throwable#printStackTrace()}. + */ +public class DataflowWorkerLoggingFormatter extends Formatter { + private static final DateTimeFormatter DATE_FORMATTER = + ISODateTimeFormat.dateTime().withZoneUTC(); + public static final String MDC_DATAFLOW_JOB_ID = "dataflow.jobId"; + public static final String MDC_DATAFLOW_WORKER_ID = "dataflow.workerId"; + public static final String MDC_DATAFLOW_WORK_ID = "dataflow.workId"; + + @Override + public String format(LogRecord record) { + String exception = formatException(record.getThrown()); + return DATE_FORMATTER.print(record.getMillis()) + + " " + MoreObjects.firstNonNull(LEVELS.get(record.getLevel()), + record.getLevel().getName()) + + " " + MoreObjects.firstNonNull(MDC.get(MDC_DATAFLOW_JOB_ID), "unknown") + + " " + MoreObjects.firstNonNull(MDC.get(MDC_DATAFLOW_WORKER_ID), "unknown") + + " " + MoreObjects.firstNonNull(MDC.get(MDC_DATAFLOW_WORK_ID), "unknown") + + " " + record.getThreadID() + + " " + record.getLoggerName() + + " " + record.getMessage() + "\n" + + (exception != null ? exception : ""); + } + + /** + * Formats the throwable as per {@link Throwable#printStackTrace()}. + * + * @param thrown The throwable to format. + * @return A string containing the contents of {@link Throwable#printStackTrace()}. + */ + private String formatException(Throwable thrown) { + if (thrown == null) { + return null; + } + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + thrown.printStackTrace(pw); + pw.close(); + return sw.toString(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java new file mode 100644 index 0000000000000..80ccf7084bcbf --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java @@ -0,0 +1,88 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker.logging; + +import com.google.common.collect.ImmutableBiMap; + +import java.io.IOException; +import java.util.logging.ConsoleHandler; +import java.util.logging.FileHandler; +import java.util.logging.Formatter; +import java.util.logging.Handler; +import java.util.logging.Level; +import java.util.logging.LogManager; +import java.util.logging.Logger; + +/** + * Sets up java.util.Logging configuration on the Dataflow Worker Harness with a + * console and file logger. The console and file loggers use the + * {@link DataflowWorkerLoggingFormatter} format. A user can override + * the logging level and location by specifying the Java system properties + * "dataflow.worker.logging.level" and "dataflow.worker.logging.location" respectively. + * The default log level is INFO and the default location is a file named dataflow-worker.log + * within the systems temporary directory. + */ +public class DataflowWorkerLoggingInitializer { + private static final String DEFAULT_LOGGING_LOCATION = "/tmp/dataflow-worker.log"; + private static final String ROOT_LOGGER_NAME = ""; + public static final String DATAFLOW_WORKER_LOGGING_LEVEL = "dataflow.worker.logging.level"; + public static final String DATAFLOW_WORKER_LOGGING_LOCATION = "dataflow.worker.logging.location"; + public static final ImmutableBiMap LEVELS = + ImmutableBiMap.builder() + .put(Level.SEVERE, "ERROR") + .put(Level.WARNING, "WARNING") + .put(Level.INFO, "INFO") + .put(Level.FINE, "DEBUG") + .put(Level.FINEST, "TRACE") + .build(); + private static final String DEFAULT_LOG_LEVEL = LEVELS.get(Level.INFO); + + public void initialize() { + initialize(LogManager.getLogManager()); + } + + void initialize(LogManager logManager) { + try { + Level logLevel = LEVELS.inverse().get( + System.getProperty(DATAFLOW_WORKER_LOGGING_LEVEL, DEFAULT_LOG_LEVEL)); + Formatter formatter = new DataflowWorkerLoggingFormatter(); + + FileHandler fileHandler = new FileHandler( + System.getProperty(DATAFLOW_WORKER_LOGGING_LOCATION, DEFAULT_LOGGING_LOCATION), + true /* Append so that we don't squash existing logs */); + fileHandler.setFormatter(formatter); + fileHandler.setLevel(logLevel); + + ConsoleHandler consoleHandler = new ConsoleHandler(); + consoleHandler.setFormatter(formatter); + consoleHandler.setLevel(logLevel); + + // Reset the global log manager, get the root logger and remove the default log handlers. + logManager.reset(); + Logger rootLogger = logManager.getLogger(ROOT_LOGGER_NAME); + for (Handler handler : rootLogger.getHandlers()) { + rootLogger.removeHandler(handler); + } + + rootLogger.setLevel(logLevel); + rootLogger.addHandler(consoleHandler); + rootLogger.addHandler(fileHandler); + } catch (SecurityException | IOException e) { + throw new ExceptionInInitializerError(e); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/package-info.java new file mode 100644 index 0000000000000..615ed64743922 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/package-info.java @@ -0,0 +1,24 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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. + */ + +/** + * Implementation of the harness that runs on each Google Compute Engine instance to coordinate + * execution of Pipeline code. + */ +@ParametersAreNonnullByDefault +package com.google.cloud.dataflow.sdk.runners.worker; + +import javax.annotation.ParametersAreNonnullByDefault; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java new file mode 100644 index 0000000000000..d4fe32ffd86f3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java @@ -0,0 +1,374 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionView; + +import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Collection; + +/** + * An assertion on the contents of a {@link PCollection} + * incorporated into the pipeline. Such an assertion + * can be checked no matter what kind of + * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} is + * used, so it's good for testing using the + * {@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner}, + * the + * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}, + * etc. + * + *

Note that the {@code DataflowAssert} call must precede the call + * to {@link com.google.cloud.dataflow.sdk.Pipeline#run}. + * + *

Examples of use: + *

{@code
+ * Pipeline p = TestPipeline.create();
+ * ...
+ * PCollection output =
+ *      input
+ *      .apply(ParDo.of(new TestDoFn()));
+ * DataflowAssert.that(output)
+ *     .containsInAnyOrder("out1", "out2", "out3");
+ * ...
+ * PCollection ints = ...
+ * PCollection sum =
+ *     ints
+ *     .apply(Combine.globally(new SumInts()));
+ * DataflowAssert.that(sum)
+ *     .is(42);
+ * ...
+ * p.run();
+ * }
+ * + *

JUnit and Hamcrest must be linked in by any code that uses DataflowAssert. + * + * @param The type of elements in the input collection. + */ +public class DataflowAssert { + /** + * Constructs an IterableAssert for the elements of the provided + * {@code PCollection}. + */ + public static IterableAssert that(PCollection futureResult) { + return new IterableAssert<>(futureResult.apply(View.asIterable())); + } + + /** + * Constructs an IterableAssert for the value of the provided + * {@code PCollection>}, which must be a singleton. + */ + public static IterableAssert thatSingletonIterable( + PCollection> futureResult) { + return new IterableAssert<>(futureResult.apply(View.>asSingleton())); + } + + /** + * Constructs an IterableAssert for the value of the provided + * {@code PCollectionView, ?>}. + */ + public static IterableAssert thatIterable( + PCollectionView, ?> futureResult) { + return new IterableAssert<>(futureResult); + } + + /** + * An assertion about the contents of a {@link PCollectionView<, ?>} + */ + public static class IterableAssert implements Serializable { + private final PCollectionView, ?> actualResults; + + private IterableAssert(PCollectionView, ?> futureResult) { + actualResults = futureResult; + } + + /** + * Applies a SerializableFunction to check the elements of the Iterable. + * + *

Returns this IterableAssert. + */ + public IterableAssert satisfies( + final SerializableFunction, Void> checkerFn) { + + actualResults.getPipeline() + .apply(Create.of((Void) null)) + .setCoder(VoidCoder.of()) + .apply(ParDo + .withSideInputs(actualResults) + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + Iterable actualContents = c.sideInput(actualResults); + checkerFn.apply(actualContents); + } + })); + + return this; + } + + /** + * Checks that the Iterable contains the expected elements, in any + * order. + * + *

Returns this IterableAssert. + */ + public IterableAssert containsInAnyOrder(T... expectedElements) { + return this.satisfies(new AssertContainsInAnyOrder(expectedElements)); + } + + /** + * Checks that the Iterable contains the expected elements, in any + * order. + * + *

Returns this IterableAssert. + */ + public IterableAssert containsInAnyOrder( + Collection expectedElements) { + return this.satisfies(new AssertContainsInAnyOrder(expectedElements)); + } + + /** + * Checks that the Iterable contains the expected elements, in the + * specified order. + * + *

Returns this IterableAssert. + */ + public IterableAssert containsInOrder(T... expectedElements) { + return this.satisfies(new AssertContainsInOrder(expectedElements)); + } + + /** + * Checks that the Iterable contains the expected elements, in the + * specified order. + * + *

Returns this IterableAssert. + */ + public IterableAssert containsInOrder(Collection expectedElements) { + return this.satisfies(new AssertContainsInOrder(expectedElements)); + } + + /** + * SerializableFunction that performs an {@code Assert.assertThat()} + * operation using a {@code Matcher} operation that takes an array + * of elements. + */ + static class AssertThatIterable extends AssertThat, T[]> { + AssertThatIterable(T[] expected, + String matcherClassName, + String matcherFactoryMethodName) { + super(expected, Object[].class, + matcherClassName, matcherFactoryMethodName); + } + } + + /** + * SerializableFunction that verifies that an Iterable contains + * expected items in any order. + */ + static class AssertContainsInAnyOrder extends AssertThatIterable { + AssertContainsInAnyOrder(T... expected) { + super(expected, + "org.hamcrest.collection.IsIterableContainingInAnyOrder", + "containsInAnyOrder"); + } + @SuppressWarnings("unchecked") + AssertContainsInAnyOrder(Collection expected) { + this((T[]) expected.toArray()); + } + } + + /** + * SerializableFunction that verifies that an Iterable contains + * expected items in the provided order. + */ + static class AssertContainsInOrder extends AssertThatIterable { + AssertContainsInOrder(T... expected) { + super(expected, + "org.hamcrest.collection.IsIterableContainingInOrder", + "contains"); + } + @SuppressWarnings("unchecked") + AssertContainsInOrder(Collection expected) { + this((T[]) expected.toArray()); + } + } + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Constructs a SingletonAssert for the value of the provided + * {@code PCollection}, which must be a singleton. + */ + public static SingletonAssert thatSingleton(PCollection futureResult) { + return new SingletonAssert<>(futureResult.apply(View.asSingleton())); + } + + /** + * An assertion about a single value. + */ + public static class SingletonAssert implements Serializable { + private final PCollectionView actualResult; + + private SingletonAssert(PCollectionView futureResult) { + actualResult = futureResult; + } + + /** + * Applies a SerializableFunction to check the value of this + * SingletonAssert's view. + * + *

Returns this SingletonAssert. + */ + public SingletonAssert satisfies(final SerializableFunction checkerFn) { + actualResult.getPipeline() + .apply(Create.of((Void) null)) + .setCoder(VoidCoder.of()) + .apply(ParDo + .withSideInputs(actualResult) + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + T actualContents = c.sideInput(actualResult); + checkerFn.apply(actualContents); + } + })); + + return this; + } + + /** + * Checks that the value of this SingletonAssert's view is equal + * to the expected value. + * + *

Returns this SingletonAssert. + */ + public SingletonAssert is(T expectedValue) { + return this.satisfies(new AssertIs(expectedValue)); + } + + /** + * SerializableFunction that performs an {@code Assert.assertThat()} + * operation using a {@code Matcher} operation that takes a single element. + */ + static class AssertThatValue extends AssertThat { + AssertThatValue(T expected, + String matcherClassName, + String matcherFactoryMethodName) { + super(expected, Object.class, + matcherClassName, matcherFactoryMethodName); + } + } + + /** + * SerializableFunction that verifies that a value is equal to an + * expected value. + */ + public static class AssertIs extends AssertThatValue { + AssertIs(T expected) { + super(expected, "org.hamcrest.core.IsEqual", "equalTo"); + } + } + } + + ///////////////////////////////////////////////////////////////////////////// + + + // Do not instantiate. + private DataflowAssert() {} + + /** + * SerializableFunction that performs an {@code Assert.assertThat()} + * operation using a {@code Matcher} operation. + * + *

The MatcherFactory should take an {@code Expected} and + * produce a Matcher to be used to check an {@code Actual} value + * against. + */ + public static class AssertThat + implements SerializableFunction { + final Expected expected; + final Class expectedClass; + final String matcherClassName; + final String matcherFactoryMethodName; + + AssertThat(Expected expected, + Class expectedClass, + String matcherClassName, + String matcherFactoryMethodName) { + this.expected = expected; + this.expectedClass = expectedClass; + this.matcherClassName = matcherClassName; + this.matcherFactoryMethodName = matcherFactoryMethodName; + } + + @Override + public Void apply(Actual in) { + try { + Method matcherFactoryMethod = Class.forName(this.matcherClassName) + .getMethod(this.matcherFactoryMethodName, expectedClass); + Object matcher = matcherFactoryMethod.invoke(null, (Object) expected); + Method assertThatMethod = Class.forName("org.junit.Assert") + .getMethod("assertThat", + Object.class, + Class.forName("org.hamcrest.Matcher")); + assertThatMethod.invoke(null, in, matcher); + } catch (InvocationTargetException e) { + // An error in the assertThat or matcher itself. + throw new RuntimeException(e); + } catch (ReflectiveOperationException e) { + // An error looking up the classes and methods. + throw new RuntimeException( + "DataflowAssert requires that JUnit and Hamcrest be linked in.", + e); + } + return null; + } + } + + /** + * SerializableFunction that performs an {@code Assert.assertThat()} + * operation using a {@code Matcher} operation that takes a single element. + */ + static class AssertThatValue extends AssertThat { + AssertThatValue(T expected, + String matcherClassName, + String matcherFactoryMethodName) { + super(expected, Object.class, + matcherClassName, matcherFactoryMethodName); + } + } + + /** + * SerializableFunction that verifies that a value is equal to an + * expected value. + */ + public static class AssertIs extends AssertThatValue { + public AssertIs(T expected) { + super(expected, "org.hamcrest.core.IsEqual", "equalTo"); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/RunnableOnService.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/RunnableOnService.java new file mode 100644 index 0000000000000..048ea36a25338 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/RunnableOnService.java @@ -0,0 +1,29 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +/** + * Category tag for tests that can be run on the DataflowPipelineRunner if the + * runIntegrationTestOnService System property is set to true. + * Example usage: + *


+ *     {@literal @}Test
+ *     {@literal @}Category(RunnableOnService.class)
+ *     public void testParDo() {...
+ * 
+ */ +public interface RunnableOnService {} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineOptions.java new file mode 100644 index 0000000000000..e9f8f828120fd --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineOptions.java @@ -0,0 +1,26 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions; + +/** + * A set of options used to configure the {@link TestPipeline}. + */ +public interface TestDataflowPipelineOptions extends BlockingDataflowPipelineOptions { + +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineRunner.java new file mode 100644 index 0000000000000..96da50189a905 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestDataflowPipelineRunner.java @@ -0,0 +1,45 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner; +import com.google.cloud.dataflow.sdk.util.MonitoringUtil; + +/** + * TestDataflowPipelineRunner is a pipeline runner that wraps a + * DataflowPipelineRunner when running tests against the {@link TestPipeline}. + * + * @see TestPipeline + */ +public class TestDataflowPipelineRunner extends BlockingDataflowPipelineRunner { + TestDataflowPipelineRunner( + DataflowPipelineRunner internalRunner, + MonitoringUtil.JobMessagesHandler jobMessagesHandler) { + super(internalRunner, jobMessagesHandler); + } + + @Override + public PipelineJobState run(Pipeline pipeline) { + PipelineJobState state = super.run(pipeline); + if (state.getJobState() != MonitoringUtil.JobState.DONE) { + throw new AssertionError("The dataflow failed."); + } + return state; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java new file mode 100644 index 0000000000000..6044365a664d9 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java @@ -0,0 +1,164 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.PipelineResult; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; +import com.google.cloud.dataflow.sdk.util.MonitoringUtil; +import com.google.common.base.Optional; +import com.google.common.collect.Iterators; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Iterator; + +/** + * A creator of test pipelines which can be used inside of tests that can be + * configured to run locally or against the live service. + * + *

It is recommended to tag hand-selected tests for this purpose using the + * RunnableOnService Category annotation, as each test run against the service + * will spin up and tear down a single VM. + * + *

In order to run tests on the dataflow pipeline service, the following + * conditions must be met: + *

    + *
  • runIntegrationTestOnService System property must be set to true. + *
  • System property "projectName" must be set to your Cloud project. + *
  • System property "temp_gcs_directory" must be set to a valid GCS bucket. + *
  • Jars containing the SDK and test classes must be added to the test classpath. + *
+ * + *

Use {@link DataflowAssert} for tests, as it integrates with this test + * harness in both direct and remote execution modes. For example: + * + *

{@code
+ * Pipeline p = TestPipeline.create();
+ * PCollection output = ...
+ *
+ * DataflowAssert.that(output)
+ *     .containsInAnyOrder(1, 2, 3, 4);
+ * p.run();
+ * }
+ * + */ +public class TestPipeline extends Pipeline { + private static final String PROPERTY_DATAFLOW_OPTIONS = "dataflowOptions"; + private static final Logger LOG = LoggerFactory.getLogger(TestPipeline.class); + private static final ObjectMapper MAPPER = new ObjectMapper(); + + /** + * Creates and returns a new test pipeline. + * + *

Use {@link DataflowAssert} to add tests, then call + * {@link Pipeline#run} to execute the pipeline and check the tests. + */ + public static TestPipeline create() { + if (Boolean.parseBoolean(System.getProperty("runIntegrationTestOnService"))) { + TestDataflowPipelineOptions options = getPipelineOptions(); + LOG.info("Using passed in options: " + options); + return new TestPipeline(createRunner(options), options); + } else { + DirectPipelineRunner directRunner = DirectPipelineRunner.createForTest(); + return new TestPipeline(directRunner, directRunner.getPipelineOptions()); + } + } + + private TestPipeline(PipelineRunner runner, PipelineOptions options) { + super(runner, options); + } + + /** + * Creates and returns a TestDataflowPipelineRunner based on + * configuration via system properties. + */ + private static TestDataflowPipelineRunner createRunner( + TestDataflowPipelineOptions options) { + + DataflowPipelineRunner dataflowRunner = DataflowPipelineRunner + .fromOptions(options); + return new TestDataflowPipelineRunner(dataflowRunner, + new MonitoringUtil.PrintHandler(options.getJobMessageOutput())); + } + + /** + * Creates PipelineOptions for testing with a DataflowPipelineRunner. + */ + static TestDataflowPipelineOptions getPipelineOptions() { + try { + TestDataflowPipelineOptions options = MAPPER.readValue( + System.getProperty(PROPERTY_DATAFLOW_OPTIONS), PipelineOptions.class) + .as(TestDataflowPipelineOptions.class); + options.setAppName(getAppName()); + options.setJobName(getJobName()); + return options; + } catch (IOException e) { + throw new RuntimeException("Unable to instantiate test options from system property " + + PROPERTY_DATAFLOW_OPTIONS + ":" + System.getProperty(PROPERTY_DATAFLOW_OPTIONS), e); + } + } + + /** Returns the class name of the test, or a default name. */ + private static String getAppName() { + Optional stackTraceElement = findCallersStackTrace(); + if (stackTraceElement.isPresent()) { + String className = stackTraceElement.get().getClassName(); + return className.contains(".") + ? className.substring(className.lastIndexOf(".") + 1) + : className; + } + return "UnitTest"; + } + + /** Returns the method name of the test, or a default name. */ + private static String getJobName() { + Optional stackTraceElement = findCallersStackTrace(); + if (stackTraceElement.isPresent()) { + return stackTraceElement.get().getMethodName(); + } + return "unittestjob"; + } + + /** Returns the {@link StackTraceElement} of the calling class. */ + private static Optional findCallersStackTrace() { + Iterator elements = + Iterators.forArray(Thread.currentThread().getStackTrace()); + // First find the TestPipeline class in the stack trace. + while (elements.hasNext()) { + StackTraceElement next = elements.next(); + if (TestPipeline.class.getName().equals(next.getClassName())) { + break; + } + } + // Then find the first instance after which is not the TestPipeline + while (elements.hasNext()) { + StackTraceElement next = elements.next(); + if (!TestPipeline.class.getName().equals(next.getClassName())) { + return Optional.of(next); + } + } + return Optional.absent(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowingFnTestUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowingFnTestUtils.java new file mode 100644 index 0000000000000..687cb64530efa --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowingFnTestUtils.java @@ -0,0 +1,185 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; + +import org.joda.time.Instant; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * A utility class for testing {@link WindowingFn}s. + */ +public class WindowingFnTestUtils { + + /** + * Creates a Set of elements to be used as expected output in + * {@link #runWindowingFn}. + */ + public static Set set(long... timestamps) { + Set result = new HashSet<>(); + for (long timestamp : timestamps) { + result.add(timestampValue(timestamp)); + } + return result; + } + + + /** + * Runs the {@link WindowingFn} over the provided input, returning a map + * of windows to the timestamps in those windows. + */ + public static Map> runWindowingFn( + WindowingFn windowingFn, + List timestamps) throws Exception { + + final TestWindowSet windowSet = new TestWindowSet(); + for (final Long timestamp : timestamps) { + for (W window : windowingFn.assignWindows( + new TestAssignContext(new Instant(timestamp), windowingFn))) { + windowSet.put(window, timestampValue(timestamp)); + } + windowingFn.mergeWindows(new TestMergeContext(windowSet, windowingFn)); + } + Map> actual = new HashMap<>(); + for (W window : windowSet.windows()) { + actual.put(window, windowSet.get(window)); + } + return actual; + } + + private static String timestampValue(long timestamp) { + return "T" + new Instant(timestamp); + } + + /** + * Test implementation of AssignContext. + */ + private static class TestAssignContext + extends WindowingFn.AssignContext { + private Instant timestamp; + + public TestAssignContext(Instant timestamp, WindowingFn windowingFn) { + windowingFn.super(); + this.timestamp = timestamp; + } + + @Override + public T element() { + return null; + } + + @Override + public Instant timestamp() { + return timestamp; + } + + @Override + public Collection windows() { + return null; + } + } + + /** + * Test implementation of MergeContext. + */ + private static class TestMergeContext + extends WindowingFn.MergeContext { + private TestWindowSet windowSet; + + public TestMergeContext( + TestWindowSet windowSet, WindowingFn windowingFn) { + windowingFn.super(); + this.windowSet = windowSet; + } + + @Override + public Collection windows() { + return windowSet.windows(); + } + + @Override + public void merge(Collection toBeMerged, W mergeResult) { + windowSet.merge(toBeMerged, mergeResult); + } + } + + /** + * A WindowSet useful for testing WindowingFns which simply + * collects the placed elements into multisets. + */ + private static class TestWindowSet { + + private Map> elements = new HashMap<>(); + private List> emitted = new ArrayList<>(); + + public void put(W window, V value) { + Set all = elements.get(window); + if (all == null) { + all = new HashSet<>(); + elements.put(window, all); + } + all.add(value); + } + + public void remove(W window) { + elements.remove(window); + } + + public void merge(Collection otherWindows, W window) { + if (otherWindows.isEmpty()) { + return; + } + Set merged = new HashSet<>(); + if (elements.containsKey(window) && !otherWindows.contains(window)) { + merged.addAll(elements.get(window)); + } + for (W w : otherWindows) { + if (!elements.containsKey(w)) { + throw new IllegalArgumentException("Tried to merge a non-existent window:" + w); + } + merged.addAll(elements.get(w)); + elements.remove(w); + } + elements.put(window, merged); + } + + public void markCompleted(W window) {} + + public Collection windows() { + return elements.keySet(); + } + + public boolean contains(W window) { + return elements.containsKey(window); + } + + // For testing. + + public Set get(W window) { + return elements.get(window); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/package-info.java new file mode 100644 index 0000000000000..799c1ac98bc8c --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/package-info.java @@ -0,0 +1,21 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Defines utilities for unit testing Dataflow pipelines. The tests for the {@code PTransform}s and + * examples included the Dataflow SDK provide examples of using these utilities. + */ +package com.google.cloud.dataflow.sdk.testing; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Aggregator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Aggregator.java new file mode 100644 index 0000000000000..13ad17efa7027 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Aggregator.java @@ -0,0 +1,64 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +/** + * An {@code Aggregator} enables arbitrary monitoring in user code. + * + *

Aggregators are created by calling {@link DoFn.Context#createAggregator}, + * typically from {@link DoFn#startBundle}. Elements can be added to the + * {@code Aggregator} by calling {@link Aggregator#addValue}. + * + *

Aggregators are visible in the monitoring UI, when the pipeline is run + * using DataflowPipelineRunner or BlockingDataflowPipelineRunner, along with + * their current value. Aggregators may not become visible until the system + * begins executing the ParDo transform which created them and/or their initial + * value is changed. + * + *

Example: + *

 {@code
+ * class MyDoFn extends DoFn {
+ *   private Aggregator myAggregator;
+ *
+ *   {@literal @}Override
+ *   public void startBundle(Context c) {
+ *     myAggregator = c.createAggregator("myCounter", new Sum.SumIntegerFn());
+ *   }
+ *
+ *   {@literal @}Override
+ *   public void processElement(ProcessContext c) {
+ *     myAggregator.addValue(1);
+ *   }
+ * }
+ * } 
+ * + * @param the type of input values + */ +public interface Aggregator { + + /** + * Adds a new value into the Aggregator. + */ + public void addValue(VI value); + + // TODO: Consider the following additional API conveniences: + // - In addition to createAggregator(), consider adding getAggregator() to + // avoid the need to store the aggregator locally in a DoFn, i.e., create + // if not already present. + // - Add a shortcut for the most common aggregator: + // c.createAggregator("name", new Sum.SumIntegerFn()). +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java new file mode 100644 index 0000000000000..ff5687fe30fb3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java @@ -0,0 +1,723 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.coders.CustomCoder; +import com.google.cloud.dataflow.sdk.coders.ListCoder; +import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.google.common.collect.UnmodifiableIterator; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.PriorityQueue; + +/** + * {@code PTransform}s for getting an idea of a {@code PCollection}'s + * data distribution using approximate {@code N}-tiles, either + * globally or per-key. + */ +public class ApproximateQuantiles { + + /** + * Returns a {@code PTransform} that takes a {@code PCollection} + * and returns a {@code PCollection>} whose sinlge value is a + * {@code List} of the approximate {@code N}-tiles of the elements + * of the input {@code PCollection}. This gives an idea of the + * distribution of the input elements. + * + *

The computed {@code List} is of size {@code numQuantiles}, + * and contains the input elements' minimum value, + * {@code numQuantiles-2} intermediate values, and maximum value, in + * sorted order, using the given {@code Comparator} to order values. + * To compute traditional {@code N}-tiles, one should use + * {@code ApproximateQuantiles.globally(compareFn, N+1)}. + * + *

If there are fewer input elements than {@code numQuantiles}, + * then the result {@code List} will contain all the input elements, + * in sorted order. + * + *

The argument {@code Comparator} must be {@code Serializable}. + * + *

Example of use: + *

 {@code
+   * PCollection pc = ...;
+   * PCollection> quantiles =
+   *     pc.apply(ApproximateQuantiles.globally(stringCompareFn, 11));
+   * } 
+ * + * @param the type of the elements in the input {@code PCollection} + * @param numQuantiles the number of elements in the resulting + * quantile values {@code List} + * @param compareFn the function to use to order the elements + */ + public static & Serializable> + PTransform, PCollection>> globally( + int numQuantiles, C compareFn) { + return Combine.globally( + ApproximateQuantilesCombineFn.create(numQuantiles, compareFn)); + } + + /** + * Like {@link #globally(int, Comparator)}, but sorts using the + * elements' natural ordering. + * + * @param the type of the elements in the input {@code PCollection} + * @param numQuantiles the number of elements in the resulting + * quantile values {@code List} + */ + public static > + PTransform, PCollection>> globally(int numQuantiles) { + return Combine.globally( + ApproximateQuantilesCombineFn.create(numQuantiles)); + } + + /** + * Returns a {@code PTransform} that takes a + * {@code PCollection>} and returns a + * {@code PCollection>>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to a {@code List} of the approximate + * {@code N}-tiles of the values associated with that key in the + * input {@code PCollection}. This gives an idea of the + * distribution of the input values for each key. + * + *

Each of the computed {@code List}s is of size {@code numQuantiles}, + * and contains the input values' minimum value, + * {@code numQuantiles-2} intermediate values, and maximum value, in + * sorted order, using the given {@code Comparator} to order values. + * To compute traditional {@code N}-tiles, one should use + * {@code ApproximateQuantiles.perKey(compareFn, N+1)}. + * + *

If a key has fewer than {@code numQuantiles} values + * associated with it, then that key's output {@code List} will + * contain all the key's input values, in sorted order. + * + *

The argument {@code Comparator} must be {@code Serializable}. + * + *

Example of use: + *

 {@code
+   * PCollection> pc = ...;
+   * PCollection>> quantilesPerKey =
+   *     pc.apply(ApproximateQuantiles.perKey(stringCompareFn, 11));
+   * } 
+ * + *

See {@link Combine.PerKey} for how this affects timestamps and windowing. + * + * @param the type of the keys in the input and output + * {@code PCollection}s + * @param the type of the values in the input {@code PCollection} + * @param numQuantiles the number of elements in the resulting + * quantile values {@code List} + * @param compareFn the function to use to order the elements + */ + public static & Serializable> + PTransform>, PCollection>>> + perKey(int numQuantiles, C compareFn) { + return Combine.perKey( + ApproximateQuantilesCombineFn.create(numQuantiles, compareFn) + .asKeyedFn()); + } + + /** + * Like {@link #perKey(int, Comparator)}, but sorts + * values using the their natural ordering. + * + * @param the type of the keys in the input and output + * {@code PCollection}s + * @param the type of the values in the input {@code PCollection} + * @param numQuantiles the number of elements in the resulting + * quantile values {@code List} + */ + public static > + PTransform>, PCollection>>> + perKey(int numQuantiles) { + return Combine.perKey( + ApproximateQuantilesCombineFn.create(numQuantiles) + .asKeyedFn()); + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * The {@code ApproximateQuantilesCombineFn} combiner gives an idea + * of the distribution of a collection of values using approximate + * {@code N}-tiles. The output of this combiner is a {@code List} + * of size {@code numQuantiles}, containing the input values' + * minimum value, {@code numQuantiles-2} intermediate values, and + * maximum value, in sorted order, so for traditional + * {@code N}-tiles, one should use + * {@code ApproximateQuantilesCombineFn#create(N+1)}. + * + *

If there are fewer values to combine than + * {@code numQuantiles}, then the result {@code List} will contain all the + * values being combined, in sorted order. + * + *

Values are ordered using either a specified + * {@code Comparator} or the values' natural ordering. + * + *

To evaluate the quantiles we use the "New Algorithm" described here: + *

+   *   [MRL98] Manku, Rajagopalan & Lindsay, "Approximate Medians and other
+   *   Quantiles in One Pass and with Limited Memory", Proc. 1998 ACM
+   *   SIGMOD, Vol 27, No 2, p 426-435, June 1998.
+   *   http://citeseerx.ist.psu.edu/viewdoc/download?doi=10.1.1.6.6513&rep=rep1&type=pdf
+   * 
+ * + *

The default error bound is {@code 1 / N}, though in practice + * the accuracy tends to be much better.

See + * {@link #create(int, Comparator, long, double)} for + * more information about the meaning of {@code epsilon}, and + * {@link #withEpsilon} for a convenient way to adjust it. + * + * @param the type of the values being combined + */ + public static class ApproximateQuantilesCombineFn + & Serializable> + extends AccumulatingCombineFn + .QuantileState, List> { + + /** + * The cost (in time and space) to compute quantiles to a given + * accuracy is a function of the total number of elements in the + * data set. If an estimate is not known or specified, we use + * this as an upper bound. If this is too low, errors may exceed + * the requested tolerance; if too high, efficiency may be + * non-optimal. The impact is logarithmic with respect to this + * value, so this default should be fine for most uses. + */ + public static final long DEFAULT_MAX_NUM_ELEMENTS = (long) 1e9; + + /** The comparison function to use. */ + private final C compareFn; + + /** + * Number of quantiles to produce. The size of the final output + * list, including the minimum and maximum, is numQuantiles. + */ + private final int numQuantiles; + + /** The size of the buffers, corresponding to k in the referenced paper. */ + private final int bufferSize; + + /** The number of buffers, corresponding to b in the referenced paper. */ + private final int numBuffers; + + private final double epsilon; + private final long maxNumElements; + + /** + * Used to alternate between biasing up and down in the even weight collapse + * operation. + */ + private int offsetJitter = 0; + + /** + * Returns an approximate quantiles combiner with the given + * {@code compareFn} and desired number of quantiles. A total of + * {@code numQuantiles} elements will appear in the output list, + * including the minimum and maximum. + * + *

The {@code Comparator} must be {@code Serializable}. + * + *

The default error bound is {@code 1 / numQuantiles} which + * holds as long as the number of elements is less than + * {@link #DEFAULT_MAX_NUM_ELEMENTS}. + */ + public static & Serializable> + ApproximateQuantilesCombineFn create( + int numQuantiles, C compareFn) { + return create(numQuantiles, compareFn, + DEFAULT_MAX_NUM_ELEMENTS, 1.0 / numQuantiles); + } + + /** + * Like {@link #create(int, Comparator)}, but sorts + * values using their natural ordering. + */ + public static > + ApproximateQuantilesCombineFn> create(int numQuantiles) { + return create(numQuantiles, new Top.Largest()); + } + + /** + * Returns an {@code ApproximateQuantilesCombineFn} that's like + * this one except that it uses the specified {@code epsilon} + * value. Does not modify this combiner. + * + *

See {@link #create(int, Comparator, long, + * double)} for more information about the meaning of + * {@code epsilon}. + */ + public ApproximateQuantilesCombineFn withEpsilon(double epsilon) { + return create(numQuantiles, compareFn, maxNumElements, epsilon); + } + + /** + * Returns an {@code ApproximateQuantilesCombineFn} that's like + * this one except that it uses the specified {@code maxNumElements} + * value. Does not modify this combiner. + * + *

See {@link #create(int, Comparator, long, double)} for more + * information about the meaning of {@code maxNumElements}. + */ + public ApproximateQuantilesCombineFn withMaxInputSize( + long maxNumElements) { + return create(numQuantiles, compareFn, maxNumElements, maxNumElements); + } + + /** + * Creates an approximate quantiles combiner with the given + * {@code compareFn} and desired number of quantiles. A total of + * {@code numQuantiles} elements will appear in the output list, + * including the minimum and maximum. + * + *

The {@code Comparator} must be {@code Serializable}. + * + *

The default error bound is {@code epsilon} which is holds as long + * as the number of elements is less than {@code maxNumElements}. + * Specifically, if one considers the input as a sorted list x_1, ..., x_N, + * then the distance between the each exact quantile x_c and its + * approximation x_c' is bounded by {@code |c - c'| < epsilon * N}. + * Note that these errors are worst-case scenarios; in practice the accuracy + * tends to be much better. + */ + public static & Serializable> + ApproximateQuantilesCombineFn create( + int numQuantiles, + C compareFn, + long maxNumElements, + double epsilon) { + // Compute optimal b and k. + int b = 2; + while ((b - 2) * (1 << (b - 2)) < epsilon * maxNumElements) { + b++; + } + b--; + int k = Math.max(2, (int) Math.ceil(maxNumElements / (1 << (b - 1)))); + return new ApproximateQuantilesCombineFn<>( + numQuantiles, compareFn, k, b, epsilon, maxNumElements); + } + + private ApproximateQuantilesCombineFn(int numQuantiles, + C compareFn, + int bufferSize, + int numBuffers, + double epsilon, + long maxNumElements) { + Preconditions.checkArgument(numQuantiles >= 2); + Preconditions.checkArgument(bufferSize >= 2); + Preconditions.checkArgument(numBuffers >= 2); + Preconditions.checkArgument(compareFn instanceof Serializable); + this.numQuantiles = numQuantiles; + this.compareFn = compareFn; + this.bufferSize = bufferSize; + this.numBuffers = numBuffers; + this.epsilon = epsilon; + this.maxNumElements = maxNumElements; + } + + @Override + public QuantileState createAccumulator() { + return new QuantileState(); + } + + @Override + public Coder getAccumulatorCoder( + CoderRegistry registry, Coder elementCoder) { + return new QuantileStateCoder(elementCoder); + } + + /** + * Compact summarization of a collection on which quantiles can be + * estimated. + */ + class QuantileState + extends AccumulatingCombineFn + .QuantileState, List> + .Accumulator { + + private T min; + private T max; + + /** + * The set of buffers, ordered by level from smallest to largest. + */ + private PriorityQueue buffers = + new PriorityQueue<>(numBuffers + 1); + + /** + * The algorithm requires that the manipulated buffers always be filled + * to capacity to perform the collapse operation. This operation can + * be extended to buffers of varying sizes by introducing the notion of + * fractional weights, but it's easier to simply combine the remainders + * from all shards into new, full buffers and then take them into account + * when computing the final output. + */ + private List unbufferedElements = Lists.newArrayList(); + + public QuantileState() { } + + public QuantileState(T elem) { + min = elem; + max = elem; + unbufferedElements.add(elem); + } + + public QuantileState(T min, T max, Collection unbufferedElements, + Collection buffers) { + this.min = min; + this.max = max; + this.unbufferedElements.addAll(unbufferedElements); + this.buffers.addAll(buffers); + } + + /** + * Add a new element to the collection being summarized by this state. + */ + @Override + public void addInput(T elem) { + if (isEmpty()) { + min = max = elem; + } else if (compareFn.compare(elem, min) < 0) { + min = elem; + } else if (compareFn.compare(elem, max) > 0) { + max = elem; + } + addUnbuffered(elem); + } + + /** + * Add a new buffer to the unbuffered list, creating a new buffer and + * collapsing if needed. + */ + private void addUnbuffered(T elem) { + unbufferedElements.add(elem); + if (unbufferedElements.size() == bufferSize) { + Collections.sort(unbufferedElements, compareFn); + buffers.add(new QuantileBuffer(unbufferedElements)); + unbufferedElements = Lists.newArrayListWithCapacity(bufferSize); + collapseIfNeeded(); + } + } + + /** + * Updates this as if adding all elements seen by other. + */ + @Override + public void mergeAccumulator(QuantileState other) { + if (other.isEmpty()) { + return; + } + if (min == null || compareFn.compare(other.min, min) < 0) { + min = other.min; + } + if (max == null || compareFn.compare(other.max, max) > 0) { + max = other.max; + } + for (T elem : other.unbufferedElements) { + addUnbuffered(elem); + } + buffers.addAll(other.buffers); + collapseIfNeeded(); + } + + public boolean isEmpty() { + return unbufferedElements.size() == 0 && buffers.size() == 0; + } + + private void collapseIfNeeded() { + while (buffers.size() > numBuffers) { + List toCollapse = Lists.newArrayList(); + toCollapse.add(buffers.poll()); + toCollapse.add(buffers.poll()); + int minLevel = toCollapse.get(1).level; + while (!buffers.isEmpty() && buffers.peek().level == minLevel) { + toCollapse.add(buffers.poll()); + } + buffers.add(collapse(toCollapse)); + } + } + + private QuantileBuffer collapse(Iterable buffers) { + int newLevel = 0; + long newWeight = 0; + for (QuantileBuffer buffer : buffers) { + // As presented in the paper, there should always be at least two + // buffers of the same (minimal) level to collapse, but it is possible + // to violate this condition when combining buffers from independently + // computed shards. If they differ we take the max. + newLevel = Math.max(newLevel, buffer.level + 1); + newWeight += buffer.weight; + } + List newElements = + interpolate(buffers, bufferSize, newWeight, offset(newWeight)); + return new QuantileBuffer(newLevel, newWeight, newElements); + } + + /** + * Outputs numQuantiles elements consisting of the minimum, maximum, and + * numQuantiles - 2 evenly spaced intermediate elements. + * + * Returns the empty list if no elements have been added. + */ + @Override + public List extractOutput() { + if (isEmpty()) { + return Lists.newArrayList(); + } + long totalCount = unbufferedElements.size(); + for (QuantileBuffer buffer : buffers) { + totalCount += bufferSize * buffer.weight; + } + List all = Lists.newArrayList(buffers); + if (!unbufferedElements.isEmpty()) { + Collections.sort(unbufferedElements, compareFn); + all.add(new QuantileBuffer(unbufferedElements)); + } + double step = 1.0 * totalCount / (numQuantiles - 1); + double offset = (1.0 * totalCount - 1) / (numQuantiles - 1); + List quantiles = interpolate(all, numQuantiles - 2, step, offset); + quantiles.add(0, min); + quantiles.add(max); + return quantiles; + } + } + + /** + * A single buffer in the sense of the referenced algorithm. + */ + private class QuantileBuffer implements Comparable { + private int level; + private long weight; + private List elements; + + public QuantileBuffer(List elements) { + this(0, 1, elements); + } + + public QuantileBuffer(int level, long weight, List elements) { + this.level = level; + this.weight = weight; + this.elements = elements; + } + + @Override + public int compareTo(QuantileBuffer other) { + return this.level - other.level; + } + + @Override + public String toString() { + return "QuantileBuffer[" + + "level=" + level + + ", weight=" + + weight + ", elements=" + elements + "]"; + } + + public Iterator> weightedIterator() { + return new UnmodifiableIterator>() { + Iterator iter = elements.iterator(); + @Override public boolean hasNext() { return iter.hasNext(); } + @Override public WeightedElement next() { + return WeightedElement.of(weight, iter.next()); + } + }; + } + } + + /** + * Coder for QuantileState. + */ + private class QuantileStateCoder extends CustomCoder { + + private final Coder elementCoder; + private final Coder> elementListCoder; + + public QuantileStateCoder(Coder elementCoder) { + this.elementCoder = elementCoder; + this.elementListCoder = ListCoder.of(elementCoder); + } + + @Override + public void encode( + QuantileState state, OutputStream outStream, Coder.Context context) + throws CoderException, IOException { + Coder.Context nestedContext = context.nested(); + elementCoder.encode(state.min, outStream, nestedContext); + elementCoder.encode(state.max, outStream, nestedContext); + elementListCoder.encode( + state.unbufferedElements, outStream, nestedContext); + BigEndianIntegerCoder.of().encode( + state.buffers.size(), outStream, nestedContext); + for (QuantileBuffer buffer : state.buffers) { + encodeBuffer(buffer, outStream, nestedContext); + } + } + + @Override + public QuantileState decode(InputStream inStream, Coder.Context context) + throws CoderException, IOException { + Coder.Context nestedContext = context.nested(); + T min = elementCoder.decode(inStream, nestedContext); + T max = elementCoder.decode(inStream, nestedContext); + List unbufferedElements = + elementListCoder.decode(inStream, nestedContext); + int numBuffers = + BigEndianIntegerCoder.of().decode(inStream, nestedContext); + List buffers = new ArrayList<>(numBuffers); + for (int i = 0; i < numBuffers; i++) { + buffers.add(decodeBuffer(inStream, nestedContext)); + } + return new QuantileState(min, max, unbufferedElements, buffers); + } + + private void encodeBuffer( + QuantileBuffer buffer, OutputStream outStream, Coder.Context context) + throws CoderException, IOException { + DataOutputStream outData = new DataOutputStream(outStream); + outData.writeInt(buffer.level); + outData.writeLong(buffer.weight); + elementListCoder.encode(buffer.elements, outStream, context); + } + + private QuantileBuffer decodeBuffer( + InputStream inStream, Coder.Context context) + throws IOException, CoderException { + DataInputStream inData = new DataInputStream(inStream); + return new QuantileBuffer( + inData.readInt(), + inData.readLong(), + elementListCoder.decode(inStream, context)); + } + + /** + * Notifies ElementByteSizeObserver about the byte size of the + * encoded value using this coder. + */ + @Override + public void registerByteSizeObserver( + QuantileState state, + ElementByteSizeObserver observer, + Coder.Context context) + throws Exception { + Coder.Context nestedContext = context.nested(); + elementCoder.registerByteSizeObserver( + state.min, observer, nestedContext); + elementCoder.registerByteSizeObserver( + state.max, observer, nestedContext); + elementListCoder.registerByteSizeObserver( + state.unbufferedElements, observer, nestedContext); + + BigEndianIntegerCoder.of().registerByteSizeObserver( + state.buffers.size(), observer, nestedContext); + for (QuantileBuffer buffer : state.buffers) { + observer.update(4L + 8); + + elementListCoder.registerByteSizeObserver( + buffer.elements, observer, nestedContext); + } + } + + @Override + public boolean isDeterministic() { + return elementListCoder.isDeterministic(); + } + } + + /** + * If the weight is even, we must round up our down. Alternate between + * these two options to avoid a bias. + */ + private long offset(long newWeight) { + if (newWeight % 2 == 1) { + return (newWeight + 1) / 2; + } else { + offsetJitter = 2 - offsetJitter; + return (newWeight + offsetJitter) / 2; + } + } + + /** + * Emulates taking the ordered union of all elements in buffers, repeated + * according to their weight, and picking out the (k * step + offset)-th + * elements of this list for {@code 0 <= k < count}. + */ + private List interpolate(Iterable buffers, + int count, double step, double offset) { + List>> iterators = Lists.newArrayList(); + for (QuantileBuffer buffer : buffers) { + iterators.add(buffer.weightedIterator()); + } + // Each of the buffers is already sorted by element. + Iterator> sorted = Iterators.mergeSorted( + iterators, + new Comparator>() { + @Override + public int compare(WeightedElement a, WeightedElement b) { + return compareFn.compare(a.value, b.value); + } + }); + + List newElements = Lists.newArrayListWithCapacity(count); + WeightedElement weightedElement = sorted.next(); + double current = weightedElement.weight; + for (int j = 0; j < count; j++) { + double target = j * step + offset; + while (current <= target && sorted.hasNext()) { + weightedElement = sorted.next(); + current += weightedElement.weight; + } + newElements.add(weightedElement.value); + } + return newElements; + } + + /** An element and its weight. */ + private static class WeightedElement { + public long weight; + public T value; + private WeightedElement(long weight, T value) { + this.weight = weight; + this.value = value; + } + public static WeightedElement of(long weight, T value) { + return new WeightedElement<>(weight, value); + } + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java new file mode 100644 index 0000000000000..9308a010a2a9e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java @@ -0,0 +1,426 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.Coder.Context; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.SerializableCoder; +import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.hash.Hashing; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.PriorityQueue; + +/** + * {@code PTransform}s for estimating the number of distinct elements + * in a {@code PCollection}, or the number of distinct values + * associated with each key in a {@code PCollection} of {@code KV}s. + */ +public class ApproximateUnique { + + /** + * Returns a {@code PTransform} that takes a {@code PCollection} + * and returns a {@code PCollection} containing a single value + * that is an estimate of the number of distinct elements in the + * input {@code PCollection}. + * + *

The {@code sampleSize} parameter controls the estimation + * error. The error is about {@code 2 / sqrt(sampleSize)}, so for + * {@code ApproximateUnique.globally(10000)} the estimation error is + * about 2%. Similarly, for {@code ApproximateUnique.of(16)} the + * estimation error is about 50%. If there are fewer than + * {@code sampleSize} distinct elements then the returned result + * will be exact with extremely high probability (the chance of a + * hash collision is about {@code sampleSize^2 / 2^65}). + * + *

This transform approximates the number of elements in a set + * by computing the top {@code sampleSize} hash values, and using + * that to extrapolate the size of the entire set of hash values by + * assuming the rest of the hash values are as densely distributed + * as the top {@code sampleSize}. + * + *

See also {@link #globally(double)}. + * + *

Example of use: + *

 {@code
+   * PCollection pc = ...;
+   * PCollection approxNumDistinct =
+   *     pc.apply(ApproximateUnique.globally(1000));
+   * } 
+ * + * @param the type of the elements in the input {@code PCollection} + * @param sampleSize the number of entries in the statistical + * sample; the higher this number, the more accurate the + * estimate will be; should be {@code >= 16} + * @throws IllegalArgumentException if the {@code sampleSize} + * argument is too small + */ + public static Globally globally(int sampleSize) { + return new Globally<>(sampleSize); + } + + /** + * Like {@link #globally(int)}, but specifies the desired maximum + * estimation error instead of the sample size. + * + * @param the type of the elements in the input {@code PCollection} + * @param maximumEstimationError the maximum estimation error, which + * should be in the range {@code [0.01, 0.5]} + * @throws IllegalArgumentException if the + * {@code maximumEstimationError} argument is out of range + */ + public static Globally globally(double maximumEstimationError) { + return new Globally<>(maximumEstimationError); + } + + /** + * Returns a {@code PTransform} that takes a + * {@code PCollection>} and returns a + * {@code PCollection>} that contains an output element + * mapping each distinct key in the input {@code PCollection} to an + * estimate of the number of distinct values associated with that + * key in the input {@code PCollection}. + * + *

See {@link #globally(int)} for an explanation of the + * {@code sampleSize} parameter. A separate sampling is computed + * for each distinct key of the input. + * + *

See also {@link #perKey(double)}. + * + *

Example of use: + *

 {@code
+   * PCollection> pc = ...;
+   * PCollection> approxNumDistinctPerKey =
+   *     pc.apply(ApproximateUnique.perKey(1000));
+   * } 
+ * + * @param the type of the keys in the input and output + * {@code PCollection}s + * @param the type of the values in the input {@code PCollection} + * @param sampleSize the number of entries in the statistical + * sample; the higher this number, the more accurate the + * estimate will be; should be {@code >= 16} + * @throws IllegalArgumentException if the {@code sampleSize} + * argument is too small + */ + public static PerKey perKey(int sampleSize) { + return new PerKey<>(sampleSize); + } + + /** + * Like {@link #perKey(int)}, but specifies the desired maximum + * estimation error instead of the sample size. + * + * @param the type of the keys in the input and output + * {@code PCollection}s + * @param the type of the values in the input {@code PCollection} + * @param maximumEstimationError the maximum estimation error, which + * should be in the range {@code [0.01, 0.5]} + * @throws IllegalArgumentException if the + * {@code maximumEstimationError} argument is out of range + */ + public static PerKey perKey(double maximumEstimationError) { + return new PerKey<>(maximumEstimationError); + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * {@code PTransform} for estimating the number of distinct elements + * in a {@code PCollection}. + * + * @param the type of the elements in the input {@code PCollection} + */ + static class Globally extends PTransform, PCollection> { + + /** + * The number of entries in the statistical sample; the higher this number, + * the more accurate the estimate will be. + */ + private final long sampleSize; + + /** + * @see ApproximateUnique#globally(int) + */ + public Globally(int sampleSize) { + if (sampleSize < 16) { + throw new IllegalArgumentException( + "ApproximateUnique needs a sampleSize " + + ">= 16 for an estimation error <= 50%. " + + "In general, the estimation " + + "error is about 2 / sqrt(sampleSize)."); + } + this.sampleSize = sampleSize; + } + + /** + * @see ApproximateUnique#globally(double) + */ + public Globally(double maximumEstimationError) { + if (maximumEstimationError < 0.01 || maximumEstimationError > 0.5) { + throw new IllegalArgumentException( + "ApproximateUnique needs an " + + "estimation error between 1% (0.01) and 50% (0.5)."); + } + this.sampleSize = sampleSizeFromEstimationError(maximumEstimationError); + } + + @Override + public PCollection apply(PCollection input) { + Coder coder = input.getCoder(); + return input.apply( + Combine.globally( + new ApproximateUniqueCombineFn<>(sampleSize, coder))); + } + + @Override + protected String getKindString() { + return "ApproximateUnique.Globally"; + } + } + + /** + * {@code PTransform} for estimating the number of distinct values + * associated with each key in a {@code PCollection} of {@code KV}s. + * + * @param the type of the keys in the input and output + * {@code PCollection}s + * @param the type of the values in the input {@code PCollection} + */ + static class PerKey + extends PTransform>, PCollection>> { + + private final long sampleSize; + + /** + * @see ApproximateUnique#perKey(int) + */ + public PerKey(int sampleSize) { + if (sampleSize < 16) { + throw new IllegalArgumentException( + "ApproximateUnique needs a " + + "sampleSize >= 16 for an estimation error <= 50%. In general, " + + "the estimation error is about 2 / sqrt(sampleSize)."); + } + this.sampleSize = sampleSize; + } + + /** + * @see ApproximateUnique#perKey(double) + */ + public PerKey(double estimationError) { + if (estimationError < 0.01 || estimationError > 0.5) { + throw new IllegalArgumentException( + "ApproximateUnique.PerKey needs an " + + "estimation error between 1% (0.01) and 50% (0.5)."); + } + this.sampleSize = sampleSizeFromEstimationError(estimationError); + } + + @Override + public PCollection> apply(PCollection> input) { + Coder> inputCoder = input.getCoder(); + if (!(inputCoder instanceof KvCoder)) { + throw new IllegalStateException( + "ApproximateUnique.PerKey requires its input to use KvCoder"); + } + @SuppressWarnings("unchecked") + final Coder coder = ((KvCoder) inputCoder).getValueCoder(); + + return input.apply( + Combine.perKey(new ApproximateUniqueCombineFn<>( + sampleSize, coder).asKeyedFn())); + } + + @Override + protected String getKindString() { + return "ApproximateUnique.PerKey"; + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * {@code CombineFn} that computes an estimate of the number of + * distinct values that were combined. + * + *

Hashes input elements, computes the top {@code sampleSize} + * hash values, and uses those to extrapolate the size of the entire + * set of hash values by assuming the rest of the hash values are as + * densely distributed as the top {@code sampleSize}. + * + *

Used to implement + * {@link #globally(int) ApproximatUnique.globally(...)} and + * {@link #perKey(int) ApproximatUnique.perKey(...)}. + * + * @param the type of the values being combined + */ + public static class ApproximateUniqueCombineFn extends + CombineFn { + + /** + * The size of the space of hashes returned by the hash function. + */ + static final double HASH_SPACE_SIZE = + Long.MAX_VALUE - (double) Long.MIN_VALUE; + + /** + * A heap utility class to efficiently track the largest added elements. + */ + public static class LargestUnique implements Serializable { + private PriorityQueue heap = new PriorityQueue<>(); + private final long sampleSize; + + /** + * Creates a heap to track the largest {@code sampleSize} elements. + * + * @param sampleSize the size of the heap + */ + public LargestUnique(long sampleSize) { + this.sampleSize = sampleSize; + } + + /** + * Adds a value to the heap, returning whether the value is (large enough + * to be) in the heap. + */ + public boolean add(Long value) { + if (heap.contains(value)) { + return true; + } else if (heap.size() < sampleSize) { + heap.add(value); + return true; + } else if (value > heap.element()) { + heap.remove(); + heap.add(value); + return true; + } else { + return false; + } + } + + /** + * Returns the values in the heap, ordered largest to smallest. + */ + public List extractOrderedList() { + // The only way to extract the order from the heap is element-by-element + // from smallest to largest. + Long[] array = new Long[heap.size()]; + for (int i = heap.size() - 1; i >= 0; i--) { + array[i] = heap.remove(); + } + return Arrays.asList(array); + } + } + + private final long sampleSize; + private final Coder coder; + + public ApproximateUniqueCombineFn(long sampleSize, Coder coder) { + this.sampleSize = sampleSize; + this.coder = coder; + } + + @Override + public LargestUnique createAccumulator() { + return new LargestUnique(sampleSize); + } + + @Override + public void addInput(LargestUnique heap, T input) { + try { + heap.add(hash(input, coder)); + } catch (Throwable e) { + throw new RuntimeException(e); + } + } + + @Override + public LargestUnique mergeAccumulators(Iterable heaps) { + Iterator iterator = heaps.iterator(); + LargestUnique heap = iterator.next(); + while (iterator.hasNext()) { + List largestHashes = iterator.next().extractOrderedList(); + for (long hash : largestHashes) { + if (!heap.add(hash)) { + break; // The remainder of this list is all smaller. + } + } + } + return heap; + } + + @Override + public Long extractOutput(LargestUnique heap) { + List largestHashes = heap.extractOrderedList(); + if (largestHashes.size() < sampleSize) { + return (long) largestHashes.size(); + } else { + long smallestSampleHash = largestHashes.get(largestHashes.size() - 1); + double sampleSpaceSize = Long.MAX_VALUE - (double) smallestSampleHash; + // This formula takes into account the possibility of hash collisions, + // which become more likely than not for 2^32 distinct elements. + // Note that log(1+x) ~ x for small x, so for sampleSize << maxHash + // log(1 - sampleSize/sampleSpace) / log(1 - 1/sampleSpace) ~ sampleSize + // and hence estimate ~ sampleSize * HASH_SPACE_SIZE / sampleSpace + // as one would expect. + double estimate = Math.log1p(-sampleSize / sampleSpaceSize) + / Math.log1p(-1 / sampleSpaceSize) + * HASH_SPACE_SIZE / sampleSpaceSize; + return Math.round(estimate); + } + } + + @Override + public Coder getAccumulatorCoder(CoderRegistry registry, + Coder inputCoder) { + return SerializableCoder.of(LargestUnique.class); + } + + /** + * Encodes the given element using the given coder and hashes the encoding. + */ + static long hash(T element, Coder coder) + throws CoderException, IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + coder.encode(element, baos, Context.OUTER); + return Hashing.murmur3_128().hashBytes(baos.toByteArray()).asLong(); + } + } + + /** + * Computes the sampleSize based on the desired estimation error. + * + * @param estimationError should be bounded by [0.01, 0.5] + * @return the sample size needed for the desired estimation error + */ + static long sampleSizeFromEstimationError(double estimationError) { + return Math.round(Math.ceil(4.0 / Math.pow(estimationError, 2.0))); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java new file mode 100644 index 0000000000000..9b374665451ed --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -0,0 +1,1045 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + +/** + * {@code PTransform}s for combining {@code PCollection} elements + * globally and per-key. + */ +public class Combine { + + /** + * Returns a {@link Globally Combine.Globally} {@code PTransform} + * that uses the given {@code SerializableFunction} to combine all + * the elements of the input {@code PCollection} into a singleton + * {@code PCollection} value. The types of the input elements and the + * output value must be the same. + * + *

If the input {@code PCollection} is empty, the ouput will contain a the + * default value of the combining function if the input is windowed into + * the {@link GlobalWindow}; otherwise, the output will be empty. Note: this + * behavior is subject to change. + * + *

See {@link Globally Combine.Globally} for more information. + */ + public static Globally globally( + SerializableFunction, V> combiner) { + return globally(SimpleCombineFn.of(combiner)); + } + + /** + * Returns a {@link Globally Combine.Globally} {@code PTransform} + * that uses the given {@code CombineFn} to combine all the elements + * of the input {@code PCollection} into a singleton {@code PCollection} + * value. The types of the input elements and the output value can + * differ. + * + * If the input {@code PCollection} is empty, the ouput will contain a the + * default value of the combining function if the input is windowed into + * the {@link GlobalWindow}; otherwise, the output will be empty. Note: this + * behavior is subject to change. + * + *

See {@link Globally Combine.Globally} for more information. + */ + public static Globally globally( + CombineFn fn) { + return new Globally<>(fn); + } + + /** + * Returns a {@link PerKey Combine.PerKey} {@code PTransform} that + * first groups its input {@code PCollection} of {@code KV}s by keys and + * windows, then invokes the given function on each of the values lists to + * produce a combined value, and then returns a {@code PCollection} + * of {@code KV}s mapping each distinct key to its combined value for each + * window. + * + *

Each output element is in the window by which its corresponding input + * was grouped, and has the timestamp of the end of that window. The output + * {@code PCollection} has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * as the input. + * + *

See {@link PerKey Combine.PerKey} for more information. + */ + public static PerKey perKey( + SerializableFunction, V> fn) { + return perKey(Combine.SimpleCombineFn.of(fn)); + } + + /** + * Returns a {@link PerKey Combine.PerKey} {@code PTransform} that + * first groups its input {@code PCollection} of {@code KV}s by keys and + * windows, then invokes the given function on each of the values lists to + * produce a combined value, and then returns a {@code PCollection} + * of {@code KV}s mapping each distinct key to its combined value for each + * window. + * + *

Each output element is in the window by which its corresponding input + * was grouped, and has the timestamp of the end of that window. The output + * {@code PCollection} has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * as the input. + * + *

See {@link PerKey Combine.PerKey} for more information. + */ + public static PerKey perKey( + CombineFn fn) { + return perKey(fn.asKeyedFn()); + } + + /** + * Returns a {@link PerKey Combine.PerKey} {@code PTransform} that + * first groups its input {@code PCollection} of {@code KV}s by keys and + * windows, then invokes the given function on each of the key/values-lists + * pairs to produce a combined value, and then returns a + * {@code PCollection} of {@code KV}s mapping each distinct key to + * its combined value for each window. + * + *

Each output element is in the window by which its corresponding input + * was grouped, and has the timestamp of the end of that window. The output + * {@code PCollection} has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * as the input. + * + *

See {@link PerKey Combine.PerKey} for more information. + */ + public static PerKey perKey( + KeyedCombineFn fn) { + return new PerKey<>(fn); + } + + /** + * Returns a {@link GroupedValues Combine.GroupedValues} + * {@code PTransform} that takes a {@code PCollection} of + * {@code KV}s where a key maps to an {@code Iterable} of values, e.g., + * the result of a {@code GroupByKey}, then uses the given + * {@code SerializableFunction} to combine all the values associated + * with a key, ignoring the key. The type of the input and + * output values must be the same. + * + *

Each output element has the same timestamp and is in the same window + * as its corresponding input element, and the output + * {@code PCollection} has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * associated with it as the input. + * + *

See {@link GroupedValues Combine.GroupedValues} for more information. + * + *

Note that {@link #perKey(SerializableFunction)} is typically + * more convenient to use than {@link GroupByKey} followed by + * {@code groupedValues(...)}. + */ + public static GroupedValues groupedValues( + SerializableFunction, V> fn) { + return groupedValues(SimpleCombineFn.of(fn)); + } + + /** + * Returns a {@link GroupedValues Combine.GroupedValues} + * {@code PTransform} that takes a {@code PCollection} of + * {@code KV}s where a key maps to an {@code Iterable} of values, e.g., + * the result of a {@code GroupByKey}, then uses the given + * {@code CombineFn} to combine all the values associated with a + * key, ignoring the key. The types of the input and output values + * can differ. + * + *

Each output element has the same timestamp and is in the same window + * as its corresponding input element, and the output + * {@code PCollection} has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * associated with it as the input. + * + *

See {@link GroupedValues Combine.GroupedValues} for more information. + * + *

Note that {@link #perKey(CombineFn)} is typically + * more convenient to use than {@link GroupByKey} followed by + * {@code groupedValues(...)}. + */ + public static GroupedValues groupedValues( + CombineFn fn) { + return groupedValues(fn.asKeyedFn()); + } + + /** + * Returns a {@link GroupedValues Combine.GroupedValues} + * {@code PTransform} that takes a {@code PCollection} of + * {@code KV}s where a key maps to an {@code Iterable} of values, e.g., + * the result of a {@code GroupByKey}, then uses the given + * {@code KeyedCombineFn} to combine all the values associated with + * each key. The combining function is provided the key. The types + * of the input and output values can differ. + * + *

Each output element has the same timestamp and is in the same window + * as its corresponding input element, and the output + * {@code PCollection} has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * associated with it as the input. + * + *

See {@link GroupedValues Combine.GroupedValues} for more information. + * + *

Note that {@link #perKey(KeyedCombineFn)} is typically + * more convenient to use than {@link GroupByKey} followed by + * {@code groupedValues(...)}. + */ + public static GroupedValues groupedValues( + KeyedCombineFn fn) { + return new GroupedValues<>(fn); + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * A {@code CombineFn} specifies how to combine a + * collection of input values of type {@code VI} into a single + * output value of type {@code VO}. It does this via one or more + * intermediate mutable accumulator values of type {@code VA}. + * + *

The overall process to combine a collection of input + * {@code VI} values into a single output {@code VO} value is as + * follows: + * + *

    + * + *
  1. The input {@code VI} values are partitioned into one or more + * batches. + * + *
  2. For each batch, the {@link #createAccumulator} operation is + * invoked to create a fresh mutable accumulator value of type + * {@code VA}, initialized to represent the combination of zero + * values. + * + *
  3. For each input {@code VI} value in a batch, the + * {@link #addInput} operation is invoked to add the value to that + * batch's accumulator {@code VA} value. The accumulator may just + * record the new value (e.g., if {@code VA == List}, or may do + * work to represent the combination more compactly. + * + *
  4. The {@link #mergeAccumulators} operation is invoked to + * combine a collection of accumulator {@code VA} values into a + * single combined output accumulator {@code VA} value, once the + * merging accumulators have had all all the input values in their + * batches added to them. This operation is invoked repeatedly, + * until there is only one accumulator value left. + * + *
  5. The {@link #extractOutput} operation is invoked on the final + * accumulator {@code VA} value to get the output {@code VO} value. + * + *
+ * + *

For example: + *

 {@code
+   * public class AverageFn extends CombineFn {
+   *   public static class Accum {
+   *     int sum = 0;
+   *     int count = 0;
+   *   }
+   *   public Accum createAccumulator() { return new Accum(); }
+   *   public void addInput(Accum accum, Integer input) {
+   *       accum.sum += input;
+   *       accum.count++;
+   *   }
+   *   public Accum mergeAccumulators(Iterable accums) {
+   *     Accum merged = createAccumulator();
+   *     for (Accum accum : accums) {
+   *       merged.sum += accum.sum;
+   *       merged.count += accum.count;
+   *     }
+   *     return merged;
+   *   }
+   *   public Double extractOutput(Accum accum) {
+   *     return ((double) accum.sum) / accum.count;
+   *   }
+   * }
+   * PCollection pc = ...;
+   * PCollection average = pc.apply(Combine.globally(new AverageFn()));
+   * } 
+ * + *

Combining functions used by {@link Combine.Globally}, + * {@link Combine.PerKey}, {@link Combine.GroupedValues}, and + * {@code PTransforms} derived from them should be + * associative and commutative. Associativity is + * required because input values are first broken up into subgroups + * before being combined, and their intermediate results further + * combined, in an arbitrary tree structure. Commutativity is + * required because any order of the input values is ignored when + * breaking up input values into groups. + * + * @param type of input values + * @param type of mutable accumulator values + * @param type of output values + */ + public abstract static class CombineFn implements Serializable { + /** + * Returns a new, mutable accumulator value, representing the + * accumulation of zero input values. + */ + public abstract VA createAccumulator(); + + /** + * Adds the given input value to the given accumulator, + * modifying the accumulator. + */ + public abstract void addInput(VA accumulator, VI input); + + /** + * Returns an accumulator representing the accumulation of all the + * input values accumulated in the merging accumulators. + * + *

May modify any of the argument accumulators. May return a + * fresh accumulator, or may return one of the (modified) argument + * accumulators. + */ + public abstract VA mergeAccumulators(Iterable accumulators); + + /** + * Returns the output value that is the result of combining all + * the input values represented by the given accumulator. + */ + public abstract VO extractOutput(VA accumulator); + + /** + * Applies this {@code CombineFn} to a collection of input values + * to produce a combined output value. + * + *

Useful when testing the behavior of a {@code CombineFn} + * separately from a {@code Combine} transform. + */ + public VO apply(Iterable inputs) { + VA accum = createAccumulator(); + for (VI input : inputs) { + addInput(accum, input); + } + return extractOutput(accum); + } + + /** + * Returns the {@code Coder} to use for accumulator {@code VA} + * values, or null if it is not able to be inferred. + * + *

By default, uses the knowledge of the {@code Coder} being used + * for {@code VI} values and the enclosing {@code Pipeline}'s + * {@code CoderRegistry} to try to infer the Coder for {@code VA} + * values. + */ + public Coder getAccumulatorCoder( + CoderRegistry registry, Coder inputCoder) { + return registry.getDefaultCoder( + getClass(), + CombineFn.class, + ImmutableMap.of("VI", inputCoder), + "VA"); + } + + /** + * Returns the {@code Coder} to use by default for output + * {@code VO} values, or null if it is not able to be inferred. + * + *

By default, uses the knowledge of the {@code Coder} being + * used for input {@code VI} values and the enclosing + * {@code Pipeline}'s {@code CoderRegistry} to try to infer the + * Coder for {@code VO} values. + */ + public Coder getDefaultOutputCoder( + CoderRegistry registry, Coder inputCoder) { + return registry.getDefaultCoder( + getClass(), + CombineFn.class, + ImmutableMap.of("VI", inputCoder, + "VA", getAccumulatorCoder(registry, inputCoder)), + "VO"); + } + + /** + * Converts this {@code CombineFn} into an equivalent + * {@link KeyedCombineFn}, which ignores the keys passed to it and + * combines the values according to this {@code CombineFn}. + * + * @param the type of the (ignored) keys + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public KeyedCombineFn asKeyedFn() { + // The key, an object, is never even looked at. + return new KeyedCombineFn() { + @Override + public VA createAccumulator(K key) { + return CombineFn.this.createAccumulator(); + } + + @Override + public void addInput(K key, VA accumulator, VI input) { + CombineFn.this.addInput(accumulator, input); + } + + @Override + public VA mergeAccumulators(K key, Iterable accumulators) { + return CombineFn.this.mergeAccumulators(accumulators); + } + + @Override + public VO extractOutput(K key, VA accumulator) { + return CombineFn.this.extractOutput(accumulator); + } + + @Override + public Coder getAccumulatorCoder( + CoderRegistry registry, Coder keyCoder, Coder inputCoder) { + return CombineFn.this.getAccumulatorCoder(registry, inputCoder); + } + + @Override + public Coder getDefaultOutputCoder( + CoderRegistry registry, Coder keyCoder, Coder inputCoder) { + return CombineFn.this.getDefaultOutputCoder(registry, inputCoder); + } + }; + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * A {@code CombineFn} that uses a subclass of + * {@link AccumulatingCombineFn.Accumulator} as its accumulator + * type. By defining the operations of the {@code Accumulator} + * helper class, the operations of the enclosing {@code CombineFn} + * are automatically provided. This can reduce the code required to + * implement a {@code CombineFn}. + * + *

For example, the example from {@link CombineFn} above can be + * expressed using {@code AccumulatingCombineFn} more concisely as + * follows: + * + *

 {@code
+   * public class AverageFn
+   *     extends AccumulatingCombineFn {
+   *   public Accum createAccumulator() { return new Accum(); }
+   *   public class Accum
+   *       extends AccumulatingCombineFn
+   *               .Accumulator {
+   *     private int sum = 0;
+   *     private int count = 0;
+   *     public void addInput(Integer input) {
+   *       sum += input;
+   *       count++;
+   *     }
+   *     public void mergeAccumulator(Accum other) {
+   *       sum += other.sum;
+   *       count += other.count;
+   *     }
+   *     public Double extractOutput() {
+   *       return ((double) sum) / count;
+   *     }
+   *   }
+   * }
+   * PCollection pc = ...;
+   * PCollection average = pc.apply(Combine.globally(new AverageFn()));
+   * } 
+ * + * @param type of input values + * @param type of mutable accumulator values + * @param type of output values + */ + public abstract static class AccumulatingCombineFn + .Accumulator, VO> + extends CombineFn { + + /** + * The type of mutable accumulator values used by this + * {@code AccumulatingCombineFn}. + */ + public abstract class Accumulator implements Serializable { + /** + * Adds the given input value to this accumulator, modifying + * this accumulator. + */ + public abstract void addInput(VI input); + + /** + * Adds the input values represented by the given accumulator + * into this accumulator. + */ + public abstract void mergeAccumulator(VA other); + + /** + * Returns the output value that is the result of combining all + * the input values represented by this accumulator. + */ + public abstract VO extractOutput(); + } + + @Override + public final void addInput(VA accumulator, VI input) { + accumulator.addInput(input); + } + + @Override + public final VA mergeAccumulators(Iterable accumulators) { + VA accumulator = createAccumulator(); + for (VA partial : accumulators) { + accumulator.mergeAccumulator(partial); + } + return accumulator; + } + + @Override + public final VO extractOutput(VA accumulator) { + return accumulator.extractOutput(); + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + + /** + * A {@code KeyedCombineFn} specifies how to combine + * a collection of input values of type {@code VI}, associated with + * a key of type {@code K}, into a single output value of type + * {@code VO}. It does this via one or more intermediate mutable + * accumulator values of type {@code VA}. + * + *

The overall process to combine a collection of input + * {@code VI} values associated with an input {@code K} key into a + * single output {@code VO} value is as follows: + * + *

    + * + *
  1. The input {@code VI} values are partitioned into one or more + * batches. + * + *
  2. For each batch, the {@link #createAccumulator} operation is + * invoked to create a fresh mutable accumulator value of type + * {@code VA}, initialized to represent the combination of zero + * values. + * + *
  3. For each input {@code VI} value in a batch, the + * {@link #addInput} operation is invoked to add the value to that + * batch's accumulator {@code VA} value. The accumulator may just + * record the new value (e.g., if {@code VA == List}, or may do + * work to represent the combination more compactly. + * + *
  4. The {@link #mergeAccumulators} operation is invoked to + * combine a collection of accumulator {@code VA} values into a + * single combined output accumulator {@code VA} value, once the + * merging accumulators have had all all the input values in their + * batches added to them. This operation is invoked repeatedly, + * until there is only one accumulator value left. + * + *
  5. The {@link #extractOutput} operation is invoked on the final + * accumulator {@code VA} value to get the output {@code VO} value. + * + *
+ * + * All of these operations are passed the {@code K} key that the + * values being combined are associated with. + * + *

For example: + *

 {@code
+   * public class ConcatFn
+   *     extends KeyedCombineFn {
+   *   public static class Accum {
+   *     String s = "";
+   *   }
+   *   public Accum createAccumulator(String key) { return new Accum(); }
+   *   public void addInput(String key, Accum accum, Integer input) {
+   *       accum.s += "+" + input;
+   *   }
+   *   public Accum mergeAccumulators(String key, Iterable accums) {
+   *     Accum merged = new Accum();
+   *     for (Accum accum : accums) {
+   *       merged.s += accum.s;
+   *     }
+   *     return merged;
+   *   }
+   *   public String extractOutput(String key, Accum accum) {
+   *     return key + accum.s;
+   *   }
+   * }
+   * PCollection> pc = ...;
+   * PCollection> pc2 = pc.apply(
+   *     Combine.perKey(new ConcatFn()));
+   * } 
+ * + *

Keyed combining functions used by {@link Combine.PerKey}, + * {@link Combine.GroupedValues}, and {@code PTransforms} derived + * from them should be associative and commutative. + * Associativity is required because input values are first broken + * up into subgroups before being combined, and their intermediate + * results further combined, in an arbitrary tree structure. + * Commutativity is required because any order of the input values + * is ignored when breaking up input values into groups. + * + * @param type of keys + * @param type of input values + * @param type of mutable accumulator values + * @param type of output values + */ + public abstract static class KeyedCombineFn + implements Serializable { + /** + * Returns a new, mutable accumulator value representing the + * accumulation of zero input values. + * + * @param key the key that all the accumulated values using the + * accumulator are associated with + */ + public abstract VA createAccumulator(K key); + + /** + * Adds the given input value to the given accumulator, + * modifying the accumulator. + * + * @param key the key that all the accumulated values using the + * accumulator are associated with + */ + public abstract void addInput(K key, VA accumulator, VI value); + + /** + * Returns an accumulator representing the accumulation of all the + * input values accumulated in the merging accumulators. + * + *

May modify any of the argument accumulators. May return a + * fresh accumulator, or may return one of the (modified) argument + * accumulators. + * + * @param key the key that all the accumulators are associated + * with + */ + public abstract VA mergeAccumulators(K key, Iterable accumulators); + + /** + * Returns the output value that is the result of combining all + * the input values represented by the given accumulator. + * + * @param key the key that all the accumulated values using the + * accumulator are associated with + */ + public abstract VO extractOutput(K key, VA accumulator); + + /** + * Applies this {@code KeyedCombineFn} to a key and a collection + * of input values to produce a combined output value. + * + *

Useful when testing the behavior of a {@code KeyedCombineFn} + * separately from a {@code Combine} transform. + */ + public VO apply(K key, Iterable inputs) { + VA accum = createAccumulator(key); + for (VI input : inputs) { + addInput(key, accum, input); + } + return extractOutput(key, accum); + } + + /** + * Returns the {@code Coder} to use for accumulator {@code VA} + * values, or null if it is not able to be inferred. + * + *

By default, uses the knowledge of the {@code Coder} being + * used for {@code K} keys and input {@code VI} values and the + * enclosing {@code Pipeline}'s {@code CoderRegistry} to try to + * infer the Coder for {@code VA} values. + */ + public Coder getAccumulatorCoder( + CoderRegistry registry, Coder keyCoder, Coder inputCoder) { + return registry.getDefaultCoder( + getClass(), + KeyedCombineFn.class, + ImmutableMap.of("K", keyCoder, "VI", inputCoder), + "VA"); + } + + /** + * Returns the {@code Coder} to use by default for output + * {@code VO} values, or null if it is not able to be inferred. + * + *

By default, uses the knowledge of the {@code Coder} being + * used for {@code K} keys and input {@code VI} values and the + * enclosing {@code Pipeline}'s {@code CoderRegistry} to try to + * infer the Coder for {@code VO} values. + */ + public Coder getDefaultOutputCoder( + CoderRegistry registry, Coder keyCoder, Coder inputCoder) { + return registry.getDefaultCoder( + getClass(), + KeyedCombineFn.class, + ImmutableMap.of( + "K", keyCoder, + "VI", inputCoder, + "VA", getAccumulatorCoder(registry, keyCoder, inputCoder)), + "VO"); + } + } + + + //////////////////////////////////////////////////////////////////////////// + + /** + * {@code Combine.Globally} takes a {@code PCollection} + * and returns a {@code PCollection} whose single element is the result of + * combining all the elements of the input {@code PCollection}, + * using a specified + * {@link CombineFn CombineFn}. It is common + * for {@code VI == VO}, but not required. Common combining + * functions include sums, mins, maxes, and averages of numbers, + * conjunctions and disjunctions of booleans, statistical + * aggregations, etc. + * + *

Example of use: + *

 {@code
+   * PCollection pc = ...;
+   * PCollection sum = pc.apply(
+   *     Combine.globally(new Sum.SumIntegerFn()));
+   * } 
+ * + *

Combining can happen in parallel, with different subsets of the + * input {@code PCollection} being combined separately, and their + * intermediate results combined further, in an arbitrary tree + * reduction pattern, until a single result value is produced. + * + *

By default, the {@code Coder} of the output {@code PValue} + * is inferred from the concrete type of the + * {@code CombineFn}'s output type {@code VO}. + * + *

See also {@link #perKey}/{@link PerKey Combine.PerKey} and + * {@link #groupedValues}/{@link GroupedValues Combine.GroupedValues}, + * which are useful for combining values associated with each key in + * a {@code PCollection} of {@code KV}s. + * + * @param type of input values + * @param type of output values + */ + public static class Globally + extends PTransform, PCollection> { + + private final CombineFn fn; + + private Globally(CombineFn fn) { + this.fn = fn; + } + + @Override + public PCollection apply(PCollection input) { + PCollection output = input + .apply(WithKeys.of((Void) null)) + .setCoder(KvCoder.of(VoidCoder.of(), input.getCoder())) + .apply(Combine.perKey(fn.asKeyedFn())) + .apply(Values.create()); + + if (input.getWindowingFn().isCompatible(new GlobalWindow())) { + return insertDefaultValueIfEmpty(output); + } else { + return output; + } + } + + private PCollection insertDefaultValueIfEmpty(PCollection maybeEmpty) { + final PCollectionView, ?> maybeEmptyView = maybeEmpty.apply( + View.asIterable()); + return maybeEmpty.getPipeline() + .apply(Create.of((Void) null)).setCoder(VoidCoder.of()) + .apply(ParDo.of( + new DoFn() { + @Override + public void processElement(DoFn.ProcessContext c) { + Iterator combined = c.sideInput(maybeEmptyView).iterator(); + if (combined.hasNext()) { + c.output(combined.next()); + } else { + c.output(fn.apply(Collections.emptyList())); + } + } + }).withSideInputs(maybeEmptyView)) + .setCoder(maybeEmpty.getCoder()); + } + + @Override + protected String getKindString() { + return "Combine.Globally"; + } + } + + /** + * Converts a {@link SerializableFunction} from {@code Iterable}s + * to {@code V}s into a simple {@link CombineFn} over {@code V}s. + * + *

Used in the implementation of convenience methods like + * {@link #globally(SerializableFunction)}, + * {@link #perKey(SerializableFunction)}, and + * {@link #groupedValues(SerializableFunction)}. + */ + static class SimpleCombineFn extends CombineFn, V> { + /** + * Returns a {@code CombineFn} that uses the given + * {@code SerializableFunction} to combine values. + */ + public static SimpleCombineFn of( + SerializableFunction, V> combiner) { + return new SimpleCombineFn<>(combiner); + } + + /** + * The number of values to accumulate before invoking the combiner + * function to combine them. + */ + private static final int BUFFER_SIZE = 20; + + /** The combiner function. */ + private final SerializableFunction, V> combiner; + + private SimpleCombineFn(SerializableFunction, V> combiner) { + this.combiner = combiner; + } + + @Override + public List createAccumulator() { + return new ArrayList<>(); + } + + @Override + public void addInput(List accumulator, V input) { + accumulator.add(input); + if (accumulator.size() > BUFFER_SIZE) { + V combined = combiner.apply(accumulator); + accumulator.clear(); + accumulator.add(combined); + } + } + + @Override + public List mergeAccumulators(Iterable> accumulators) { + List singleton = new ArrayList<>(); + singleton.add(combiner.apply(Iterables.concat(accumulators))); + return singleton; + } + + @Override + public V extractOutput(List accumulator) { + return combiner.apply(accumulator); + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * {@code PerKey} takes a + * {@code PCollection>}, groups it by key, applies a + * combining function to the {@code VI} values associated with each + * key to produce a combined {@code VO} value, and returns a + * {@code PCollection>} representing a map from each + * distinct key of the input {@code PCollection} to the corresponding + * combined value. {@code VI} and {@code VO} are often the same. + * + *

This is a concise shorthand for an application of + * {@link GroupByKey} followed by an application of + * {@link GroupedValues Combine.GroupedValues}. See those + * operations for more details on how keys are compared for equality + * and on the default {@code Coder} for the output. + * + *

Example of use: + *

 {@code
+   * PCollection> salesRecords = ...;
+   * PCollection> totalSalesPerPerson =
+   *     salesRecords.apply(Combine.perKey(
+   *         new Sum.SumDoubleFn()));
+   * } 
+ * + *

Each output element is in the window by which its corresponding input + * was grouped, and has the timestamp of the end of that window. The output + * {@code PCollection} has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * as the input. + * + * @param the type of the keys of the input and output + * {@code PCollection}s + * @param the type of the values of the input {@code PCollection} + * @param the type of the values of the output {@code PCollection} + */ + public static class PerKey + extends PTransform>, PCollection>> { + + private final transient KeyedCombineFn fn; + + private PerKey( + KeyedCombineFn fn) { + this.fn = fn; + } + + @Override + public PCollection> apply(PCollection> input) { + return input + .apply(GroupByKey.create()) + .apply(Combine.groupedValues(fn)); + } + + @Override + protected String getKindString() { + return "Combine.PerKey"; + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * {@code GroupedValues} takes a + * {@code PCollection>>}, such as the result of + * {@link GroupByKey}, applies a specified + * {@link KeyedCombineFn KeyedCombineFn} + * to each of the input {@code KV>} elements to + * produce a combined output {@code KV} element, and returns a + * {@code PCollection>} containing all the combined output + * elements. It is common for {@code VI == VO}, but not required. + * Common combining functions include sums, mins, maxes, and averages + * of numbers, conjunctions and disjunctions of booleans, statistical + * aggregations, etc. + * + *

Example of use: + *

 {@code
+   * PCollection> pc = ...;
+   * PCollection>> groupedByKey = pc.apply(
+   *     new GroupByKey());
+   * PCollection> sumByKey = groupedByKey.apply(
+   *     Combine.groupedValues(
+   *         new Sum.SumIntegerFn()));
+   * } 
+ * + *

See also {@link #perKey}/{@link PerKey Combine.PerKey} + * which captures the common pattern of "combining by key" in a + * single easy-to-use {@code PTransform}. + * + *

Combining for different keys can happen in parallel. Moreover, + * combining of the {@code Iterable} values associated a single + * key can happen in parallel, with different subsets of the values + * being combined separately, and their intermediate results combined + * further, in an arbitrary tree reduction pattern, until a single + * result value is produced for each key. + * + *

By default, the {@code Coder} of the keys of the output + * {@code PCollection>} is that of the keys of the input + * {@code PCollection>}, and the {@code Coder} of the values + * of the output {@code PCollection>} is inferred from the + * concrete type of the {@code KeyedCombineFn}'s output + * type {@code VO}. + * + *

Each output element has the same timestamp and is in the same window + * as its corresponding input element, and the output + * {@code PCollection} has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * associated with it as the input. + * + *

See also {@link #globally}/{@link Globally Combine.Globally}, + * which combines all the values in a {@code PCollection} into a + * single value in a {@code PCollection}. + * + * @param type of input and output keys + * @param type of input values + * @param type of output values + */ + public static class GroupedValues + extends PTransform + >>, + PCollection>> { + + private final KeyedCombineFn fn; + + private GroupedValues(KeyedCombineFn fn) { + this.fn = fn; + } + + /** + * Returns the KeyedCombineFn used by this Combine operation. + */ + public KeyedCombineFn getFn() { + return fn; + } + + @Override + public PCollection> apply( + PCollection>> input) { + Coder> outputCoder = getDefaultOutputCoder(); + return input.apply(ParDo.of( + new DoFn>, KV>() { + @Override + public void processElement(ProcessContext c) { + K key = c.element().getKey(); + c.output(KV.of(key, fn.apply(key, c.element().getValue()))); + } + })).setCoder(outputCoder); + } + + private KvCoder getKvCoder() { + Coder>> inputCoder = + getInput().getCoder(); + if (!(inputCoder instanceof KvCoder)) { + throw new IllegalStateException( + "Combine.GroupedValues requires its input to use KvCoder"); + } + @SuppressWarnings({"unchecked", "rawtypes"}) + KvCoder> kvCoder = (KvCoder) inputCoder; + Coder keyCoder = kvCoder.getKeyCoder(); + Coder> kvValueCoder = kvCoder.getValueCoder(); + if (!(kvValueCoder instanceof IterableCoder)) { + throw new IllegalStateException( + "Combine.GroupedValues requires its input values to use " + + "IterableCoder"); + } + IterableCoder inputValuesCoder = (IterableCoder) kvValueCoder; + Coder inputValueCoder = inputValuesCoder.getElemCoder(); + return KvCoder.of(keyCoder, inputValueCoder); + } + + @SuppressWarnings("unchecked") + public Coder getAccumulatorCoder() { + KvCoder kvCoder = getKvCoder(); + return ((KeyedCombineFn) fn).getAccumulatorCoder( + getCoderRegistry(), kvCoder.getKeyCoder(), kvCoder.getValueCoder()); + } + + @Override + public Coder> getDefaultOutputCoder() { + KvCoder kvCoder = getKvCoder(); + @SuppressWarnings("unchecked") + Coder outputValueCoder = ((KeyedCombineFn) fn) + .getDefaultOutputCoder( + getCoderRegistry(), kvCoder.getKeyCoder(), kvCoder.getValueCoder()); + return KvCoder.of(kvCoder.getKeyCoder(), outputValueCoder); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java new file mode 100644 index 0000000000000..1303b0a98634a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java @@ -0,0 +1,163 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +/** + * {@code Count} takes a {@code PCollection} and returns a + * {@code PCollection>} representing a map from each + * distinct element of the input {@code PCollection} to the number of times + * that element occurs in the input. Each of the keys in the output + * {@code PCollection} is unique. + * + *

Two values of type {@code T} are compared for equality not by + * regular Java {@link Object#equals}, but instead by first encoding + * each of the elements using the {@code PCollection}'s {@code Coder}, and then + * comparing the encoded bytes. This admits efficient parallel + * evaluation. + * + *

By default, the {@code Coder} of the keys of the output + * {@code PCollection} is the same as the {@code Coder} of the + * elements of the input {@code PCollection}. + * + *

Each output element is in the window by which its corresponding input + * was grouped, and has the timestamp of the end of that window. The output + * {@code PCollection} has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * as the input. + * + *

Example of use: + *

 {@code
+ * PCollection words = ...;
+ * PCollection> wordCounts =
+ *     words.apply(Count.create());
+ * } 
+ */ +public class Count { + + /** + * Returns a {@link Globally Count.Globally} {@link PTransform} + * that counts the number of elements in its input {@link PCollection}. + * + *

See {@link Globally Count.Globally} for more details. + */ + public static Globally globally() { + return new Globally<>(); + } + + /** + * Returns a {@link PerElement Count.PerElement} {@link PTransform} + * that counts the number of occurrences of each element in its + * input {@link PCollection}. + * + *

See {@link PerElement Count.PerElement} for more details. + */ + public static PerElement perElement() { + return new PerElement<>(); + } + + /////////////////////////////////////// + + /** + * {@code Count.Globally} takes a {@code PCollection} and returns a + * {@code PCollection} containing a single element which is the total + * number of elements in the {@code PCollection}. + * + *

Example of use: + *

 {@code
+   * PCollection words = ...;
+   * PCollection wordCount =
+   *     words.apply(Count.globally());
+   * } 
+ * + * @param the type of the elements of the input {@code PCollection} + */ + public static class Globally + extends PTransform, PCollection> { + + public Globally() { } + + @Override + public PCollection apply(PCollection input) { + return + input + .apply(ParDo.named("Init") + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(1L); + } + })) + .apply(Sum.longsGlobally()); + } + } + + /** + * {@code Count.PerElement} takes a {@code PCollection} and returns a + * {@code PCollection>} representing a map from each + * distinct element of the input {@code PCollection} to the number of times + * that element occurs in the input. Each of the keys in the output + * {@code PCollection} is unique. + * + *

This transform compares two values of type {@code T} by first + * encoding each element using the input {@code PCollection}'s + * {@code Coder}, then comparing the encoded bytes. Because of this, + * the input coder must be deterministic. (See + * {@link com.google.cloud.dataflow.sdk.coders.Coder#isDeterministic()} for more detail). + * Performing the comparison in this manner admits efficient parallel evaluation. + * + *

By default, the {@code Coder} of the keys of the output + * {@code PCollection} is the same as the {@code Coder} of the + * elements of the input {@code PCollection}. + * + *

Example of use: + *

 {@code
+   * PCollection words = ...;
+   * PCollection> wordCounts =
+   *     words.apply(Count.perElement());
+   * } 
+ * + * @param the type of the elements of the input {@code PCollection}, and + * the type of the keys of the output {@code PCollection} + */ + public static class PerElement + extends PTransform, PCollection>> { + + public PerElement() { } + + @Override + public PCollection> apply(PCollection input) { + return + input + .apply(ParDo.named("Init") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + c.output(KV.of(c.element(), 1L)); + } + })) + .apply(Sum.longsPerKey()); + } + + @Override + public String getKindString() { + return "Count.PerElement"; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java new file mode 100644 index 0000000000000..93747ea6462f7 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java @@ -0,0 +1,314 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PBegin; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.TimestampedValue; +import com.google.common.reflect.TypeToken; + +import org.joda.time.Instant; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * {@code Create} takes a collection of elements of type {@code T} + * known when the pipeline is constructed and returns a + * {@code PCollection} containing the elements. + * + *

Example of use: + *

 {@code
+ * Pipeline p = ...;
+ *
+ * PCollection pc = p.apply(Create.of(3, 4, 5)).setCoder(BigEndianIntegerCoder.of());
+ *
+ * Map map = ...;
+ * PCollection> pt =
+ *     p.apply(Create.of(map))
+ *      .setCoder(KvCoder.of(StringUtf8Coder.of(),
+ *                           BigEndianIntegerCoder.of()));
+ * } 
+ * + *

Note that {@link PCollection#setCoder} must be called + * explicitly to set the encoding of the resulting + * {@code PCollection}, since {@code Create} does not infer the + * encoding. + * + *

A good use for {@code Create} is when a {@code PCollection} + * needs to be created without dependencies on files or other external + * entities. This is especially useful during testing. + * + *

Caveat: {@code Create} only supports small in-memory datasets, + * particularly when submitting jobs to the Google Cloud Dataflow + * service. + * + *

{@code Create} can automatically determine the {@code Coder} to use + * if all elements are the same type, and a default exists for that type. + * See {@link com.google.cloud.dataflow.sdk.coders.CoderRegistry} for details + * on how defaults are determined. + * + * @param the type of the elements of the resulting {@code PCollection} + */ +public class Create extends PTransform> { + + /** + * Returns a new {@code Create} root transform that produces a + * {@link PCollection} containing the specified elements. + * + *

The argument should not be modified after this is called. + * + *

The elements will have a timestamp of negative infinity, see + * {@link Create#timestamped} for a way of creating a {@code PCollection} + * with timestamped elements. + * + *

The result of applying this transform should have its + * {@link Coder} specified explicitly, via a call to + * {@link PCollection#setCoder}. + */ + public static Create of(Iterable elems) { + return new Create<>(elems); + } + + /** + * Returns a new {@code Create} root transform that produces a + * {@link PCollection} containing the specified elements. + * + *

The elements will have a timestamp of negative infinity, see + * {@link Create#timestamped} for a way of creating a {@code PCollection} + * with timestamped elements. + * + *

The argument should not be modified after this is called. + * + *

The result of applying this transform should have its + * {@link Coder} specified explicitly, via a call to + * {@link PCollection#setCoder}. + */ + public static Create of(T... elems) { + return of(Arrays.asList(elems)); + } + + /** + * Returns a new {@code Create} root transform that produces a + * {@link PCollection} of {@link KV}s corresponding to the keys and + * values of the specified {@code Map}. + * + *

The elements will have a timestamp of negative infinity, see + * {@link Create#timestamped} for a way of creating a {@code PCollection} + * with timestamped elements. + * + *

The result of applying this transform should have its + * {@link Coder} specified explicitly, via a call to + * {@link PCollection#setCoder}. + */ + public static Create> of(Map elems) { + List> kvs = new ArrayList<>(elems.size()); + for (Map.Entry entry : elems.entrySet()) { + kvs.add(KV.of(entry.getKey(), entry.getValue())); + } + return of(kvs); + } + + /** + * Returns a new root transform that produces a {@link PCollection} containing + * the specified elements with the specified timestamps. + * + *

The argument should not be modified after this is called. + */ + public static CreateTimestamped timestamped(Iterable> elems) { + return new CreateTimestamped<>(elems); + } + + /** + * Returns a new root transform that produces a {@link PCollection} containing + * the specified elements with the specified timestamps. + * + *

The argument should not be modified after this is called. + */ + public static CreateTimestamped timestamped(TimestampedValue... elems) { + return new CreateTimestamped(Arrays.asList(elems)); + } + + /** + * Returns a new root transform that produces a {@link PCollection} containing + * the specified elements with the specified timestamps. + * + *

The arguments should not be modified after this is called. + * + * @throws IllegalArgumentException if there are a different number of values + * and timestamps + */ + public static CreateTimestamped timestamped( + Iterable values, Iterable timestamps) { + List> elems = new ArrayList<>(); + Iterator valueIter = values.iterator(); + Iterator timestampIter = timestamps.iterator(); + while (valueIter.hasNext() && timestampIter.hasNext()) { + elems.add(TimestampedValue.of(valueIter.next(), new Instant(timestampIter.next()))); + } + Preconditions.checkArgument( + !valueIter.hasNext() && !timestampIter.hasNext(), + "Expect sizes of values and timestamps are same."); + return new CreateTimestamped<>(elems); + } + + @Override + public PCollection apply(PInput input) { + return PCollection.createPrimitiveOutputInternal(new GlobalWindow()); + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** The elements of the resulting PCollection. */ + private final Iterable elems; + + /** + * Constructs a {@code Create} transform that produces a + * {@link PCollection} containing the specified elements. + * + *

The argument should not be modified after this is called. + */ + private Create(Iterable elems) { + this.elems = elems; + } + + public Iterable getElements() { + return elems; + } + + @Override + protected Coder getDefaultOutputCoder() { + // First try to deduce a coder using the types of the elements. + Class elementType = null; + for (T elem : elems) { + Class type = elem.getClass(); + if (elementType == null) { + elementType = type; + } else if (!elementType.equals(type)) { + // Elements are not the same type, require a user-specified coder. + elementType = null; + break; + } + } + if (elementType == null) { + return super.getDefaultOutputCoder(); + } + if (elementType.getTypeParameters().length == 0) { + Coder candidate = getCoderRegistry().getDefaultCoder(TypeToken.of(elementType)); + if (candidate != null) { + return candidate; + } + } + + // If that fails, try to deduce a coder using the elements themselves + Coder coder = null; + for (T elem : elems) { + Coder c = getCoderRegistry().getDefaultCoder(elem); + if (coder == null) { + coder = c; + } else if (!Objects.equals(c, coder)) { + coder = null; + break; + } + } + if (coder != null) { + return coder; + } + + return super.getDefaultOutputCoder(); + } + + /** + * A {@code PTransform} that creates a {@code PCollection} whose elements have + * associated timestamps. + */ + private static class CreateTimestamped extends PTransform> { + /** The timestamped elements of the resulting PCollection. */ + private final Iterable> elems; + + private CreateTimestamped(Iterable> elems) { + this.elems = elems; + } + + @Override + public PCollection apply(PBegin input) { + PCollection> intermediate = input.apply(Create.of(elems)); + if (!elems.iterator().hasNext()) { + // There aren't any elements, so we can provide a fake coder instance. + // If we don't set a Coder here, users of CreateTimestamped have + // no way to set the coder of the intermediate PCollection. + intermediate.setCoder((Coder) TimestampedValue.TimestampedValueCoder.of(VoidCoder.of())); + } + + return intermediate.apply(ParDo.of(new ConvertTimestamps())); + } + + private static class ConvertTimestamps extends DoFn, T> { + @Override + public void processElement(ProcessContext c) { + c.outputWithTimestamp(c.element().getValue(), c.element().getTimestamp()); + } + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + Create.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + Create transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateHelper(transform, context); + } + }); + } + + private static void evaluateHelper( + Create transform, + DirectPipelineRunner.EvaluationContext context) { + // Convert the Iterable of elems into a List of elems. + List listElems; + if (transform.elems instanceof Collection) { + Collection collectionElems = (Collection) transform.elems; + listElems = new ArrayList<>(collectionElems.size()); + } else { + listElems = new ArrayList<>(); + } + for (T elem : transform.elems) { + listElems.add( + context.ensureElementEncodable(transform.getOutput(), elem)); + } + context.setPCollection(transform.getOutput(), listElems); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java new file mode 100644 index 0000000000000..3c61ab38557d7 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java @@ -0,0 +1,330 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; +import com.google.cloud.dataflow.sdk.values.CodedTupleTagMap; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.reflect.TypeToken; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.List; + +/** + * The argument to {@link ParDo} providing the code to use to process + * elements of the input + * {@link com.google.cloud.dataflow.sdk.values.PCollection}. + * + *

See {@link ParDo} for more explanation, examples of use, and + * discussion of constraints on {@code DoFn}s, including their + * serializability, lack of access to global shared mutable state, + * requirements for failure tolerance, and benefits of optimization. + * + *

{@code DoFn}s can be tested in the context of a particular + * {@code Pipeline} by running that {@code Pipeline} on sample input + * and then checking its output. Unit testing of a {@code DoFn}, + * separately from any {@code ParDo} transform or {@code Pipeline}, + * can be done via the {@link DoFnTester} harness. + * + * @param the type of the (main) input elements + * @param the type of the (main) output elements + */ +public abstract class DoFn implements Serializable { + + /** Information accessible to all methods in this {@code DoFn}. */ + public abstract class Context { + + /** + * Returns the {@code PipelineOptions} specified with the + * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} + * invoking this {@code DoFn}. The {@code PipelineOptions} will + * be the default running via {@link DoFnTester}. + */ + public abstract PipelineOptions getPipelineOptions(); + + /** + * Returns the value of the side input. + * + * @throws IllegalArgumentException if this is not a side input + * @see ParDo#withSideInput + */ + public abstract T sideInput(PCollectionView view); + + /** + * Adds the given element to the main output {@code PCollection}. + * + *

If invoked from {@link DoFn#processElement}, the output + * element will have the same timestamp and be in the same windows + * as the input element passed to {@link DoFn#processElement}). + * + *

Is is illegal to invoke this from {@link #startBundle} or + * {@link #finishBundle} unless the input {@code PCollection} is + * windowed by the + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. + * If this is the case, the output element will have a timestamp + * of negative infinity and be in the + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. + */ + public abstract void output(O output); + + /** + * Adds the given element to the main output {@code PCollection}, + * with the given timestamp. + * + *

If invoked from {@link DoFn#processElement}), the timestamp + * must not be older than the input element's timestamp minus + * {@link DoFn#getAllowedTimestampSkew}. The output element will + * be in the same windows as the input element. + * + *

Is is illegal to invoke this from {@link #startBundle} or + * {@link #finishBundle} unless the input {@code PCollection} is + * windowed by the + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. + * If this is the case, the output element's timestamp will be + * the given timestamp and its window will be the + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. + */ + public abstract void outputWithTimestamp(O output, Instant timestamp); + + /** + * Adds the given element to the side output {@code PCollection} with the + * given tag. + * + *

The caller of {@code ParDo} uses {@link ParDo#withOutputTags} to + * specify the tags of side outputs that it consumes. Non-consumed side + * outputs, e.g., outputs for monitoring purposes only, don't necessarily + * need to be specified. + * + *

The output element will have the same timestamp and be in the same + * windows as the input element passed to {@link DoFn#processElement}). + * + * @throws IllegalArgumentException if the number of outputs exceeds + * the limit of 1,000 outputs per DoFn + * @see ParDo#withOutputTags + */ + public abstract void sideOutput(TupleTag tag, T output); + + // TODO: add sideOutputWithTimestamp[AndWindows] + + /** + * Returns an aggregator with aggregation logic specified by the CombineFn + * argument. The name provided should be unique across aggregators created + * within the containing ParDo transform application. + * + *

All instances of this DoFn in the containing ParDo + * transform application should define aggregators consistently, + * i.e., an aggregator with a given name always specifies the same + * combiner in all DoFn instances in the containing ParDo + * transform application. + * + * @throws IllegalArgumentException if the given CombineFn is not + * supported as aggregator's combiner, or if the given name collides + * with another aggregator or system-provided counter. + */ + public abstract Aggregator createAggregator( + String name, Combine.CombineFn combiner); + + /** + * Returns an aggregator with aggregation logic specified by the + * SerializableFunction argument. The name provided should be unique across + * aggregators created within the containing ParDo transform application. + * + *

All instances of this DoFn in the containing ParDo + * transform application should define aggregators consistently, + * i.e., an aggregator with a given name always specifies the same + * combiner in all DoFn instances in the containing ParDo + * transform application. + * + * @throws IllegalArgumentException if the given SerializableFunction is + * not supported as aggregator's combiner, or if the given name collides + * with another aggregator or system-provided counter. + */ + public abstract Aggregator createAggregator( + String name, SerializableFunction, AO> combiner); + } + + /** + * Information accessible when running {@link DoFn#processElement}. + */ + public abstract class ProcessContext extends Context { + + /** + * Returns the input element to be processed. + */ + public abstract I element(); + + /** + * Returns this {@code DoFn}'s state associated with the input + * element's key. This state can be used by the {@code DoFn} to + * store whatever information it likes with that key. Unlike + * {@code DoFn} instance variables, this state is persistent and + * can be arbitrarily large; it is more expensive than instance + * variable state, however. It is particularly intended for + * streaming computations. + * + *

Requires that this {@code DoFn} implements + * {@link RequiresKeyedState}. + * + *

Each {@link ParDo} invocation with this {@code DoFn} as an + * argument will maintain its own {@code KeyedState} maps, one per + * key. + * + * @throws UnsupportedOperationException if this {@link DoFn} does + * not implement {@link RequiresKeyedState} + */ + public abstract KeyedState keyedState(); + + /** + * Returns the timestamp of the input element. + * + *

See {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} + * for more information. + */ + public abstract Instant timestamp(); + + /** + * Returns the set of windows to which the input element has been assigned. + * + *

See {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} + * for more information. + */ + public abstract Collection windows(); + } + + /** + * Returns the allowed timestamp skew duration, which is the maximum + * duration that timestamps can be shifted backward in + * {@link DoFn.Context#outputWithTimestamp}. + * + * The default value is {@code Duration.ZERO}, in which case + * timestamps can only be shifted forward to future. For infinite + * skew, return {@code Duration.millis(Long.MAX_VALUE)}. + */ + public Duration getAllowedTimestampSkew() { + return Duration.ZERO; + } + + /** + * Interface for signaling that a {@link DoFn} needs to maintain + * per-key state, accessed via + * {@link DoFn.ProcessContext#keyedState}. + * + *

This functionality is experimental and likely to change. + */ + public interface RequiresKeyedState {} + + /** + * Interface for interacting with keyed state. + * + *

This functionality is experimental and likely to change. + */ + public interface KeyedState { + /** + * Updates this {@code KeyedState} in place so that the given tag + * maps to the given value. + * + * @throws IOException if encoding the given value fails + */ + public void store(CodedTupleTag tag, T value) throws IOException; + + /** + * Returns the value associated with the given tag in this + * {@code KeyedState}, or {@code null} if the tag has no asssociated + * value. + * + *

See {@link #lookup(List)} to look up multiple tags at + * once. It is significantly more efficient to look up multiple + * tags all at once rather than one at a time. + * + * @throws IOException if decoding the requested value fails + */ + public T lookup(CodedTupleTag tag) throws IOException; + + /** + * Returns a map from the given tags to the values associated with + * those tags in this {@code KeyedState}. A tag will map to null if + * the tag had no associated value. + * + *

See {@link #lookup(CodedTupleTag)} to look up a single + * tag. + * + * @throws CoderException if decoding any of the requested values fails + */ + public CodedTupleTagMap lookup(List> tags) throws IOException; + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Prepares this {@code DoFn} instance for processing a batch of elements. + * + *

By default, does nothing. + */ + public void startBundle(Context c) throws Exception { + } + + /** + * Processes an input element. + */ + public abstract void processElement(ProcessContext c) throws Exception; + + /** + * Finishes processing this batch of elements. This {@code DoFn} + * instance will be thrown away after this operation returns. + * + *

By default, does nothing. + */ + public void finishBundle(Context c) throws Exception { + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Returns a {@link TypeToken} capturing what is known statically + * about the input type of this {@code DoFn} instance's most-derived + * class. + * + *

See {@link #getOutputTypeToken} for more discussion. + */ + TypeToken getInputTypeToken() { + return new TypeToken(getClass()) {}; + } + + /** + * Returns a {@link TypeToken} capturing what is known statically + * about the output type of this {@code DoFn} instance's + * most-derived class. + * + *

In the normal case of a concrete {@code DoFn} subclass with + * no generic type parameters of its own (including anonymous inner + * classes), this will be a complete non-generic type, which is good + * for choosing a default output {@code Coder} for the output + * {@code PCollection}. + */ + TypeToken getOutputTypeToken() { + return new TypeToken(getClass()) {}; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java new file mode 100644 index 0000000000000..3e23b5ed04506 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java @@ -0,0 +1,357 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.DoFnRunner; +import com.google.cloud.dataflow.sdk.util.PTuple; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TupleTagList; +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * A harness for unit-testing a {@link DoFn}. + * + *

For example: + * + *

 {@code
+ * DoFn fn = ...;
+ *
+ * DoFnTester fnTester = DoFnTester.of(fn);
+ *
+ * // Set arguments shared across all batches:
+ * fnTester.setSideInputs(...);      // If fn takes side inputs.
+ * fnTester.setSideOutputTags(...);  // If fn writes to side outputs.
+ *
+ * // Process a batch containing a single input element:
+ * Input testInput = ...;
+ * List testOutputs = fnTester.processBatch(testInput);
+ * Assert.assertThat(testOutputs,
+ *                   JUnitMatchers.hasItems(...));
+ *
+ * // Process a bigger batch:
+ * Assert.assertThat(fnTester.processBatch(i1, i2, ...),
+ *                   JUnitMatchers.hasItems(...));
+ * } 
+ * + * @param the type of the {@code DoFn}'s (main) input elements + * @param the type of the {@code DoFn}'s (main) output elements + */ +public class DoFnTester { + /** + * Returns a {@code DoFnTester} supporting unit-testing of the given + * {@link DoFn}. + */ + @SuppressWarnings("unchecked") + public static DoFnTester of(DoFn fn) { + return new DoFnTester(fn); + } + + /** + * Registers the tuple of values of the side input {@link PCollectionView}s to + * pass to the {@link DoFn} under test. + * + *

If needed, first creates a fresh instance of the {@link DoFn} + * under test. + * + *

If this isn't called, {@code DoFnTester} assumes the + * {@link DoFn} takes no side inputs. + */ + public void setSideInputs(Map, Iterable>> sideInputs) { + this.sideInputs = sideInputs; + resetState(); + } + + /** + * Registers the values of a side input {@link PCollectionView} to + * pass to the {@link DoFn} under test. + * + *

If needed, first creates a fresh instance of the {@code DoFn} + * under test. + * + *

If this isn't called, {@code DoFnTester} assumes the + * {@code DoFn} takes no side inputs. + */ + public void setSideInput(PCollectionView sideInput, Iterable> value) { + sideInputs.put(sideInput, value); + } + + /** + * Registers the values for a side input {@link PCollectionView} to + * pass to the {@link DoFn} under test. All values are placed + * in the global window. + */ + public void setSideInputInGlobalWindow( + PCollectionView sideInput, + Iterable value) { + sideInputs.put( + sideInput, + Iterables.transform(value, new Function>() { + @Override + public WindowedValue apply(Object input) { + return WindowedValue.valueInGlobalWindow(input); + } + })); + } + + + /** + * Registers the list of {@code TupleTag}s that can be used by the + * {@code DoFn} under test to output to side output + * {@code PCollection}s. + * + *

If needed, first creates a fresh instance of the DoFn under test. + * + *

If this isn't called, {@code DoFnTester} assumes the + * {@code DoFn} doesn't emit to any side outputs. + */ + public void setSideOutputTags(TupleTagList sideOutputTags) { + this.sideOutputTags = sideOutputTags.getAll(); + resetState(); + } + + /** + * A convenience operation that first calls {@link #startBundle}, + * then calls {@link #processElement} on each of the arguments, then + * calls {@link #finishBundle}, then returns the result of + * {@link #takeOutputElements}. + */ + public List processBatch(I... inputElements) { + startBundle(); + for (I inputElement : inputElements) { + processElement(inputElement); + } + finishBundle(); + return takeOutputElements(); + } + + /** + * Calls {@link DoFn#startBundle} on the {@code DoFn} under test. + * + *

If needed, first creates a fresh instance of the DoFn under test. + */ + public void startBundle() { + resetState(); + initializeState(); + fnRunner.startBundle(); + state = State.STARTED; + } + + /** + * Calls {@link DoFn#processElement} on the {@code DoFn} under test, in a + * context where {@link DoFn.ProcessContext#element} returns the + * given element. + * + *

Will call {@link #startBundle} automatically, if it hasn't + * already been called. + * + * @throws IllegalStateException if the {@code DoFn} under test has already + * been finished + */ + public void processElement(I element) { + if (state == State.FINISHED) { + throw new IllegalStateException("finishBundle() has already been called"); + } + if (state == State.UNSTARTED) { + startBundle(); + } + fnRunner.processElement(WindowedValue.valueInGlobalWindow(element)); + } + + /** + * Calls {@link DoFn#finishBundle} of the {@code DoFn} under test. + * + *

Will call {@link #startBundle} automatically, if it hasn't + * already been called. + * + * @throws IllegalStateException if the {@code DoFn} under test has already + * been finished + */ + public void finishBundle() { + if (state == State.FINISHED) { + throw new IllegalStateException("finishBundle() has already been called"); + } + if (state == State.UNSTARTED) { + startBundle(); + } + fnRunner.finishBundle(); + state = State.FINISHED; + } + + /** + * Returns the elements output so far to the main output. Does not + * clear them, so subsequent calls will continue to include these + * elements. + * + * @see #takeOutputElements + * @see #clearOutputElements + * + * TODO: provide accessors that take and return {@code WindowedValue}s + * in order to test timestamp- and window-sensitive DoFns. + */ + public List peekOutputElements() { + // TODO: Should we return an unmodifiable list? + return Lists.transform(fnRunner.getReceiver(mainOutputTag), + new Function() { + @Override + public O apply(Object input) { + return ((WindowedValue) input).getValue(); + } + }); + + } + + /** + * Clears the record of the elements output so far to the main output. + * + * @see #peekOutputElements + */ + public void clearOutputElements() { + peekOutputElements().clear(); + } + + /** + * Returns the elements output so far to the main output. + * Clears the list so these elements don't appear in future calls. + * + * @see #peekOutputElements + */ + public List takeOutputElements() { + List resultElems = new ArrayList<>(peekOutputElements()); + clearOutputElements(); + return resultElems; + } + + /** + * Returns the elements output so far to the side output with the + * given tag. Does not clear them, so subsequent calls will + * continue to include these elements. + * + * @see #takeSideOutputElements + * @see #clearSideOutputElements + */ + public List peekSideOutputElements(TupleTag tag) { + // TODO: Should we return an unmodifiable list? + return Lists.transform(fnRunner.getReceiver(tag), + new Function() { + @Override + public T apply(Object input) { + return ((WindowedValue) input).getValue(); + }}); + } + + /** + * Clears the record of the elements output so far to the side + * output with the given tag. + * + * @see #peekSideOutputElements + */ + public void clearSideOutputElements(TupleTag tag) { + peekSideOutputElements(tag).clear(); + } + + /** + * Returns the elements output so far to the side output with the given tag. + * Clears the list so these elements don't appear in future calls. + * + * @see #peekSideOutputElements + */ + public List takeSideOutputElements(TupleTag tag) { + List resultElems = new ArrayList<>(peekSideOutputElements(tag)); + clearSideOutputElements(tag); + return resultElems; + } + + ///////////////////////////////////////////////////////////////////////////// + + /** The possible states of processing a DoFn. */ + enum State { UNSTARTED, STARTED, FINISHED } + + final PipelineOptions options = PipelineOptionsFactory.create(); + + /** The original DoFn under test. */ + final DoFn origFn; + + /** The side input values to provide to the DoFn under test. */ + private Map, Iterable>> sideInputs = + new HashMap<>(); + + /** The output tags used by the DoFn under test. */ + TupleTag mainOutputTag = new TupleTag<>(); + List> sideOutputTags = new ArrayList<>(); + + /** The original DoFn under test, if started. */ + DoFn fn; + + /** The DoFnRunner if processing is in progress. */ + DoFnRunner fnRunner; + + /** Counters for user-defined Aggregators if processing is in progress. */ + CounterSet counterSet; + // TODO: expose counterSet through a getter method, once we have + // a convenient public API for it. + + /** The state of processing of the DoFn under test. */ + State state; + + DoFnTester(DoFn origFn) { + this.origFn = origFn; + resetState(); + } + + void resetState() { + fn = null; + fnRunner = null; + counterSet = null; + state = State.UNSTARTED; + } + + @SuppressWarnings("unchecked") + void initializeState() { + fn = (DoFn) + SerializableUtils.deserializeFromByteArray( + SerializableUtils.serializeToByteArray(origFn), + origFn.toString()); + counterSet = new CounterSet(); + PTuple runnerSideInputs = PTuple.empty(); + for (Map.Entry, Iterable>> entry + : sideInputs.entrySet()) { + runnerSideInputs = runnerSideInputs.and(entry.getKey().getTagInternal(), entry.getValue()); + } + fnRunner = DoFnRunner.createWithListOutputs( + options, + fn, + runnerSideInputs, + mainOutputTag, + sideOutputTags, + (new BatchModeExecutionContext()).createStepContext("stepName"), + counterSet.getAddCounterMutator()); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/First.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/First.java new file mode 100644 index 0000000000000..9e4f3b099d48a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/First.java @@ -0,0 +1,106 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionView; + +/** + * {@code First} takes a {@code PCollection} and a limit, and + * produces a new {@code PCollection} containing up to limit + * elements of the input {@code PCollection}. + * + *

If the input and output {@code PCollection}s are ordered, then + * {@code First} will select the first elements, otherwise it will + * select any elements. + * + *

If limit is less than or equal to the size of the input + * {@code PCollection}, then all the input's elements will be selected. + * + *

All of the elements of the output {@code PCollection} should fit into + * main memory of a single worker machine. This operation does not + * run in parallel. + * + *

Example of use: + *

 {@code
+ * PCollection input = ...;
+ * PCollection output = input.apply(First.of(100));
+ * } 
+ * + * @param the type of the elements of the input and output + * {@code PCollection}s + */ +public class First extends PTransform, PCollection> { + /** + * Returns a {@code First} {@code PTransform}. + * + * @param the type of the elements of the input and output + * {@code PCollection}s + * @param limit the numer of elements to take from the input + */ + public static First of(long limit) { + return new First<>(limit); + } + + private final long limit; + + /** + * Constructs a {@code First} PTransform that, when applied, + * produces a new PCollection containing up to {@code limit} + * elements of its input {@code PCollection}. + */ + private First(long limit) { + this.limit = limit; + if (limit < 0) { + throw new IllegalArgumentException( + "limit argument to First should be non-negative"); + } + } + + private static class CopyFirstDoFn extends DoFn { + long limit; + final PCollectionView, ?> iterableView; + + public CopyFirstDoFn(long limit, PCollectionView, ?> iterableView) { + this.limit = limit; + this.iterableView = iterableView; + } + + @Override + public void processElement(ProcessContext c) { + for (T i : c.sideInput(iterableView)) { + if (limit-- <= 0) { + break; + } + c.output(i); + } + } + } + + @Override + public PCollection apply(PCollection in) { + PCollectionView, ?> iterableView = in.apply(View.asIterable()); + return + in.getPipeline() + .apply(Create.of((Void) null)).setCoder(VoidCoder.of()) + .apply(ParDo + .withSideInputs(iterableView) + .of(new CopyFirstDoFn<>(limit, iterableView))) + .setCoder(in.getCoder()); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java new file mode 100644 index 0000000000000..14b2169b97bfa --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java @@ -0,0 +1,206 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; + +import java.util.ArrayList; +import java.util.List; + +/** + * {@code Flatten} takes multiple {@code PCollection}s bundled + * into a {@code PCollectionList} and returns a single + * {@code PCollection} containing all the elements in all the input + * {@code PCollection}s. The name "Flatten" suggests taking a list of + * lists and flattening them into a single list. + * + *

Example of use: + *

 {@code
+ * PCollection pc1 = ...;
+ * PCollection pc2 = ...;
+ * PCollection pc3 = ...;
+ * PCollectionList pcs = PCollectionList.of(pc1).and(pc2).and(pc3);
+ * PCollection merged = pcs.apply(Flatten..create());
+ * } 
+ * + *

By default, the {@code Coder} of the output {@code PCollection} + * is the same as the {@code Coder} of the first {@code PCollection} + * in the input {@code PCollectionList} (if the + * {@code PCollectionList} is non-empty). + * + */ +public class Flatten { + + /** + * Returns a {@link PTransform} that flattens a {@link PCollectionList} + * into a {@link PCollection} containing all the elements of all + * the {@link PCollection}s in its input. + * + *

If any of the inputs to {@code Flatten} require window merging, + * all inputs must have equal {@link WindowingFn}s. + * The output elements of {@code Flatten} are in the same windows and + * have the same timestamps as their corresponding input elements. The output + * {@code PCollection} will have the same + * {@link WindowingFn} as all of the inputs. + * + * @param the type of the elements in the input and output + * {@code PCollection}s. + */ + public static FlattenPCollectionList pCollections() { + return new FlattenPCollectionList<>(); + } + + @Deprecated + public static FlattenPCollectionList create() { + return pCollections(); + } + + /** + * Returns a {@code PTransform} that takes a {@code PCollection>} + * and returns a {@code PCollection} containing all the elements from + * all the {@code Iterable}s. + * + *

Example of use: + *

 {@code
+   * PCollection> pcOfIterables = ...;
+   * PCollection pc = pcOfIterables.apply(Flatten.iterables());
+   * } 
+ * + *

By default, the output {@code PCollection} encodes its elements + * using the same {@code Coder} that the input uses for + * the elements in its {@code Iterable}. + * + * @param the type of the elements of the input {@code Iterable} and + * the output {@code PCollection} + */ + public static FlattenIterables iterables() { + return new FlattenIterables<>(); + } + + /** + * A {@link PTransform} that flattens a {@link PCollectionList} + * into a {@link PCollection} containing all the elements of all + * the {@link PCollection}s in its input. + * + * @param the type of the elements in the input and output + * {@code PCollection}s. + */ + public static class FlattenPCollectionList + extends PTransform, PCollection> { + + private FlattenPCollectionList() { } + + @Override + public PCollection apply(PCollectionList inputs) { + WindowingFn windowingFn; + if (!getInput().getAll().isEmpty()) { + windowingFn = getInput().get(0).getWindowingFn(); + for (PCollection input : getInput().getAll()) { + if (!windowingFn.isCompatible(input.getWindowingFn())) { + throw new IllegalStateException( + "Inputs to Flatten had incompatible window windowingFns: " + + windowingFn + ", " + input.getWindowingFn()); + } + } + } else { + windowingFn = new GlobalWindow(); + } + + return PCollection.createPrimitiveOutputInternal(windowingFn); + } + + @Override + protected Coder getDefaultOutputCoder() { + List> inputs = getInput().getAll(); + if (inputs.isEmpty()) { + // Cannot infer a Coder from an empty list of input PCollections. + return null; + } + // Use the Coder of the first input. + return inputs.get(0).getCoder(); + } + + } + + /** + * {@code FlattenIterables} takes a {@code PCollection>} and returns a + * {@code PCollection} that contains all the elements from each iterable. + * Implements {@link #fromIterable}. + * + * @param the type of the elements of the input {@code Iterable}s and + * the output {@code PCollection} + */ + public static class FlattenIterables + extends PTransform>, PCollection> { + + @Override + public PCollection apply(PCollection> in) { + Coder> inCoder = in.getCoder(); + if (!(inCoder instanceof IterableCoder)) { + throw new IllegalArgumentException( + "expecting the input Coder to be an IterableCoder"); + } + IterableCoder iterableCoder = (IterableCoder) inCoder; + Coder elemCoder = iterableCoder.getElemCoder(); + + return in.apply(ParDo.of( + new DoFn, T>() { + @Override + public void processElement(ProcessContext c) { + for (T i : c.element()) { + c.output(i); + } + } + })) + .setCoder(elemCoder); + } + } + + ///////////////////////////////////////////////////////////////////////////// + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + FlattenPCollectionList.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + FlattenPCollectionList transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateHelper(transform, context); + } + }); + } + + private static void evaluateHelper( + FlattenPCollectionList transform, + DirectPipelineRunner.EvaluationContext context) { + List> outputElems = new ArrayList<>(); + PCollectionList inputs = transform.getInput(); + + for (PCollection input : inputs.getAll()) { + outputElems.addAll(context.getPCollectionValuesWithMetadata(input)); + } + + context.setPCollectionValuesWithMetadata(transform.getOutput(), outputElems); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java new file mode 100644 index 0000000000000..d7a4de64e50d3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -0,0 +1,517 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.ValueWithMetadata; +import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.NonMergingWindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.joda.time.Instant; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * {@code GroupByKey} takes a {@code PCollection>}, + * groups the values by key and windows, and returns a + * {@code PCollection>>} representing a map from + * each distinct key and window of the input {@code PCollection} to an + * {@code Iterable} over all the values associated with that key in + * the input. Each key in the output {@code PCollection} is unique within + * each window. + * + *

{@code GroupByKey} is analogous to converting a multi-map into + * a uni-map, and related to {@code GROUP BY} in SQL. It corresponds + * to the "shuffle" step between the Mapper and the Reducer in the + * MapReduce framework. + * + *

Two keys of type {@code K} are compared for equality + * not by regular Java {@link Object#equals}, but instead by + * first encoding each of the keys using the {@code Coder} of the + * keys of the input {@code PCollection}, and then comparing the + * encoded bytes. This admits efficient parallel evaluation. Note that + * this requires that the {@code Coder} of the keys be deterministic (see + * {@link Coder#isDeterministic()}). If the key {@code Coder} is not + * deterministic, an exception is thrown at runtime. + * + *

By default, the {@code Coder} of the keys of the output + * {@code PCollection} is the same as that of the keys of the input, + * and the {@code Coder} of the elements of the {@code Iterable} + * values of the output {@code PCollection} is the same as the + * {@code Coder} of the values of the input. + * + *

Example of use: + *

 {@code
+ * PCollection> urlDocPairs = ...;
+ * PCollection>> urlToDocs =
+ *     urlDocPairs.apply(GroupByKey.create());
+ * PCollection results =
+ *     urlToDocs.apply(ParDo.of(new DoFn>, R>() {
+ *       public void processElement(ProcessContext c) {
+ *         String url = c.element().getKey();
+ *         Iterable docsWithThatUrl = c.element().getValue();
+ *         ... process all docs having that url ...
+ *       }}));
+ * } 
+ * + *

{@code GroupByKey} is a key primitive in data-parallel + * processing, since it is the main way to efficiently bring + * associated data together into one location. It is also a key + * determiner of the performance of a data-parallel pipeline. + * + *

See {@link com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey} + * for a way to group multiple input PCollections by a common key at once. + * + *

See {@link Combine.PerKey} for a common pattern of + * {@code GroupByKey} followed by {@link Combine.GroupedValues}. + * + *

When grouping, windows that can be merged according to the {@link WindowingFn} + * of the input {@code PCollection} will be merged together, and a group + * corresponding to the new, merged window will be emitted. + * The timestamp for each group is the upper bound of its window, e.g., the most + * recent timestamp that can be assigned into the window, and the group will be + * in the window that it corresponds to. The output {@code PCollection} will + * have the same {@link WindowingFn} as the input. + * + *

If the {@link WindowingFn} of the input requires merging, it is not + * valid to apply another {@code GroupByKey} without first applying a new + * {@link WindowingFn}. + * + * @param the type of the keys of the input and output + * {@code PCollection}s + * @param the type of the values of the input {@code PCollection} + * and the elements of the {@code Iterable}s in the output + * {@code PCollection} + */ +public class GroupByKey + extends PTransform>, + PCollection>>> { + /** + * Returns a {@code GroupByKey} {@code PTransform}. + * + * @param the type of the keys of the input and output + * {@code PCollection}s + * @param the type of the values of the input {@code PCollection} + * and the elements of the {@code Iterable}s in the output + * {@code PCollection} + */ + public static GroupByKey create() { + return new GroupByKey<>(); + } + + + ///////////////////////////////////////////////////////////////////////////// + + @Override + public PCollection>> apply(PCollection> input) { + return applyHelper(input, false, false); + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Helper transform that makes timestamps and window assignments + * explicit in the value part of each key/value pair. + */ + public static class ReifyTimestampsAndWindows + extends PTransform>, + PCollection>>> { + @Override + public PCollection>> apply( + PCollection> input) { + Coder> inputCoder = getInput().getCoder(); + KvCoder inputKvCoder = (KvCoder) inputCoder; + Coder keyCoder = inputKvCoder.getKeyCoder(); + Coder inputValueCoder = inputKvCoder.getValueCoder(); + Coder> outputValueCoder = FullWindowedValueCoder.of( + inputValueCoder, getInput().getWindowingFn().windowCoder()); + Coder>> outputKvCoder = + KvCoder.of(keyCoder, outputValueCoder); + return input.apply(ParDo.of( + new DoFn, KV>>() { + @Override + public void processElement(ProcessContext c) { + KV kv = c.element(); + K key = kv.getKey(); + V value = kv.getValue(); + c.output(KV.of( + key, + WindowedValue.of(value, c.timestamp(), c.windows()))); + }})) + .setCoder(outputKvCoder); + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Helper transform that sorts the values associated with each key + * by timestamp. + */ + public static class SortValuesByTimestamp + extends PTransform>>>, + PCollection>>>> { + @Override + public PCollection>>> apply( + PCollection>>> input) { + return input.apply(ParDo.of( + new DoFn>>, + KV>>>() { + @Override + public void processElement(ProcessContext c) { + KV>> kvs = c.element(); + K key = kvs.getKey(); + Iterable> unsortedValues = kvs.getValue(); + List> sortedValues = new ArrayList<>(); + for (WindowedValue value : unsortedValues) { + sortedValues.add(value); + } + Collections.sort(sortedValues, + new Comparator>() { + @Override + public int compare(WindowedValue e1, WindowedValue e2) { + return e1.getTimestamp().compareTo(e2.getTimestamp()); + } + }); + c.output(KV.>>of(key, sortedValues)); + }})) + .setCoder(getInput().getCoder()); + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Helper transform that takes a collection of timestamp-ordered + * values associated with each key, groups the values by window, + * combines windows as needed, and for each window in each key, + * outputs a collection of key/value-list pairs implicitly assigned + * to the window and with the timestamp derived from that window. + */ + public static class GroupAlsoByWindow + extends PTransform>>>, + PCollection>>> { + private final WindowingFn windowingFn; + + public GroupAlsoByWindow(WindowingFn windowingFn) { + this.windowingFn = windowingFn; + } + + @Override + public PCollection>> apply( + PCollection>>> input) { + Coder>>> inputCoder = + getInput().getCoder(); + KvCoder>> inputKvCoder = + (KvCoder>>) inputCoder; + Coder keyCoder = inputKvCoder.getKeyCoder(); + Coder>> inputValueCoder = + inputKvCoder.getValueCoder(); + IterableCoder> inputIterableValueCoder = + (IterableCoder>) inputValueCoder; + Coder> inputIterableElementCoder = + inputIterableValueCoder.getElemCoder(); + WindowedValueCoder inputIterableWindowedValueCoder = + (WindowedValueCoder) inputIterableElementCoder; + Coder inputIterableElementValueCoder = + inputIterableWindowedValueCoder.getValueCoder(); + Coder> outputValueCoder = + IterableCoder.of(inputIterableElementValueCoder); + Coder>> outputKvCoder = + KvCoder.of(keyCoder, outputValueCoder); + + return input.apply(ParDo.of( + new GroupAlsoByWindowsDoFn( + windowingFn, inputIterableElementValueCoder))) + .setCoder(outputKvCoder); + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Primitive helper transform that groups by key only, ignoring any + * window assignments. + */ + public static class GroupByKeyOnly + extends PTransform>, + PCollection>>> { + // TODO: Define and implement sorting by value. + boolean sortsValues = false; + + public GroupByKeyOnly() { } + + @Override + public PCollection>> apply(PCollection> input) { + WindowingFn windowingFn = getInput().getWindowingFn(); + if (!(windowingFn instanceof NonMergingWindowingFn)) { + // Prevent merging windows again, without explicit user + // involvement, e.g., by Window.into() or Window.remerge(). + windowingFn = new InvalidWindowingFn( + "WindowingFn has already been consumed by previous GroupByKey", + windowingFn); + } + return PCollection.>>createPrimitiveOutputInternal( + windowingFn); + } + + @Override + public void finishSpecifying() { + // Verify that the input Coder> is a KvCoder, and that + // the key coder is deterministic. + Coder keyCoder = getKeyCoder(); + if (!keyCoder.isDeterministic()) { + throw new IllegalStateException( + "the key Coder must be deterministic for grouping"); + } + if (getOutput().isOrdered()) { + throw new IllegalStateException( + "the result of a GroupByKey cannot be specified to be ordered"); + } + super.finishSpecifying(); + } + + /** + * Returns the {@code Coder} of the input to this transform, which + * should be a {@code KvCoder}. + */ + KvCoder getInputKvCoder() { + Coder> inputCoder = getInput().getCoder(); + if (!(inputCoder instanceof KvCoder)) { + throw new IllegalStateException( + "GroupByKey requires its input to use KvCoder"); + } + return (KvCoder) inputCoder; + } + + /** + * Returns the {@code Coder} of the keys of the input to this + * transform, which is also used as the {@code Coder} of the keys of + * the output of this transform. + */ + Coder getKeyCoder() { + return getInputKvCoder().getKeyCoder(); + } + + /** + * Returns the {@code Coder} of the values of the input to this transform. + */ + Coder getInputValueCoder() { + return getInputKvCoder().getValueCoder(); + } + + /** + * Returns the {@code Coder} of the {@code Iterable} values of the + * output of this transform. + */ + Coder> getOutputValueCoder() { + return IterableCoder.of(getInputValueCoder()); + } + + /** + * Returns the {@code Coder} of the output of this transform. + */ + KvCoder> getOutputKvCoder() { + return KvCoder.of(getKeyCoder(), getOutputValueCoder()); + } + + @Override + protected Coder>> getDefaultOutputCoder() { + return getOutputKvCoder(); + } + + /** + * Returns whether this GBK sorts values. + */ + boolean sortsValues() { + return sortsValues; + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + GroupByKeyOnly.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + GroupByKeyOnly transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateHelper(transform, context); + } + }); + } + + private static void evaluateHelper( + GroupByKeyOnly transform, + DirectPipelineRunner.EvaluationContext context) { + PCollection> input = transform.getInput(); + + List>> inputElems = + context.getPCollectionValuesWithMetadata(input); + + Coder keyCoder = transform.getKeyCoder(); + + Map, List> groupingMap = new HashMap<>(); + + for (ValueWithMetadata> elem : inputElems) { + K key = elem.getValue().getKey(); + V value = elem.getValue().getValue(); + Instant timestamp = elem.getTimestamp(); + byte[] encodedKey; + try { + encodedKey = encodeToByteArray(keyCoder, key); + } catch (CoderException exn) { + // TODO: Put in better element printing: + // truncate if too long. + throw new IllegalArgumentException( + "unable to encode key " + key + " of input to " + transform + + " using " + keyCoder, + exn); + } + GroupingKey groupingKey = new GroupingKey<>(key, encodedKey); + List values = groupingMap.get(groupingKey); + if (values == null) { + values = new ArrayList(); + groupingMap.put(groupingKey, values); + } + values.add(value); + } + + List>>> outputElems = + new ArrayList<>(); + for (Map.Entry, List> entry : groupingMap.entrySet()) { + GroupingKey groupingKey = entry.getKey(); + K key = groupingKey.getKey(); + List values = entry.getValue(); + values = context.randomizeIfUnordered( + transform.sortsValues(), values, true /* inPlaceAllowed */); + outputElems.add(ValueWithMetadata + .of(WindowedValue.valueInEmptyWindows(KV.>of(key, values))) + .withKey(key)); + } + + context.setPCollectionValuesWithMetadata(transform.getOutput(), + outputElems); + } + + public PCollection>> applyHelper( + PCollection> input, boolean isStreaming, boolean runnerSortsByTimestamp) { + Coder> inputCoder = getInput().getCoder(); + if (!(inputCoder instanceof KvCoder)) { + throw new IllegalStateException( + "GroupByKey requires its input to use KvCoder"); + } + // This operation groups by the combination of key and window, + // merging windows as needed, using the windows assigned to the + // key/value input elements and the window merge operation of the + // windowing function associated with the input PCollection. + WindowingFn windowingFn = getInput().getWindowingFn(); + if (windowingFn instanceof InvalidWindowingFn) { + String cause = ((InvalidWindowingFn) windowingFn).getCause(); + throw new IllegalStateException( + "GroupByKey must have a valid Window merge function. " + + "Invalid because: " + cause); + } + if (windowingFn.isCompatible(new GlobalWindow())) { + // The input PCollection is using the degenerate default + // windowing function, which uses a single global window for all + // elements. We can implement this using a more-primitive + // non-window-aware GBK transform. + return input.apply(new GroupByKeyOnly()); + + } else if (isStreaming) { + // If using the streaming runner, the service will do the insertion of + // the GroupAlsoByWindow step. + // TODO: Remove this case once the Dataflow Runner handles GBK directly + return input.apply(new GroupByKeyOnly()); + + } else { + // By default, implement GroupByKey[AndWindow] via a series of lower-level + // operations. + PCollection>>> gbkOutput = input + // Make each input element's timestamp and assigned windows + // explicit, in the value part. + .apply(new ReifyTimestampsAndWindows()) + + // Group by just the key. + .apply(new GroupByKeyOnly>()); + + if (!runnerSortsByTimestamp) { + // Sort each key's values by timestamp. GroupAlsoByWindow requires + // its input to be sorted by timestamp. + gbkOutput = gbkOutput.apply(new SortValuesByTimestamp()); + } + + return gbkOutput + // Group each key's values by window, merging windows as needed. + .apply(new GroupAlsoByWindow(windowingFn)); + } + } + + private static class GroupingKey { + private K key; + private byte[] encodedKey; + + public GroupingKey(K key, byte[] encodedKey) { + this.key = key; + this.encodedKey = encodedKey; + } + + public K getKey() { return key; } + + @Override + public boolean equals(Object o) { + if (o instanceof GroupingKey) { + GroupingKey that = (GroupingKey) o; + return Arrays.equals(this.encodedKey, that.encodedKey); + } else { + return false; + } + } + + @Override + public int hashCode() { return Arrays.hashCode(encodedKey); } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Keys.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Keys.java new file mode 100644 index 0000000000000..08a801b15ec2d --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Keys.java @@ -0,0 +1,68 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +/** + * {@code Keys} takes a {@code PCollection} of {@code KV}s and + * returns a {@code PCollection} of the keys. + * + *

Example of use: + *

 {@code
+ * PCollection> wordCounts = ...;
+ * PCollection words = wordCounts.apply(Keys.create());
+ * } 
+ * + *

Each output element has the same timestamp and is in the same windows + * as its corresponding input element, and the output {@code PCollection} + * has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * associated with it as the input. + * + *

See also {@link Values}. + * + * @param the type of the keys in the input {@code PCollection}, + * and the type of the elements in the output {@code PCollection} + */ +public class Keys extends PTransform>, + PCollection> { + /** + * Returns a {@code Keys} {@code PTransform}. + * + * @param the type of the keys in the input {@code PCollection}, + * and the type of the elements in the output {@code PCollection} + */ + public static Keys create() { + return new Keys<>(); + } + + private Keys() { } + + @Override + public PCollection apply(PCollection> in) { + return + in.apply(ParDo.named("Keys") + .of(new DoFn, K>() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().getKey()); + } + })); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/KvSwap.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/KvSwap.java new file mode 100644 index 0000000000000..ee73ae4087f5d --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/KvSwap.java @@ -0,0 +1,73 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +/** + * {@code KvSwap} takes a {@code PCollection>} and + * returns a {@code PCollection>}, where all the keys and + * values have been swapped. + * + *

Example of use: + *

 {@code
+ * PCollection wordsToCounts = ...;
+ * PCollection countsToWords =
+ *     wordToCounts.apply(KvSwap.create());
+ * } 
+ * + *

Each output element has the same timestamp and is in the same windows + * as its corresponding input element, and the output {@code PCollection} + * has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * associated with it as the input. + * + * @param the type of the keys in the input {@code PCollection} + * and the values in the output {@code PCollection} + * @param the type of the values in the input {@code PCollection} + * and the keys in the output {@code PCollection} + */ +public class KvSwap extends PTransform>, + PCollection>> { + /** + * Returns a {@code KvSwap} {@code PTransform}. + * + * @param the type of the keys in the input {@code PCollection} + * and the values in the output {@code PCollection} + * @param the type of the values in the input {@code PCollection} + * and the keys in the output {@code PCollection} + */ + public static KvSwap create() { + return new KvSwap<>(); + } + + private KvSwap() { } + + @Override + public PCollection> apply(PCollection> in) { + return + in.apply(ParDo.named("KvSwap") + .of(new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) { + KV e = c.element(); + c.output(KV.of(e.getValue(), e.getKey())); + } + })); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java new file mode 100644 index 0000000000000..fce9a328f1c84 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java @@ -0,0 +1,196 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +/** + * {@code PTransform}s for computing the maximum of the elements in a + * {@code PCollection}, or the maximum of the values associated with + * each key in a {@code PCollection} of {@code KV}s. + * + *

Example 1: get the maximum of a {@code PCollection} of {@code Double}s. + *

 {@code
+ * PCollection input = ...;
+ * PCollection max = input.apply(Max.doublesGlobally());
+ * } 
+ * + *

Example 2: calculate the maximum of the {@code Integer}s + * associated with each unique key (which is of type {@code String}). + *

 {@code
+ * PCollection> input = ...;
+ * PCollection> maxPerKey = input
+ *     .apply(Max.integersPerKey());
+ * } 
+ */ +public class Max { + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a + * {@code PCollection} whose contents is the maximum of the + * input {@code PCollection}'s elements, or + * {@code Integer.MIN_VALUE} if there are no elements. + */ + public static Combine.Globally integersGlobally() { + Combine.Globally combine = Combine + .globally(new MaxIntegerFn()); + combine.setName("Max"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to the maximum of the values associated with + * that key in the input {@code PCollection}. + * + *

See {@link Combine.PerKey} for how this affects timestamps and windowing. + */ + public static Combine.PerKey integersPerKey() { + Combine.PerKey combine = Combine + .perKey(new MaxIntegerFn()); + combine.setName("Max.PerKey"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a + * {@code PCollection} whose contents is the maximum of the + * input {@code PCollection}'s elements, or + * {@code Long.MIN_VALUE} if there are no elements. + */ + public static Combine.Globally longsGlobally() { + Combine.Globally combine = Combine.globally(new MaxLongFn()); + combine.setName("Max"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to the maximum of the values associated with + * that key in the input {@code PCollection}. + * + *

See {@link Combine.PerKey} for how this affects timestamps and windowing. + */ + public static Combine.PerKey longsPerKey() { + Combine.PerKey combine = Combine + .perKey(new MaxLongFn()); + combine.setName("Max.PerKey"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a + * {@code PCollection} whose contents is the maximum of the + * input {@code PCollection}'s elements, or + * {@code Double.MIN_VALUE} if there are no elements. + */ + public static Combine.Globally doublesGlobally() { + Combine.Globally combine = Combine + .globally(new MaxDoubleFn()); + combine.setName("Max"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to the maximum of the values associated with + * that key in the input {@code PCollection}. + * + *

See {@link Combine.PerKey} for how this affects timestamps and windowing. + */ + public static Combine.PerKey doublesPerKey() { + Combine.PerKey combine = Combine + .perKey(new MaxDoubleFn()); + combine.setName("Max.PerKey"); + return combine; + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * A {@code SerializableFunction} that computes the maximum of an + * {@code Iterable} of numbers of type {@code N}, useful as an + * argument to {@link Combine#globally} or {@link Combine#perKey}. + * + * @param the type of the {@code Number}s being compared + */ + public static class MaxFn> + implements SerializableFunction, N> { + + /** The smallest value of type N. */ + private final N initialValue; + + /** + * Constructs a combining function that computes the maximum over + * a collection of values of type {@code N}, given the smallest + * value of type {@code N}, which is the identity value for the + * maximum operation over {@code N}s. + */ + public MaxFn(N initialValue) { + this.initialValue = initialValue; + } + + @Override + public N apply(Iterable input) { + N max = initialValue; + for (N value : input) { + if (value.compareTo(max) > 0) { + max = value; + } + } + return max; + } + } + + /** + * A {@code SerializableFunction} that computes the maximum of an + * {@code Iterable} of {@code Integer}s, useful as an argument to + * {@link Combine#globally} or {@link Combine#perKey}. + */ + public static class MaxIntegerFn extends MaxFn { + public MaxIntegerFn() { super(Integer.MIN_VALUE); } + } + + /** + * A {@code SerializableFunction} that computes the maximum of an + * {@code Iterable} of {@code Long}s, useful as an argument to + * {@link Combine#globally} or {@link Combine#perKey}. + */ + public static class MaxLongFn extends MaxFn { + public MaxLongFn() { super(Long.MIN_VALUE); } + } + + /** + * A {@code SerializableFunction} that computes the maximum of an + * {@code Iterable} of {@code Double}s, useful as an argument to + * {@link Combine#globally} or {@link Combine#perKey}. + */ + public static class MaxDoubleFn extends MaxFn { + public MaxDoubleFn() { super(Double.MIN_VALUE); } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Mean.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Mean.java new file mode 100644 index 0000000000000..34fbb1fc29088 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Mean.java @@ -0,0 +1,143 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.coders.SerializableCoder; + +/** + * {@code PTransform}s for computing the arithmetic mean + * (a.k.a. average) of the elements in a {@code PCollection}, or the + * mean of the values associated with each key in a + * {@code PCollection} of {@code KV}s. + * + *

Example 1: get the mean of a {@code PCollection} of {@code Long}s. + *

 {@code
+ * PCollection input = ...;
+ * PCollection mean = input.apply(Mean.globally());
+ * } 
+ * + *

Example 2: calculate the mean of the {@code Integer}s + * associated with each unique key (which is of type {@code String}). + *

 {@code
+ * PCollection> input = ...;
+ * PCollection> meanPerKey =
+ *     input.apply(Mean.perKey());
+ * } 
+ */ +public class Mean { + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a + * {@code PCollection} whose contents is the mean of the + * input {@code PCollection}'s elements, or + * {@code 0} if there are no elements. + * + * @param the type of the {@code Number}s being combined + */ + public static Combine.Globally globally() { + Combine.Globally combine = Combine.globally(new MeanFn<>()); + combine.setName("Mean"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to the mean of the values associated with + * that key in the input {@code PCollection}. + * + * See {@link Combine.PerKey} for how this affects timestamps and bucketing. + * + * @param the type of the keys + * @param the type of the {@code Number}s being combined + */ + public static Combine.PerKey perKey() { + Combine.PerKey combine = Combine.perKey(new MeanFn<>()); + combine.setName("Mean.PerKey"); + return combine; + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * A {@code Combine.CombineFn} that computes the arithmetic mean + * (a.k.a. average) of an {@code Iterable} of numbers of type + * {@code N}, useful as an argument to {@link Combine#globally} or + * {@link Combine#perKey}. + * + *

Returns {@code 0} if combining zero elements. + * + * @param the type of the {@code Number}s being combined + */ + public static class MeanFn extends + Combine.AccumulatingCombineFn.CountSum, Double> { + + /** + * Constructs a combining function that computes the mean over + * a collection of values of type {@code N}. + */ + public MeanFn() {} + + /** + * Accumulator helper class for MeanFn. + */ + class CountSum + extends Combine.AccumulatingCombineFn.Accumulator { + + long count = 0; + double sum = 0.0; + + @Override + public void addInput(N element) { + count++; + sum += element.doubleValue(); + } + + @Override + public void mergeAccumulator(CountSum accumulator) { + count += accumulator.count; + sum += accumulator.sum; + } + + @Override + public Double extractOutput() { + return count == 0 ? 0.0 : sum / count; + } + } + + @Override + public CountSum createAccumulator() { + return new CountSum(); + } + + @SuppressWarnings("unchecked") + @Override + public Coder getAccumulatorCoder( + CoderRegistry registry, Coder inputCoder) { + // The casts are needed because CountSum.class is a + // Class, but we need a + // Class.CountSum>. + return SerializableCoder.of((Class) (Class) CountSum.class); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java new file mode 100644 index 0000000000000..337a051160973 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java @@ -0,0 +1,196 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +/** + * {@code PTransform}s for computing the minimum of the elements in a + * {@code PCollection}, or the minimum of the values associated with + * each key in a {@code PCollection} of {@code KV}s. + * + *

Example 1: get the minimum of a {@code PCollection} of {@code Double}s. + *

 {@code
+ * PCollection input = ...;
+ * PCollection min = input.apply(Min.doublesGlobally());
+ * } 
+ * + *

Example 2: calculate the minimum of the {@code Integer}s + * associated with each unique key (which is of type {@code String}). + *

 {@code
+ * PCollection> input = ...;
+ * PCollection> minPerKey = input
+ *     .apply(Min.integersPerKey());
+ * } 
+ */ +public class Min { + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a + * {@code PCollection} whose contents is a single value that is + * the minimum of the input {@code PCollection}'s elements, or + * {@code Integer.MAX_VALUE} if there are no elements. + */ + public static Combine.Globally integersGlobally() { + Combine.Globally combine = Combine + .globally(new MinIntegerFn()); + combine.setName("Min"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to the minimum of the values associated with + * that key in the input {@code PCollection}. + * + *

See {@link Combine.PerKey} for how this affects timestamps and windowing. + */ + public static Combine.PerKey integersPerKey() { + Combine.PerKey combine = Combine + .perKey(new MinIntegerFn()); + combine.setName("Min.PerKey"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a + * {@code PCollection} whose contents is the minimum of the + * input {@code PCollection}'s elements, or + * {@code Long.MAX_VALUE} if there are no elements. + */ + public static Combine.Globally longsGlobally() { + Combine.Globally combine = Combine.globally(new MinLongFn()); + combine.setName("Min"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to the minimum of the values associated with + * that key in the input {@code PCollection}. + * + *

See {@link Combine.PerKey} for how this affects timestamps and windowing. + */ + public static Combine.PerKey longsPerKey() { + Combine.PerKey combine = Combine + .perKey(new MinLongFn()); + combine.setName("Min.PerKey"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a + * {@code PCollection} whose contents is the minimum of the + * input {@code PCollection}'s elements, or + * {@code Double.MAX_VALUE} if there are no elements. + */ + public static Combine.Globally doublesGlobally() { + Combine.Globally combine = Combine + .globally(new MinDoubleFn()); + combine.setName("Min"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to the minimum of the values associated with + * that key in the input {@code PCollection}. + * + *

See {@link Combine.PerKey} for how this affects timestamps and windowing. + */ + public static Combine.PerKey doublesPerKey() { + Combine.PerKey combine = Combine + .perKey(new MinDoubleFn()); + combine.setName("Min.PerKey"); + return combine; + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * A {@code SerializableFunction} that computes the minimum of an + * {@code Iterable} of numbers of type {@code N}, useful as an + * argument to {@link Combine#globally} or {@link Combine#perKey}. + * + * @param the type of the {@code Number}s being compared + */ + public static class MinFn> + implements SerializableFunction, N> { + + /** The largest value of type N. */ + private final N initialValue; + + /** + * Constructs a combining function that computes the minimum over + * a collection of values of type {@code N}, given the largest + * value of type {@code N}, which is the identity value for the + * minimum operation over {@code N}s. + */ + public MinFn(N initialValue) { + this.initialValue = initialValue; + } + + @Override + public N apply(Iterable input) { + N min = initialValue; + for (N value : input) { + if (value.compareTo(min) < 0) { + min = value; + } + } + return min; + } + } + + /** + * A {@code SerializableFunction} that computes the minimum of an + * {@code Iterable} of {@code Integer}s, useful as an argument to + * {@link Combine#globally} or {@link Combine#perKey}. + */ + public static class MinIntegerFn extends MinFn { + public MinIntegerFn() { super(Integer.MAX_VALUE); } + } + + /** + * A {@code SerializableFunction} that computes the minimum of an + * {@code Iterable} of {@code Long}s, useful as an argument to + * {@link Combine#globally} or {@link Combine#perKey}. + */ + public static class MinLongFn extends MinFn { + public MinLongFn() { super(Long.MAX_VALUE); } + } + + /** + * A {@code SerializableFunction} that computes the minimum of an + * {@code Iterable} of {@code Double}s, useful as an argument to + * {@link Combine#globally} or {@link Combine#perKey}. + */ + public static class MinDoubleFn extends MinFn { + public MinDoubleFn() { super(Double.MAX_VALUE); } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java new file mode 100644 index 0000000000000..5906d7212dba0 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java @@ -0,0 +1,400 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.util.StringUtils; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.POutput; +import com.google.cloud.dataflow.sdk.values.TypedPValue; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; + +/** + * A {@code PTransform} is an operation that takes an + * {@code Input} (some subtype of {@link PInput}) and produces an + * {@code Output} (some subtype of {@link POutput}). + * + *

Common PTransforms include root PTransforms like + * {@link com.google.cloud.dataflow.sdk.io.TextIO.Read}, + * {@link Create}, processing and + * conversion operations like {@link ParDo}, + * {@link GroupByKey}, + * {@link com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey}, + * {@link Combine}, and {@link Count}, and outputting + * PTransforms like + * {@link com.google.cloud.dataflow.sdk.io.TextIO.Write}. Users also + * define their own application-specific composite PTransforms. + * + *

Each {@code PTransform} has a single + * {@code Input} type and a single {@code Output} type. Many + * PTransforms conceptually transform one input value to one output + * value, and in this case {@code Input} and {@code Output} are + * typically instances of + * {@link com.google.cloud.dataflow.sdk.values.PCollection}. + * A root + * PTransform conceptually has no input; in this case, conventionally + * a {@link com.google.cloud.dataflow.sdk.values.PBegin} object + * produced by calling {@link Pipeline#begin} is used as the input. + * An outputting PTransform conceptually has no output; in this case, + * conventionally {@link com.google.cloud.dataflow.sdk.values.PDone} + * is used as its output type. Some PTransforms conceptually have + * multiple inputs and/or outputs; in these cases special "bundling" + * classes like + * {@link com.google.cloud.dataflow.sdk.values.PCollectionList}, + * {@link com.google.cloud.dataflow.sdk.values.PCollectionTuple} + * are used + * to combine multiple values into a single bundle for passing into or + * returning from the PTransform. + * + *

A {@code PTransform} is invoked by calling + * {@code apply()} on its {@code Input}, returning its {@code Output}. + * Calls can be chained to concisely create linear pipeline segments. + * For example: + * + *

 {@code
+ * PCollection pc1 = ...;
+ * PCollection pc2 =
+ *     pc1.apply(ParDo.of(new MyDoFn>()))
+ *        .apply(GroupByKey.create())
+ *        .apply(Combine.perKey(new MyKeyedCombineFn()))
+ *        .apply(ParDo.of(new MyDoFn2,T2>()));
+ * } 
+ * + *

PTransform operations have unique names, which are used by the + * system when explaining what's going on during optimization and + * execution. Each PTransform gets a system-provided default name, + * but it's a good practice to specify an explicit name, where + * possible, using the {@code named()} method offered by some + * PTransforms such as {@link ParDo}. For example: + * + *

 {@code
+ * ...
+ * .apply(ParDo.named("Step1").of(new MyDoFn3()))
+ * ...
+ * } 
+ * + *

Each PCollection output produced by a PTransform, + * either directly or within a "bundling" class, automatically gets + * its own name derived from the name of its producing PTransform. An + * output's name can be changed by invoking + * {@link com.google.cloud.dataflow.sdk.values.PValue#setName}. + * + *

Each PCollection output produced by a PTransform + * also records a {@link com.google.cloud.dataflow.sdk.coders.Coder} + * that specifies how the elements of that PCollection + * are to be encoded as a byte string, if necessary. The + * PTransform may provide a default Coder for any of its outputs, for + * instance by deriving it from the PTransform input's Coder. If the + * PTransform does not specify the Coder for an output PCollection, + * the system will attempt to infer a Coder for it, based on + * what's known at run-time about the Java type of the output's + * elements. The enclosing {@link Pipeline}'s + * {@link com.google.cloud.dataflow.sdk.coders.CoderRegistry} + * (accessible via {@link Pipeline#getCoderRegistry}) defines the + * mapping from Java types to the default Coder to use, for a standard + * set of Java types; users can extend this mapping for additional + * types, via + * {@link com.google.cloud.dataflow.sdk.coders.CoderRegistry#registerCoder}. + * If this inference process fails, either because the Java type was + * not known at run-time (e.g., due to Java's "erasure" of generic + * types) or there was no default Coder registered, then the Coder + * should be specified manually by calling + * {@link com.google.cloud.dataflow.sdk.values.TypedPValue#setCoder} + * on the output PCollection. The Coder of every output + * PCollection must be determined one way or another + * before that output is used as an input to another PTransform, or + * before the enclosing Pipeline is run. + * + *

A small number of PTransforms are implemented natively by the + * Google Cloud Dataflow SDK; such PTransforms simply return an + * output value as their apply implementation. + * The majority of PTransforms are + * implemented as composites of other PTransforms. Such a PTransform + * subclass typically just implements {@link #apply}, computing its + * Output value from its Input value. User programs are encouraged to + * use this mechanism to modularize their own code. Such composite + * abstractions get their own name, and navigating through the + * composition hierarchy of PTransforms is supported by the monitoring + * interface. Examples of composite PTransforms can be found in this + * directory and in examples. From the caller's point of view, there + * is no distinction between a PTransform implemented natively and one + * implemented in terms of other PTransforms; both kinds of PTransform + * are invoked in the same way, using {@code apply()}. + * + *

Note on Serialization

+ * + * {@code PTransform} doesn't actually support serialization, despite + * implementing {@code Serializable}. + * + *

{@code PTransform} is marked {@code Serializable} solely + * because it is common for an anonymous {@code DoFn}, + * instance to be created within an + * {@code apply()} method of a composite {@code PTransform}. + * + *

Each of those {@code *Fn}s is {@code Serializable}, but + * unfortunately its instance state will contain a reference to the + * enclosing {@code PTransform} instance, and so attempt to serialize + * the {@code PTransform} instance, even though the {@code *Fn} + * instance never references anything about the enclosing + * {@code PTransform}. + * + *

Composite transforms, which are defined in terms of other transforms, + * should return the output of one of the composed transforms. Non-composite + * transforms, which do not apply any transforms internally, should return + * a new unbound output and register evaluators (via backend-specific + * registration methods). + * + *

The default implementation throws an exception. A derived class must + * either implement apply, or else each runner must supply a custom + * implementation via + * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner#apply}. + */ + public Output apply(Input input) { + throw new IllegalArgumentException( + "Runner " + getPipeline().getRunner() + + " has not registered an implementation for the required primitive operation " + + this); + } + + /** + * Sets the base name of this {@code PTransform}. + */ + public void setName(String name) { + this.name = name; + } + + /** + * Sets the base name of this {@code PTransform} and returns itself. + * + *

This is a shortcut for calling {@link #setName}, which allows method + * chaining. + */ + public PTransform withName(String name) { + setName(name); + return this; + } + + /** + * Returns the transform name. + * + *

This name is provided by the transform creator and is not required to be unique. + */ + public String getName() { + return name != null ? name : getDefaultName(); + } + + /** + * Returns the owning {@link Pipeline} of this {@code PTransform}. + * + * @throws IllegalStateException if the owning {@code Pipeline} hasn't been + * set yet + */ + @Deprecated + public Pipeline getPipeline() { + if (pipeline == null) { + throw new IllegalStateException("owning pipeline not set"); + } + return pipeline; + } + + /** + * Returns the input of this transform. + * + * @throws IllegalStateException if this PTransform hasn't been applied yet + * @deprecated Use pipeline.getInput(transform) + */ + @Deprecated + public Input getInput() { + @SuppressWarnings("unchecked") + Input input = (Input) getPipeline().getInput(this); + return input; + } + + /** + * Returns the output of this transform. + * + * @throws IllegalStateException if this PTransform hasn't been applied yet + * #deprecated use pipeline.getOutput(transform) + */ + @Deprecated + public Output getOutput() { + @SuppressWarnings("unchecked") + Output output = (Output) getPipeline().getOutput(this); + return output; + } + + /** + * Returns the {@link CoderRegistry}, useful for inferring + * {@link com.google.cloud.dataflow.sdk.coders.Coder}s. + * + * @throws IllegalStateException if the owning {@link Pipeline} hasn't been + * set yet + * @deprecated use pipeline.getCoderRegistry() + */ + @Deprecated + protected CoderRegistry getCoderRegistry() { + return getPipeline().getCoderRegistry(); + } + + + ///////////////////////////////////////////////////////////////////////////// + + // See the note about about PTransform's fake Serializability, to + // understand why all of its instance state is transient. + + /** + * The base name of this {@code PTransform}, e.g., from + * {@link ParDo#named(String)}, or from defaults, or {@code null} if not + * yet assigned. + */ + protected transient String name; + + /** + * The {@link Pipeline} that owns this {@code PTransform}, or {@code null} + * if not yet set. + */ + private transient Pipeline pipeline; + + protected PTransform() { + this.name = null; + } + + protected PTransform(String name) { + this.name = name; + } + + /** + * Associates this {@code PTransform} with the given {@code Pipeline}. + * + *

For internal use only. + * + * @throws IllegalArgumentException if this transform has already + * been associated with a pipeline + */ + @Deprecated + public void setPipeline(Pipeline pipeline) { + if (this.pipeline != null) { + throw new IllegalStateException( + "internal error: transform already initialized"); + } + this.pipeline = pipeline; + } + + @Override + public String toString() { + return getName() + " [" + getKindString() + "]"; + } + + /** + * Returns the name to use by default for this {@code PTransform} + * (not including the names of any enclosing {@code PTransform}s). + * + *

By default, returns {@link #getKindString}. + * + *

The caller is responsible for ensuring that names of applied + * {@code PTransform}s are unique, e.g., by adding a uniquifying + * suffix when needed. + */ + protected String getDefaultName() { + return getKindString(); + } + + /** + * Returns a string describing what kind of {@code PTransform} this is. + * + *

By default, returns the base name of this + * {@code PTransform}'s class. + */ + protected String getKindString() { + return StringUtils.approximateSimpleName(getClass()); + } + + private void writeObject(ObjectOutputStream oos) throws IOException { + // We don't really want to be serializing this object, but we + // often have serializable anonymous DoFns nested within a + // PTransform. + } + + private void readObject(ObjectInputStream oos) + throws IOException, ClassNotFoundException { + // We don't really want to be serializing this object, but we + // often have serializable anonymous DoFns nested within a + // PTransform. + } + + /** + * After building, finalizes this {@code PTransform} to + * make it ready for running. Called automatically when its + * output(s) are finished. + * + *

Not normally called by user code. + */ + public void finishSpecifying() { + getOutput().finishSpecifyingOutput(); + } + + /** + * Returns the default {@code Coder} to use for the output of this + * single-output {@code PTransform}, or {@code null} if + * none can be inferred. + * + *

By default, returns {@code null}. + */ + protected Coder getDefaultOutputCoder() { + return null; + } + + /** + * Returns the default {@code Coder} to use for the given output of + * this single-output {@code PTransform}, or {@code null} + * if none can be inferred. + */ + public Coder getDefaultOutputCoder(TypedPValue output) { + if (output != getOutput()) { + return null; + } else { + @SuppressWarnings("unchecked") + Coder defaultOutputCoder = (Coder) getDefaultOutputCoder(); + return defaultOutputCoder; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java new file mode 100644 index 0000000000000..c7d925b2b418b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java @@ -0,0 +1,1054 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.util.DirectModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.DoFnRunner; +import com.google.cloud.dataflow.sdk.util.PTuple; +import com.google.cloud.dataflow.sdk.util.StringUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionTuple; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TupleTagList; +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * {@code ParDo} is the core element-wise transform in Google Cloud + * Dataflow, invoking a user-specified function (from {@code I} to + * {@code O}) on each of the elements of the input + * {@code PCollection} to produce zero or more output elements, all + * of which are collected into the output {@code PCollection}. + * + *

Elements are processed independently, and possibly in parallel across + * distributed cloud resources. + * + *

The {@code ParDo} processing style is similar to what happens inside + * the "Mapper" or "Reducer" class of a MapReduce-style algorithm. + * + *

{@code DoFn}s

+ * + *

The function to use to process each element is specified by a + * {@link DoFn DoFn}. + * + *

Conceptually, when a {@code ParDo} transform is executed, the + * elements of the input {@code PCollection} are first divided up + * into some number of "batches". These are farmed off to distributed + * worker machines (or run locally, if using the + * {@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner}). + * For each batch of input elements, a fresh instance of the argument + * {@code DoFn} is created on a worker, then the {@code DoFn}'s + * optional {@link DoFn#startBundle} method is called to initialize it, + * then the {@code DoFn}'s required {@link DoFn#processElement} method + * is called on each of the input elements in the batch, then the + * {@code DoFn}'s optional {@link DoFn#finishBundle} method is called + * to complete its work, and finally the {@code DoFn} instance is + * thrown away. Each of the calls to any of the {@code DoFn}'s + * methods can produce zero or more output elements, which are + * collected together into a batch of output elements. All of the + * batches of output elements from all of the {@code DoFn} instances + * are "flattened" together into the output {@code PCollection}. + * + *

For example: + * + *

 {@code
+ * PCollection lines = ...;
+ * PCollection words =
+ *     lines.apply(ParDo.of(new DoFn() {
+ *         public void processElement(ProcessContext c) {
+ *           String line = c.element();
+ *           for (String word : line.split("[^a-zA-Z']+")) {
+ *             c.output(word);
+ *           }
+ *         }}));
+ * PCollection wordLengths =
+ *     words.apply(ParDo.of(new DoFn() {
+ *         public void processElement(ProcessContext c) {
+ *           String word = c.element();
+ *           Integer length = word.length();
+ *           c.output(length);
+ *         }}));
+ * } 
+ * + *

Each output element has the same timestamp and is in the same windows + * as its corresponding input element, and the output {@code PCollection} + * has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * associated with it as the input. + * + *

Naming {@code ParDo}s

+ * + *

A {@code ParDo} transform can be given a name using + * {@link #named}. While the system will automatically provide a name + * if none is specified explicitly, it is still a good practice to + * provide an explicit name, since that will probably make monitoring + * output more readable. For example: + * + *

 {@code
+ * PCollection words =
+ *     lines.apply(ParDo.named("ExtractWords")
+ *                      .of(new DoFn() { ... }));
+ * PCollection wordLengths =
+ *     words.apply(ParDo.named("ComputeWordLengths")
+ *                      .of(new DoFn() { ... }));
+ * } 
+ * + *

Side Inputs

+ * + *

While a {@code ParDo} iterates over a single "main input" + * {@code PCollection}, it can take additional "side input" + * {@code PCollectionView}s. These side input + * {@code PCollectionView}s express styles of accessing + * {@code PCollection}s computed by earlier pipeline operations, + * passed in to the {@code ParDo} transform using + * {@link #withSideInputs}, and their contents accessible to each of + * the {@code DoFn} operations via {@link DoFn.Context#sideInput}. + * For example: + * + *

 {@code
+ * PCollection words = ...;
+ * PCollection maxWordLengthCutOff = ...; // Singleton PCollection
+ * final PCollectionView maxWordLengthCutOffView =
+ *     SingletonPCollectionView.of(maxWordLengthCutOff);
+ * PCollection wordsBelowCutOff =
+ *     words.apply(ParDo.withSideInput(maxWordLengthCutOffView)
+ *                      .of(new DoFn() {
+ *         public void processElement(ProcessContext c) {
+ *           String word = c.element();
+ *           int lengthCutOff = c.sideInput(maxWordLengthCutOffView);
+ *           if (word.length() <= lengthCutOff) {
+ *             c.output(word);
+ *           }
+ *         }}));
+ * } 
+ * + *

Side Outputs

+ * + *

Optionally, a {@code ParDo} transform can produce multiple + * output {@code PCollection}s, both a "main output" + * {@code PCollection} plus any number of "side output" + * {@code PCollection}s, each keyed by a distinct {@link TupleTag}, + * and bundled in a {@link PCollectionTuple}. The {@code TupleTag}s + * to be used for the output {@code PCollectionTuple} is specified by + * invoking {@link #withOutputTags}. Unconsumed side outputs does not + * necessarily need to be explicity specified, even if the {@code DoFn} + * generates them. Within the {@code DoFn}, an element is added to the + * main output {@code PCollection} as normal, using + * {@link DoFn.Context#output}, while an element is added to a side output + * {@code PCollection} using {@link DoFn.Context#sideOutput}. For example: + * + *

 {@code
+ * PCollection words = ...;
+ * // Select words whose length is below a cut off,
+ * // plus the lengths of words that are above the cut off.
+ * // Also select words starting with "MARKER".
+ * final int wordLengthCutOff = 10;
+ * // Create tags to use for the main and side outputs.
+ * final TupleTag wordsBelowCutOffTag =
+ *     new TupleTag(){};
+ * final TupleTag wordLengthsAboveCutOffTag =
+ *     new TupleTag(){};
+ * final TupleTag markedWordsTag =
+ *     new TupleTag(){};
+ * PCollectionTuple results =
+ *     words.apply(
+ *         ParDo
+ *         // Specify the main and consumed side output tags of the
+ *         // PCollectionTuple result:
+ *         .withOutputTags(wordsBelowCutOffTag,
+ *                         TupleTagList.of(wordLengthsAboveCutOffTag)
+ *                                     .and(markedWordsTag))
+ *         .of(new DoFn() {
+ *             // Create a tag for the unconsumed side output.
+ *             final TupleTag specialWordsTag =
+ *                 new TupleTag(){};
+ *             public void processElement(ProcessContext c) {
+ *               String word = c.element();
+ *               if (word.length() <= wordLengthCutOff) {
+ *                 // Emit this short word to the main output.
+ *                 c.output(word);
+ *               } else {
+ *                 // Emit this long word's length to a side output.
+ *                 c.sideOutput(wordLengthsAboveCutOffTag, word.length());
+ *               }
+ *               if (word.startsWith("MARKER")) {
+ *                 // Emit this word to a different side output.
+ *                 c.sideOutput(markedWordsTag, word);
+ *               }
+ *               if (word.startsWith("SPECIAL")) {
+ *                 // Emit this word to the unconsumed side output.
+ *                 c.sideOutput(specialWordsTag, word);
+ *               }
+ *             }}));
+ * // Extract the PCollection results, by tag.
+ * PCollection wordsBelowCutOff =
+ *     results.get(wordsBelowCutOffTag);
+ * PCollection wordLengthsAboveCutOff =
+ *     results.get(wordLengthsAboveCutOffTag);
+ * PCollection markedWords =
+ *     results.get(markedWordsTag);
+ * } 
+ * + *

Properties May Be Specified In Any Order

+ * + * Several properties can be specified for a {@code ParDo} + * {@code PTransform}, including name, side inputs, side output tags, + * and {@code DoFn} to invoke. Only the {@code DoFn} is required; the + * name is encouraged but not required, and side inputs and side + * output tags are only specified when they're needed. These + * properties can be specified in any order, as long as they're + * specified before the {@code ParDo} {@code PTransform} is applied. + * + *

The approach used to allow these properties to be specified in + * any order, with some properties omitted, is to have each of the + * property "setter" methods defined as static factory methods on + * {@code ParDo} itself, which return an instance of either + * {@link ParDo.Unbound ParDo.Unbound} or + * {@link ParDo.Bound ParDo.Bound} nested classes, each of which offer + * property setter instance methods to enable setting additional + * properties. {@code ParDo.Bound} is used for {@code ParDo} + * transforms whose {@code DoFn} is specified and whose input and + * output static types have been bound. {@code ParDo.Unbound} is used + * for {@code ParDo} transforms that have not yet had their + * {@code DoFn} specified. Only {@code ParDo.Bound} instances can be + * applied. + * + *

Another benefit of this approach is that it reduces the number + * of type parameters that need to be specified manually. In + * particular, the input and output types of the {@code ParDo} + * {@code PTransform} are inferred automatically from the type + * parameters of the {@code DoFn} argument passed to {@link ParDo#of}. + * + *

Output Coders

+ * + *

By default, the {@code Coder} of the + * elements of the main output {@code PCollection} is inferred from the + * concrete type of the {@code DoFn}'s output type {@code O}. + * + *

By default, the {@code Coder} of the elements of a side output + * {@code PCollection} is inferred from the concrete type of the + * corresponding {@code TupleTag}'s type {@code X}. To be + * successful, the {@code TupleTag} should be created as an instance + * of a trivial anonymous subclass, with {@code {}} suffixed to the + * constructor call. Such uses block Java's generic type parameter + * inference, so the {@code } argument must be provided explicitly. + * For example: + *

 {@code
+ * // A TupleTag to use for a side input can be written concisely:
+ * final TupleTag sideInputTag = new TupleTag<>();
+ * // A TupleTag to use for a side output should be written with "{}",
+ * // and explicit generic parameter type:
+ * final TupleTag sideOutputTag = new TupleTag(){};
+ * } 
+ * This style of {@code TupleTag} instantiation is used in the example of + * multiple side outputs, above. + * + *

Ordered Input and/or Output PCollections

+ * + *

If the input {@code PCollection} is ordered (see + * {@link PCollection#setOrdered}), then each batch of the input + * processed by a {@code DoFn} instance will correspond to a + * consecutive subsequence of elements of the input, and the + * {@link DoFn#processElement} operation will be invoked on each + * element of the batch in order; otherwise, batches will correspond + * to arbitrary subsets of elements of the input, processed in + * arbitrary order. + * + *

Independently, if a main or side output {@code PCollection} is + * ordered, then the order in which elements are output to it will be + * preserved in the output {@code PCollection}; otherwise, the order + * in which elements are output to the {@code PCollection} doesn't + * matter. If the input {@code PCollection} is also ordered, then the + * sequences of elements output from the batches will be concatenated + * together in the same order as the batches appear in the input, + * supporting order-preserving transforms on {@code PCollection}s. + * + *

Serializability of {@code DoFn}s

+ * + *

A {@code DoFn} passed to a {@code ParDo} transform must be + * {@code Serializable}. This allows the {@code DoFn} instance + * created in this "main program" to be sent (in serialized form) to + * remote worker machines and reconstituted for each batch of elements + * of the input {@code PCollection} being processed. A {@code DoFn} + * can have instance variable state, and non-transient instance + * variable state will be serialized in the main program and then + * deserialized on remote worker machines for each batch of elements + * to process. + * + *

To aid in ensuring that {@code DoFn}s are properly + * {@code Serializable}, even local execution using the + * {@link DirectPipelineRunner} will serialize and then deserialize + * {@code DoFn}s before executing them on a batch. + * + *

{@code DoFn}s expressed as anonymous inner classes can be + * convenient, but due to a quirk in Java's rules for serializability, + * non-static inner or nested classes (including anonymous inner + * classes) automatically capture their enclosing class's instance in + * their serialized state. This can lead to including much more than + * intended in the serialized state of a {@code DoFn}, or even things + * that aren't {@code Serializable}. + * + *

There are two ways to avoid unintended serialized state in a + * {@code DoFn}: + * + *

    + * + *
  • Define the {@code DoFn} as a named, static class. + * + *
  • Define the {@code DoFn} as an anonymous inner class inside of + * a static method. + * + *
+ * + * Both these approaches ensure that there is no implicit enclosing + * class instance serialized along with the {@code DoFn} instance. + * + *

Prior to Java 8, any local variables of the enclosing + * method referenced from within an anonymous inner class need to be + * marked as {@code final}. If defining the {@code DoFn} as a named + * static class, such variables would be passed as explicit + * constructor arguments and stored in explicit instance variables. + * + *

There are three main ways to initialize the state of a + * {@code DoFn} instance processing a batch: + * + *

    + * + *
  • Define instance variable state (including implicit instance + * variables holding final variables captured by an anonymous inner + * class), initialized by the {@code DoFn}'s constructor (which is + * implicit for an anonymous inner class). This state will be + * automatically serialized and then deserialized in the {@code DoFn} + * instance created for each batch. This method is good for state + * known when the original {@code DoFn} is created in the main + * program, if it's not overly large. + * + *
  • Compute the state as a singleton {@code PCollection} and pass it + * in as a side input to the {@code DoFn}. This is good if the state + * needs to be computed by the pipeline, or if the state is very large + * and so is best read from file(s) rather than sent as part of the + * {@code DoFn}'s serialized state. + * + *
  • Initialize the state in each {@code DoFn} instance, in + * {@link DoFn#startBundle}. This is good if the initialization + * doesn't depend on any information known only by the main program or + * computed by earlier pipeline operations, but is the same for all + * instances of this {@code DoFn} for all program executions, say + * setting up empty caches or initializing constant data. + * + *
+ * + *

No Global Shared State

+ * + *

{@code ParDo} operations are intended to be able to run in + * parallel across multiple worker machines. This precludes easy + * sharing and updating mutable state across those machines. There is + * no support in the Google Cloud Dataflow system for communicating + * and synchronizing updates to shared state across worker machines, + * so programs should not access any mutable static variable state in + * their {@code DoFn}, without understanding that the Java processes + * for the main program and workers will each have its own independent + * copy of such state, and there won't be any automatic copying of + * that state across Java processes. All information should be + * communicated to {@code DoFn} instances via main and side inputs and + * serialized state, and all output should be communicated from a + * {@code DoFn} instance via main and side outputs, in the absence of + * external communication mechanisms written by user code. + * + *

Fault Tolerance

+ * + *

In a distributed system, things can fail: machines can crash, + * machines can be unable to communicate across the network, etc. + * While individual failures are rare, the larger the job, the greater + * the chance that something, somewhere, will fail. The Google Cloud + * Dataflow service strives to mask such failures automatically, + * principally by retrying failed {@code DoFn} batches. This means + * that a {@code DoFn} instance might process a batch partially, then + * crash for some reason, then be rerun (often on a different worker + * machine) on that same batch and on the same elements as before. + * Sometimes two or more {@code DoFn} instances will be running on the + * same batch simultaneously, with the system taking the results of + * the first instance to complete successfully. Consequently, the + * code in a {@code DoFn} needs to be written such that these + * duplicate (sequential or concurrent) executions do not cause + * problems. If the outputs of a {@code DoFn} are a pure function of + * its inputs, then this requirement is satisfied. However, if a + * {@code DoFn}'s execution has external side-effects, say performing + * updates to external HTTP services, then the {@code DoFn}'s code + * needs to take care to ensure that those updates are idempotent and + * that concurrent updates are acceptable. This property can be + * difficult to achieve, so it is advisable to strive to keep + * {@code DoFn}s as pure functions as much as possible. + * + *

Optimization

+ * + *

The Google Cloud Dataflow service automatically optimizes a + * pipeline before it is executed. A key optimization, fusion, + * relates to ParDo operations. If one ParDo operation produces a + * PCollection that is then consumed as the main input of another + * ParDo operation, the two ParDo operations will be fused + * together into a single ParDo operation and run in a single pass; + * this is "producer-consumer fusion". Similarly, if + * two or more ParDo operations have the same PCollection main input, + * they will be fused into a single ParDo which makes just one pass + * over the input PCollection; this is "sibling fusion". + * + *

If after fusion there are no more unfused references to a + * PCollection (e.g., one between a producer ParDo and a consumer + * ParDo), the PCollection itself is "fused away" and won't ever be + * written to disk, saving all the I/O and space expense of + * constructing it. + * + *

The Google Cloud Dataflow service applies fusion as much as + * possible, greatly reducing the cost of executing pipelines. As a + * result, it is essentially "free" to write ParDo operations in a + * vary modular, composable style, each ParDo operation doing one + * clear task, and stringing together sequences of ParDo operations to + * get the desired overall effect. Such programs can be easier to + * understand, easier to unit-test, easier to extend and evolve, and + * easier to reuse in new programs. The predefined library of + * PTransforms that come with Google Cloud Dataflow makes heavy use of + * this modular, composable style, trusting to the Google Cloud + * Dataflow service's optimizer to "flatten out" all the compositions + * into highly optimized stages. + * + * @see Using ParDo + */ +public class ParDo { + + /** + * Creates a {@code ParDo} {@code PTransform} with the given name. + * + *

See the discussion of Naming above for more explanation. + * + *

The resulting {@code PTransform} is incomplete, and its + * input/output types are not yet bound. Use + * {@link ParDo.Unbound#of} to specify the {@link DoFn} to + * invoke, which will also bind the input/output types of this + * {@code PTransform}. + */ + public static Unbound named(String name) { + return new Unbound().named(name); + } + + /** + * Creates a {@code ParDo} {@code PTransform} with the given + * side inputs. + * + *

Side inputs are {@link PCollectionView}s, whose contents are + * computed during pipeline execution and then made accessible to + * {@code DoFn} code via {@link DoFn.Context#sideInput}. Each + * invocation of the {@code DoFn} receives the same values for these + * side inputs. + * + *

See the discussion of Side Inputs above for more explanation. + * + *

The resulting {@code PTransform} is incomplete, and its + * input/output types are not yet bound. Use + * {@link ParDo.Unbound#of} to specify the {@link DoFn} to + * invoke, which will also bind the input/output types of this + * {@code PTransform}. + */ + public static Unbound withSideInputs(PCollectionView... sideInputs) { + return new Unbound().withSideInputs(sideInputs); + } + + /** + * Creates a {@code ParDo} with the given side inputs. + * + *

Side inputs are {@link PCollectionView}s, whose contents are + * computed during pipeline execution and then made accessible to + * {@code DoFn} code via {@link DoFn.Context#sideInput}. + * + *

See the discussion of Side Inputs above for more explanation. + * + *

The resulting {@code PTransform} is incomplete, and its + * input/output types are not yet bound. Use + * {@link ParDo.Unbound#of} to specify the {@link DoFn} to + * invoke, which will also bind the input/output types of this + * {@code PTransform}. + */ + public static Unbound withSideInputs( + Iterable> sideInputs) { + return new Unbound().withSideInputs(sideInputs); + } + + /** + * Creates a multi-output {@code ParDo} {@code PTransform} whose + * output {@link PCollection}s will be referenced using the given main + * output and side output tags. + * + *

{@link TupleTag}s are used to name (with its static element + * type {@code T}) each main and side output {@code PCollection}. + * This {@code PTransform}'s {@link DoFn} emits elements to the main + * output {@code PCollection} as normal, using + * {@link DoFn.Context#output}. The {@code DoFn} emits elements to + * a side output {@code PCollection} using + * {@link DoFn.Context#sideOutput}, passing that side output's tag + * as an argument. The result of invoking this {@code PTransform} + * will be a {@link PCollectionTuple}, and any of the the main and + * side output {@code PCollection}s can be retrieved from it via + * {@link PCollectionTuple#get}, passing the output's tag as an + * argument. + * + *

See the discussion of Side Outputs above for more explanation. + * + *

The resulting {@code PTransform} is incomplete, and its input + * type is not yet bound. Use {@link ParDo.UnboundMulti#of} + * to specify the {@link DoFn} to invoke, which will also bind the + * input type of this {@code PTransform}. + */ + public static UnboundMulti withOutputTags( + TupleTag mainOutputTag, + TupleTagList sideOutputTags) { + return new Unbound().withOutputTags(mainOutputTag, sideOutputTags); + } + + /** + * Creates a {@code ParDo} {@code PTransform} that will invoke the + * given {@link DoFn} function. + * + *

The resulting {@code PTransform}'s types have been bound, with the + * input being a {@code PCollection} and the output a + * {@code PCollection}, inferred from the types of the argument + * {@code DoFn}. It is ready to be applied, or further + * properties can be set on it first. + */ + public static Bound of(DoFn fn) { + return new Unbound().of(fn); + } + + /** + * An incomplete {@code ParDo} transform, with unbound input/output types. + * + *

Before being applied, {@link ParDo.Unbound#of} must be + * invoked to specify the {@link DoFn} to invoke, which will also + * bind the input/output types of this {@code PTransform}. + */ + public static class Unbound { + String name; + List> sideInputs = Collections.emptyList(); + + Unbound() {} + + Unbound(String name, + List> sideInputs) { + this.name = name; + this.sideInputs = sideInputs; + } + + /** + * Returns a new {@code ParDo} transform that's like this + * transform but with the specified name. Does not modify this + * transform. The resulting transform is still incomplete. + * + *

See the discussion of Naming above for more explanation. + */ + public Unbound named(String name) { + return new Unbound(name, sideInputs); + } + + /** + * Returns a new {@code ParDo} transform that's like this + * transform but with the specified side inputs. + * Does not modify this transform. The resulting transform is + * still incomplete. + * + *

See the discussion of Side Inputs above and on + * {@link ParDo#withSideInputs} for more explanation. + */ + public Unbound withSideInputs(PCollectionView... sideInputs) { + return new Unbound(name, ImmutableList.copyOf(sideInputs)); + } + + /** + * Returns a new {@code ParDo} transform that's like this + * transform but with the specified side inputs. Does not modify + * this transform. The resulting transform is still incomplete. + * + *

See the discussion of Side Inputs above and on + * {@link ParDo#withSideInputs} for more explanation. + */ + public Unbound withSideInputs( + Iterable> sideInputs) { + return new Unbound(name, ImmutableList.copyOf(sideInputs)); + } + + /** + * Returns a new multi-output {@code ParDo} transform that's like + * this transform but with the specified main and side output + * tags. Does not modify this transform. The resulting transform + * is still incomplete. + * + *

See the discussion of Side Outputs above and on + * {@link ParDo#withOutputTags} for more explanation. + */ + public UnboundMulti withOutputTags(TupleTag mainOutputTag, + TupleTagList sideOutputTags) { + return new UnboundMulti<>( + name, sideInputs, mainOutputTag, sideOutputTags); + } + + /** + * Returns a new {@code ParDo} {@code PTransform} that's like this + * transform but which will invoke the given {@link DoFn} + * function, and which has its input and output types bound. Does + * not modify this transform. The resulting {@code PTransform} is + * sufficiently specified to be applied, but more properties can + * still be specified. + */ + public Bound of(DoFn fn) { + return new Bound<>(name, sideInputs, fn); + } + } + + /** + * A {@code PTransform} that, when applied to a {@code PCollection}, + * invokes a user-specified {@code DoFn} on all its elements, + * with all its outputs collected into an output + * {@code PCollection}. + * + *

A multi-output form of this transform can be created with + * {@link ParDo.Bound#withOutputTags}. + * + * @param the type of the (main) input {@code PCollection} elements + * @param the type of the (main) output {@code PCollection} elements + */ + public static class Bound + extends PTransform, PCollection> { + // Inherits name. + List> sideInputs; + DoFn fn; + + Bound(String name, + List> sideInputs, + DoFn fn) { + super(name); + this.sideInputs = sideInputs; + this.fn = fn; + } + + /** + * Returns a new {@code ParDo} {@code PTransform} that's like this + * {@code PTransform} but with the specified name. Does not + * modify this {@code PTransform}. + * + *

See the discussion of Naming above for more explanation. + */ + public Bound named(String name) { + return new Bound<>(name, sideInputs, fn); + } + + /** + * Returns a new {@code ParDo} {@code PTransform} that's like this + * {@code PTransform} but with the specified side inputs. Does not + * modify this {@code PTransform}. + * + *

See the discussion of Side Inputs above and on + * {@link ParDo#withSideInputs} for more explanation. + */ + public Bound withSideInputs(PCollectionView... sideInputs) { + return new Bound<>(name, ImmutableList.copyOf(sideInputs), fn); + } + + /** + * Returns a new {@code ParDo} {@code PTransform} that's like this + * {@code PTransform} but with the specified side inputs. Does not + * modify this {@code PTransform}. + * + *

See the discussion of Side Inputs above and on + * {@link ParDo#withSideInputs} for more explanation. + */ + public Bound withSideInputs( + Iterable> sideInputs) { + return new Bound<>(name, ImmutableList.copyOf(sideInputs), fn); + } + + /** + * Returns a new multi-output {@code ParDo} {@code PTransform} + * that's like this {@code PTransform} but with the specified main + * and side output tags. Does not modify this {@code PTransform}. + * + *

See the discussion of Side Outputs above and on + * {@link ParDo#withOutputTags} for more explanation. + */ + public BoundMulti withOutputTags(TupleTag mainOutputTag, + TupleTagList sideOutputTags) { + return new BoundMulti<>( + name, sideInputs, mainOutputTag, sideOutputTags, fn); + } + + @Override + public PCollection apply(PCollection input) { + if (sideInputs == null) { + sideInputs = Collections.emptyList(); + } + return PCollection.createPrimitiveOutputInternal(getInput().getWindowingFn()) + .setTypeTokenInternal(fn.getOutputTypeToken()); + } + + @Override + protected Coder getDefaultOutputCoder() { + return getPipeline().getCoderRegistry().getDefaultCoder( + fn.getOutputTypeToken(), + fn.getInputTypeToken(), + ((PCollection) getInput()).getCoder()); + } + + @Override + protected String getDefaultName() { + return StringUtils.approximateSimpleName(fn.getClass()); + } + + @Override + protected String getKindString() { return "ParDo"; } + + public DoFn getFn() { + return fn; + } + + public List> getSideInputs() { + return sideInputs; + } + } + + /** + * An incomplete multi-output {@code ParDo} transform, with unbound + * input type. + * + *

Before being applied, {@link ParDo.UnboundMulti#of} must be + * invoked to specify the {@link DoFn} to invoke, which will also + * bind the input type of this {@code PTransform}. + * + * @param the type of the main output {@code PCollection} elements + */ + public static class UnboundMulti { + String name; + List> sideInputs; + TupleTag mainOutputTag; + TupleTagList sideOutputTags; + + UnboundMulti(String name, + List> sideInputs, + TupleTag mainOutputTag, + TupleTagList sideOutputTags) { + this.name = name; + this.sideInputs = sideInputs; + this.mainOutputTag = mainOutputTag; + this.sideOutputTags = sideOutputTags; + } + + /** + * Returns a new multi-output {@code ParDo} transform that's like + * this transform but with the specified name. Does not modify + * this transform. The resulting transform is still incomplete. + * + *

See the discussion of Naming above for more explanation. + */ + public UnboundMulti named(String name) { + return new UnboundMulti<>( + name, sideInputs, mainOutputTag, sideOutputTags); + } + + /** + * Returns a new multi-output {@code ParDo} transform that's like + * this transform but with the specified side inputs. Does not + * modify this transform. The resulting transform is still + * incomplete. + * + *

See the discussion of Side Inputs above and on + * {@link ParDo#withSideInputs} for more explanation. + */ + public UnboundMulti withSideInputs( + PCollectionView... sideInputs) { + return new UnboundMulti<>( + name, ImmutableList.copyOf(sideInputs), + mainOutputTag, sideOutputTags); + } + + /** + * Returns a new multi-output {@code ParDo} transform that's like + * this transform but with the specified side inputs. Does not + * modify this transform. The resulting transform is still + * incomplete. + * + *

See the discussion of Side Inputs above and on + * {@link ParDo#withSideInputs} for more explanation. + */ + public UnboundMulti withSideInputs( + Iterable> sideInputs) { + return new UnboundMulti<>( + name, ImmutableList.copyOf(sideInputs), + mainOutputTag, sideOutputTags); + } + + /** + * Returns a new multi-output {@code ParDo} {@code PTransform} + * that's like this transform but which will invoke the given + * {@link DoFn} function, and which has its input type bound. + * Does not modify this transform. The resulting + * {@code PTransform} is sufficiently specified to be applied, but + * more properties can still be specified. + */ + public BoundMulti of(DoFn fn) { + return new BoundMulti<>( + name, sideInputs, mainOutputTag, sideOutputTags, fn); + } + } + + /** + * A {@code PTransform} that, when applied to a + * {@code PCollection}, invokes a user-specified + * {@code DoFn} on all its elements, which can emit elements + * to any of the {@code PTransform}'s main and side output + * {@code PCollection}s, which are bundled into a result + * {@code PCollectionTuple}. + * + * @param the type of the (main) input {@code PCollection} elements + * @param the type of the main output {@code PCollection} elements + */ + public static class BoundMulti + extends PTransform, PCollectionTuple> { + // Inherits name. + List> sideInputs; + TupleTag mainOutputTag; + TupleTagList sideOutputTags; + DoFn fn; + + BoundMulti(String name, + List> sideInputs, + TupleTag mainOutputTag, + TupleTagList sideOutputTags, + DoFn fn) { + super(name); + this.sideInputs = sideInputs; + this.mainOutputTag = mainOutputTag; + this.sideOutputTags = sideOutputTags; + this.fn = fn; + } + + /** + * Returns a new multi-output {@code ParDo} {@code PTransform} + * that's like this {@code PTransform} but with the specified + * name. Does not modify this {@code PTransform}. + * + *

See the discussion of Naming above for more explanation. + */ + public BoundMulti named(String name) { + return new BoundMulti<>( + name, sideInputs, mainOutputTag, sideOutputTags, fn); + } + + /** + * Returns a new multi-output {@code ParDo} {@code PTransform} + * that's like this {@code PTransform} but with the specified side + * inputs. Does not modify this {@code PTransform}. + * + *

See the discussion of Side Inputs above and on + * {@link ParDo#withSideInputs} for more explanation. + */ + public BoundMulti withSideInputs( + PCollectionView... sideInputs) { + return new BoundMulti<>( + name, ImmutableList.copyOf(sideInputs), + mainOutputTag, sideOutputTags, fn); + } + + /** + * Returns a new multi-output {@code ParDo} {@code PTransform} + * that's like this {@code PTransform} but with the specified side + * inputs. Does not modify this {@code PTransform}. + * + *

See the discussion of Side Inputs above and on + * {@link ParDo#withSideInputs} for more explanation. + */ + public BoundMulti withSideInputs( + Iterable> sideInputs) { + return new BoundMulti<>( + name, ImmutableList.copyOf(sideInputs), + mainOutputTag, sideOutputTags, fn); + } + + + @Override + public PCollectionTuple apply(PCollection input) { + PCollectionTuple outputs = PCollectionTuple.ofPrimitiveOutputsInternal( + TupleTagList.of(mainOutputTag).and(sideOutputTags.getAll()), + getInput().getWindowingFn()); + + // The fn will likely be an instance of an anonymous subclass + // such as DoFn { }, thus will have a high-fidelity + // TypeToken for the output type. + outputs.get(mainOutputTag).setTypeTokenInternal(fn.getOutputTypeToken()); + + return outputs; + } + + @Override + protected Coder getDefaultOutputCoder() { + throw new RuntimeException( + "internal error: shouldn't be calling this on a multi-output ParDo"); + } + + @Override + protected String getDefaultName() { + return StringUtils.approximateSimpleName(fn.getClass()); + } + + @Override + protected String getKindString() { return "ParMultiDo"; } + + public DoFn getFn() { + return fn; + } + + public TupleTag getMainOutputTag() { + return mainOutputTag; + } + + public List> getSideInputs() { + return sideInputs; + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + Bound.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + Bound transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateSingleHelper(transform, context); + } + }); + } + + private static void evaluateSingleHelper( + Bound transform, + DirectPipelineRunner.EvaluationContext context) { + TupleTag mainOutputTag = new TupleTag<>("out"); + + DirectModeExecutionContext executionContext = new DirectModeExecutionContext(); + + DoFnRunner fnRunner = + evaluateHelper(transform.fn, context.getStepName(transform), + transform.getInput(), transform.sideInputs, + mainOutputTag, new ArrayList>(), + context, executionContext); + + context.setPCollectionValuesWithMetadata( + transform.getOutput(), + executionContext.getOutput(mainOutputTag)); + } + + ///////////////////////////////////////////////////////////////////////////// + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + BoundMulti.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + BoundMulti transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateMultiHelper(transform, context); + } + }); + } + + private static void evaluateMultiHelper( + BoundMulti transform, + DirectPipelineRunner.EvaluationContext context) { + + DirectModeExecutionContext executionContext = new DirectModeExecutionContext(); + + DoFnRunner fnRunner = + evaluateHelper(transform.fn, context.getStepName(transform), + transform.getInput(), transform.sideInputs, + transform.mainOutputTag, transform.sideOutputTags.getAll(), + context, executionContext); + + for (Map.Entry, PCollection> entry + : transform.getOutput().getAll().entrySet()) { + TupleTag tag = (TupleTag) entry.getKey(); + @SuppressWarnings("unchecked") + PCollection pc = (PCollection) entry.getValue(); + + context.setPCollectionValuesWithMetadata( + pc, + (tag == transform.mainOutputTag + ? executionContext.getOutput(tag) + : executionContext.getSideOutput(tag))); + } + } + + private static DoFnRunner evaluateHelper( + DoFn doFn, + String name, + PCollection input, + List> sideInputs, + TupleTag mainOutputTag, + List> sideOutputTags, + DirectPipelineRunner.EvaluationContext context, + DirectModeExecutionContext executionContext) { + // TODO: Run multiple shards? + DoFn fn = context.ensureSerializable(doFn); + + PTuple sideInputValues = PTuple.empty(); + for (PCollectionView view : sideInputs) { + sideInputValues = sideInputValues.and( + view.getTagInternal(), + context.getPCollectionView(view)); + } + + DoFnRunner fnRunner = + DoFnRunner.createWithListOutputs( + context.getPipelineOptions(), + fn, + sideInputValues, + mainOutputTag, + sideOutputTags, + executionContext.getStepContext(name), + context.getAddCounterMutator()); + + fnRunner.startBundle(); + + for (DirectPipelineRunner.ValueWithMetadata elem + : context.getPCollectionValuesWithMetadata(input)) { + executionContext.setKey(elem.getKey()); + fnRunner.processElement((WindowedValue) elem.getWindowedValue()); + } + + fnRunner.finishBundle(); + + return fnRunner; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java new file mode 100644 index 0000000000000..74a1359aa5ed0 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java @@ -0,0 +1,173 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; +import com.google.cloud.dataflow.sdk.values.PCollectionTuple; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TupleTagList; + +import java.io.Serializable; + +/** + * {@code Partition} takes a {@code PCollection} and a + * {@code PartitionFn}, uses the {@code PartitionFn} to split the + * elements of the input {@code PCollection} into {@code N} partitions, and + * returns a {@code PCollectionList} that bundles {@code N} + * {@code PCollection}s containing the split elements. + * + *

Example of use: + *

 {@code
+ * PCollection students = ...;
+ * // Split students up into 10 partitions, by percentile:
+ * PCollectionList studentsByPercentile =
+ *     students.apply(Partition.of(10, new PartitionFn() {
+ *         public int partitionFor(Student student, int numPartitions) {
+ *             return student.getPercentile()  // 0..99
+ *                  * numPartitions / 100;
+ *         }}))
+ * for (int i = 0; i < 10; i++) {
+ *   PCollection partition = studentsByPercentile.get(i);
+ *   ...
+ * }
+ * } 
+ * + *

By default, the {@code Coder} of each of the + * {@code PCollection}s in the output {@code PCollectionList} is the + * same as the {@code Coder} of the input {@code PCollection}. + * + *

Each output element has the same timestamp and is in the same windows + * as its corresponding input element, and each output {@code PCollection} + * has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * associated with it as the input. + * + * @param the type of the elements of the input and output + * {@code PCollection}s + */ +public class Partition + extends PTransform, PCollectionList> { + + /** + * A function object that chooses an output partition for an element. + * + * @param the type of the elements being partitioned + */ + public interface PartitionFn extends Serializable { + /** + * Chooses the partition into which to put the given element. + * + * @param elem the element to be partitioned + * @param numPartitions the total number of partitions ({@code >= 1}) + * @return index of the selected partition (in the range + * {@code [0..numPartitions-1]}) + */ + public int partitionFor(T elem, int numPartitions); + } + + /** + * Returns a new {@code Partition} {@code PTransform} that divides + * its input {@code PCollection} into the given number of partitions, + * using the given partitioning function. + * + * @param numPartitions the number of partitions to divide the input + * {@code PCollection} into + * @param partitionFn the function to invoke on each element to + * choose its output partition + * @throws IllegalArgumentException if {@code numPartitions <= 0} + */ + public static Partition of( + int numPartitions, PartitionFn partitionFn) { + return new Partition<>(new PartitionDoFn(numPartitions, partitionFn)); + } + + @Override + public PCollectionList apply(PCollection in) { + final TupleTagList outputTags = partitionDoFn.getOutputTags(); + + PCollectionTuple outputs = in.apply( + ParDo + .withOutputTags(new TupleTag(){}, outputTags) + .of(partitionDoFn)); + + PCollectionList pcs = PCollectionList.empty(in.getPipeline()); + Coder coder = in.getCoder(); + + for (TupleTag outputTag : outputTags.getAll()) { + // All the tuple tags are actually TupleTag + // And all the collections are actually PCollection + @SuppressWarnings("unchecked") + TupleTag typedOutputTag = (TupleTag) outputTag; + pcs = pcs.and(outputs.get(typedOutputTag).setCoder(coder)); + } + return pcs; + } + + ///////////////////////////////////////////////////////////////////////////// + + private final transient PartitionDoFn partitionDoFn; + + private Partition(PartitionDoFn partitionDoFn) { + this.partitionDoFn = partitionDoFn; + } + + private static class PartitionDoFn extends DoFn { + private final int numPartitions; + private final PartitionFn partitionFn; + private final TupleTagList outputTags; + + /** + * Constructs a PartitionDoFn. + * + * @throws IllegalArgumentException if {@code numPartitions <= 0} + */ + public PartitionDoFn( + int numPartitions, PartitionFn partitionFn) { + if (numPartitions <= 0) { + throw new IllegalArgumentException("numPartitions must be > 0"); + } + + this.numPartitions = numPartitions; + this.partitionFn = partitionFn; + + TupleTagList buildOutputTags = TupleTagList.empty(); + for (int partition = 0; partition < numPartitions; partition++) { + buildOutputTags = buildOutputTags.and(new TupleTag()); + } + outputTags = buildOutputTags; + } + + public TupleTagList getOutputTags() { + return outputTags; + } + + @Override + public void processElement(ProcessContext c) { + T1 input = c.element(); + int partition = partitionFn.partitionFor(input, numPartitions); + if (0 <= partition && partition < numPartitions) { + c.sideOutput((TupleTag) outputTags.get(partition), input); + } else { + throw new IndexOutOfBoundsException( + "Partition function returned out of bounds index: " + + partition + " not in [0.." + numPartitions + ")"); + } + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java new file mode 100644 index 0000000000000..2124acfbb84a5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java @@ -0,0 +1,336 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.api.client.util.Throwables; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.reflect.TypeToken; +import com.google.common.util.concurrent.RateLimiter; + +import org.joda.time.Instant; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Provides rate-limiting of user functions, using threaded execution and a + * {@link com.google.common.util.concurrent.RateLimiter} to process elements + * at the desired rate. + * + *

For example, to limit each worker to 10 requests per second: + *

{@code
+ * PCollection data = ...;
+ * data.apply(
+ *   RateLimiting.perWorker(new MyDoFn())
+ *               .withRateLimit(10)));
+ * }
+ * + *

An uncaught exception from the wrapped DoFn will result in the exception + * being rethrown in later calls to {@link RateLimitingDoFn#processElement} + * or a call to {@link RateLimitingDoFn#finishBundle}. + * + *

Rate limiting is provided as a PTransform + * ({@link RateLimitingTransform}), and also as a {@code DoFn} + * ({@link RateLimitingDoFn}). + */ +public class RateLimiting { + + /** + * Creates a new per-worker rate-limiting transform for the given + * {@link com.google.cloud.dataflow.sdk.transforms.DoFn}. + * + *

The default behavior is to process elements with multiple threads, + * but no rate limit is applied. + * + *

Use {@link RateLimitingTransform#withRateLimit} to limit the processing + * rate, and {@link RateLimitingTransform#withMaxParallelism} to control the + * maximum concurrent processing limit. + * + *

Aside from the above, the {@code DoFn} will be executed in the same manner + * as in {@link ParDo}. + * + *

Rate limiting is applied independently per-worker. + */ + public static RateLimitingTransform perWorker(DoFn doFn) { + return new RateLimitingTransform<>(doFn); + } + + /** + * A {@link PTransform} which applies rate limiting to a {@link DoFn}. + * + * @param the type of the (main) input elements + * @param the type of the (main) output elements + */ + public static class RateLimitingTransform + extends PTransform, PCollection> { + private final DoFn doFn; + private double rate = 0.0; + // TODO: set default based on num cores, or based on rate limit? + private int maxParallelism = DEFAULT_MAX_PARALLELISM; + + public RateLimitingTransform(DoFn doFn) { + this.doFn = doFn; + } + + /** + * Modifies this {@code RateLimitingTransform}, specifying a maximum + * per-worker element processing rate. + * + *

A rate of {@code N} corresponds to {@code N} elements per second. + * This rate is on a per-worker basis, so the overall rate of the job + * depends upon the number of workers. + * + *

This rate limit may not be reachable unless there is sufficient + * parallelism. + * + *

A rate of <= 0.0 disables rate limiting. + */ + public RateLimitingTransform withRateLimit( + double maxElementsPerSecond) { + this.rate = maxElementsPerSecond; + return this; + } + + /** + * Modifies this {@code RateLimitingTransform}, specifying a maximum + * per-worker parallelism. + * + *

This determines how many concurrent elements will be processed by the + * wrapped {@code DoFn}. + * + *

The desired amount of parallelism depends upon the type of work. For + * CPU-intensive work, a good starting point is to use the number of cores: + * {@code Runtime.getRuntime().availableProcessors()}. + */ + public RateLimitingTransform withMaxParallelism(int max) { + this.maxParallelism = max; + return this; + } + + @Override + public PCollection apply(PCollection input) { + return input.apply( + ParDo.of(new RateLimitingDoFn<>(doFn, rate, maxParallelism))); + } + } + + /** + * A rate-limiting {@code DoFn} wrapper. + * + * @see RateLimiting#perWorker(DoFn) + * + * @param the type of the (main) input elements + * @param the type of the (main) output elements + */ + public static class RateLimitingDoFn extends DoFn { + private static final Logger LOG = LoggerFactory.getLogger(RateLimitingDoFn.class); + + public RateLimitingDoFn(DoFn doFn, double rateLimit, + int maxParallelism) { + this.doFn = doFn; + this.rate = rateLimit; + this.maxParallelism = maxParallelism; + } + + @Override + public void startBundle(Context c) throws Exception { + doFn.startBundle(c); + + if (rate > 0.0) { + limiter = RateLimiter.create(rate); + } + executor = Executors.newCachedThreadPool(); + workTickets = new Semaphore(maxParallelism); + failure = new AtomicReference<>(); + } + + @Override + public void processElement(final ProcessContext c) throws Exception { + // Apply rate limiting up front, controlling the availability of work for + // the thread pool. This allows us to use an auto-scaling thread pool, + // which adapts the parallelism to the available work. + // The semaphore is used to avoid overwhelming the executor, by bounding + // the number of outstanding elements. + if (limiter != null) { + limiter.acquire(); + } + try { + workTickets.acquire(); + } catch (InterruptedException e) { + throw new RuntimeException("Interrupted while scheduling work", e); + } + + if (failure.get() != null) { + throw Throwables.propagate(failure.get()); + } + + executor.submit(new Runnable() { + @Override + public void run() { + try { + doFn.processElement(new WrappedContext(c)); + } catch (Throwable t) { + failure.compareAndSet(null, t); + Throwables.propagateIfPossible(t); + throw new AssertionError("Unexpected checked exception: " + t); + } finally { + workTickets.release(); + } + } + }); + } + + @Override + public void finishBundle(Context c) throws Exception { + executor.shutdown(); + // Log a periodic progress report until the queue has drained. + while (true) { + try { + if (executor.awaitTermination(30, TimeUnit.SECONDS)) { + if (failure.get() != null) { + // Handle failure propagation outside of the try/catch block. + break; + } + doFn.finishBundle(c); + return; + } + int outstanding = workTickets.getQueueLength() + + maxParallelism - workTickets.availablePermits(); + LOG.info("RateLimitingDoFn backlog: {}", outstanding); + } catch (InterruptedException e) { + throw Throwables.propagate(e); + } + } + + throw Throwables.propagate(failure.get()); + } + + @Override + TypeToken getInputTypeToken() { + return doFn.getInputTypeToken(); + } + + @Override + TypeToken getOutputTypeToken() { + return doFn.getOutputTypeToken(); + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Wraps a DoFn context, forcing single-thread output so that threads don't + * propagate through to downstream functions. + */ + private class WrappedContext extends ProcessContext { + private final ProcessContext context; + + WrappedContext(ProcessContext context) { + this.context = context; + } + + @Override + public I element() { + return context.element(); + } + + @Override + public KeyedState keyedState() { + return context.keyedState(); + } + + @Override + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); + } + + @Override + public T sideInput(PCollectionView view) { + return context.sideInput(view); + } + + @Override + public void output(O output) { + synchronized (RateLimitingDoFn.this) { + context.output(output); + } + } + + @Override + public void outputWithTimestamp(O output, Instant timestamp) { + synchronized (RateLimitingDoFn.this) { + context.outputWithTimestamp(output, timestamp); + } + } + + @Override + public void sideOutput(TupleTag tag, T output) { + synchronized (RateLimitingDoFn.this) { + context.sideOutput(tag, output); + } + } + + @Override + public Aggregator createAggregator( + String name, Combine.CombineFn combiner) { + return context.createAggregator(name, combiner); + } + + @Override + public Aggregator createAggregator( + String name, SerializableFunction, AO> combiner) { + return context.createAggregator(name, combiner); + } + + @Override + public Instant timestamp() { + return context.timestamp(); + } + + @Override + public Collection windows() { + return context.windows(); + } + } + + private final DoFn doFn; + private double rate; + private int maxParallelism; + + private transient RateLimiter limiter; + private transient ExecutorService executor; + private transient Semaphore workTickets; + private transient AtomicReference failure; + } + + /** + * Default maximum for number of concurrent elements to process. + */ + @VisibleForTesting + static final int DEFAULT_MAX_PARALLELISM = 16; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java new file mode 100644 index 0000000000000..0e4f21f75b781 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java @@ -0,0 +1,89 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +/** + * {@code RemoveDuplicates} takes a {@code PCollection} and + * returns a {@code PCollection} that has all the elements of the + * input but with duplicate elements removed such that each element is + * unique within each window. + * + *

Two values of type {@code T} are compared for equality not by + * regular Java {@link Object#equals}, but instead by first encoding + * each of the elements using the {@code PCollection}'s {@code Coder}, and then + * comparing the encoded bytes. This admits efficient parallel + * evaluation. + * + *

By default, the {@code Coder} of the output {@code PCollection} + * is the same as the {@code Coder} of the input {@code PCollection}. + * + *

Each output element is in the same window as its corresponding input + * element, and has the timestamp of the end of that window. The output + * {@code PCollection} has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * as the input. + * + *

Does not preserve any order the input PCollection might have had. + * + *

Example of use: + *

 {@code
+ * PCollection words = ...;
+ * PCollection uniqueWords =
+ *     words.apply(RemoveDuplicates.create());
+ * } 
+ * + * @param the type of the elements of the input and output + * {@code PCollection}s + */ +public class RemoveDuplicates extends PTransform, + PCollection> { + /** + * Returns a {@code RemoveDuplicates} {@code PTransform}. + * + * @param the type of the elements of the input and output + * {@code PCollection}s + */ + public static RemoveDuplicates create() { + return new RemoveDuplicates<>(); + } + + private RemoveDuplicates() { } + + @Override + public PCollection apply(PCollection in) { + return + in + .apply(ParDo.named("CreateIndex") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + c.output(KV.of(c.element(), (Void) null)); + } + })) + .apply(Combine.perKey( + new SerializableFunction, Void>() { + @Override + public Void apply(Iterable iter) { + return null; // ignore input + } + })) + .apply(Keys.create()); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java new file mode 100644 index 0000000000000..832cc996ea761 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java @@ -0,0 +1,154 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +/** + * {@code PTransform}s for taking samples of the elements in a + * {@code PCollection}, or samples of the values associated with each + * key in a {@code PCollection} of {@code KV}s. + **/ +public class Sample { + /** + * Returns a {@code PTransform} that takes a {@code PCollection}, + * selects {@code sampleSize} elements, uniformly at random, and returns a + * {@code PCollection>} containing the selected elements. + * If the input {@code PCollection} has fewer than + * {@code sampleSize} elements, then the output {@code Iterable} + * will be all the input's elements. + * + *

Example of use: + *

 {@code
+   * PCollection pc = ...;
+   * PCollection> sampleOfSize10 =
+   *     pc.apply(Sample.fixedSizeGlobally(10));
+   * } 
+ * + * @param sampleSize the number of elements to select; must be {@code >= 0} + * @param the type of the elements + */ + public static PTransform, PCollection>> + fixedSizeGlobally(int sampleSize) { + return Combine.globally(new FixedSizedSampleFn(sampleSize)); + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to a sample of {@code sampleSize} values + * associated with that key in the input {@code PCollection}, taken + * uniformly at random. If a key in the input {@code PCollection} + * has fewer than {@code sampleSize} values associated with it, then + * the output {@code Iterable} associated with that key will be + * all the values associated with that key in the input + * {@code PCollection}. + * + *

Example of use: + *

 {@code
+   * PCollection> pc = ...;
+   * PCollection>> sampleOfSize10PerKey =
+   *     pc.apply(Sample.fixedSizePerKey());
+   * } 
+ * + * @param sampleSize the number of values to select for each + * distinct key; must be {@code >= 0} + * @param the type of the keys + * @param the type of the values + */ + public static PTransform>, + PCollection>>> + fixedSizePerKey(int sampleSize) { + return Combine.perKey(new FixedSizedSampleFn(sampleSize)); + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * {@code CombineFn} that computes a fixed-size sample of a + * collection of values. + * + * @param the type of the elements + */ + public static class FixedSizedSampleFn + extends CombineFn>.Heap, Iterable> { + private final Top.TopCombineFn> topCombineFn; + private final Random rand = new Random(); + + private FixedSizedSampleFn(int sampleSize) { + if (sampleSize < 0) { + throw new IllegalArgumentException("sample size must be >= 0"); + } + topCombineFn = new Top.TopCombineFn<>(sampleSize, + new KV.OrderByKey()); + } + + @Override + public Top.TopCombineFn>.Heap createAccumulator() { + return topCombineFn.createAccumulator(); + } + + @Override + public void addInput(Top.TopCombineFn>.Heap accumulator, + T input) { + accumulator.addInput(KV.of(rand.nextInt(), input)); + } + + @Override + public Top.TopCombineFn>.Heap mergeAccumulators( + Iterable>.Heap> accumulators) { + return topCombineFn.mergeAccumulators(accumulators); + } + + @Override + public Iterable extractOutput( + Top.TopCombineFn>.Heap accumulator) { + List out = new ArrayList<>(); + for (KV element : accumulator.extractOutput()) { + out.add(element.getValue()); + } + return out; + } + + @Override + public Coder>.Heap> getAccumulatorCoder( + CoderRegistry registry, Coder inputCoder) { + return topCombineFn.getAccumulatorCoder( + registry, KvCoder.of(BigEndianIntegerCoder.of(), inputCoder)); + } + + @Override + public Coder> getDefaultOutputCoder( + CoderRegistry registry, Coder inputCoder) { + return IterableCoder.of(inputCoder); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/SerializableComparator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/SerializableComparator.java new file mode 100644 index 0000000000000..3d538faa54d85 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/SerializableComparator.java @@ -0,0 +1,28 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import java.io.Serializable; +import java.util.Comparator; + +/** + * A {@code Serializable} {@code Comparator}. + * + * @param type of values being compared + */ +public interface SerializableComparator extends Comparator, Serializable { +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/SerializableFunction.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/SerializableFunction.java new file mode 100644 index 0000000000000..857491a11fe84 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/SerializableFunction.java @@ -0,0 +1,31 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import java.io.Serializable; + +/** + * A function that computes an output value based on an input value, + * and is {@link Serializable}. + * + * @param input value type + * @param output value type + */ +public interface SerializableFunction extends Serializable { + /** Returns the result of invoking this function on the given input. */ + public O apply(I input); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java new file mode 100644 index 0000000000000..e925e4a5cc90d --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java @@ -0,0 +1,179 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +/** + * {@code PTransform}s for computing the sum of the elements in a + * {@code PCollection}, or the sum of the values associated with + * each key in a {@code PCollection} of {@code KV}s. + * + *

Example 1: get the sum of a {@code PCollection} of {@code Double}s. + *

 {@code
+ * PCollection input = ...;
+ * PCollection sum = input.apply(Sum.doublesGlobally());
+ * } 
+ * + *

Example 2: calculate the sum of the {@code Integer}s + * associated with each unique key (which is of type {@code String}). + *

 {@code
+ * PCollection> input = ...;
+ * PCollection> sumPerKey = input
+ *     .apply(Sum.integersPerKey());
+ * } 
+ */ +public class Sum { + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a + * {@code PCollection} whose contents is the sum of the + * input {@code PCollection}'s elements, or + * {@code 0} if there are no elements. + */ + public static Combine.Globally integersGlobally() { + Combine.Globally combine = Combine + .globally(new SumIntegerFn()); + combine.setName("Sum"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to the sum of the values associated with + * that key in the input {@code PCollection}. + */ + public static Combine.PerKey integersPerKey() { + Combine.PerKey combine = Combine + .perKey(new SumIntegerFn()); + combine.setName("Sum.PerKey"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a + * {@code PCollection} whose contents is the sum of the + * input {@code PCollection}'s elements, or + * {@code 0} if there are no elements. + */ + public static Combine.Globally longsGlobally() { + Combine.Globally combine = Combine.globally(new SumLongFn()); + combine.setName("Sum"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to the sum of the values associated with + * that key in the input {@code PCollection}. + */ + public static Combine.PerKey longsPerKey() { + Combine.PerKey combine = Combine + .perKey(new SumLongFn()); + combine.setName("Sum.PerKey"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a + * {@code PCollection} whose contents is the sum of the + * input {@code PCollection}'s elements, or + * {@code 0} if there are no elements. + */ + public static Combine.Globally doublesGlobally() { + Combine.Globally combine = Combine + .globally(new SumDoubleFn()); + combine.setName("Sum"); + return combine; + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to the sum of the values associated with + * that key in the input {@code PCollection}. + */ + public static Combine.PerKey doublesPerKey() { + Combine.PerKey combine = Combine + .perKey(new SumDoubleFn()); + combine.setName("Sum.PerKey"); + return combine; + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * A {@code SerializableFunction} that computes the sum of an + * {@code Iterable} of {@code Integer}s, useful as an argument to + * {@link Combine#globally} or {@link Combine#perKey}. + */ + public static class SumIntegerFn + implements SerializableFunction, Integer> { + @Override + public Integer apply(Iterable input) { + int sum = 0; + for (int value : input) { + sum += value; + } + return sum; + } + } + + /** + * A {@code SerializableFunction} that computes the sum of an + * {@code Iterable} of {@code Long}s, useful as an argument to + * {@link Combine#globally} or {@link Combine#perKey}. + */ + public static class SumLongFn + implements SerializableFunction, Long> { + @Override + public Long apply(Iterable input) { + long sum = 0; + for (long value : input) { + sum += value; + } + return sum; + } + } + + /** + * A {@code SerializableFunction} that computes the sum of an + * {@code Iterable} of {@code Double}s, useful as an argument to + * {@link Combine#globally} or {@link Combine#perKey}. + */ + public static class SumDoubleFn + implements SerializableFunction, Double> { + @Override + public Double apply(Iterable input) { + double sum = 0; + for (double value : input) { + sum += value; + } + return sum; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java new file mode 100644 index 0000000000000..1f63808fc2237 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java @@ -0,0 +1,489 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.coders.CustomCoder; +import com.google.cloud.dataflow.sdk.coders.ListCoder; +import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.PriorityQueue; + +/** + * {@code PTransform}s for finding the largest (or smallest) set + * of elements in a {@code PCollection}, or the largest (or smallest) + * set of values associated with each key in a {@code PCollection} of + * {@code KV}s. + */ +public class Top { + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a {@code PCollection>} with a + * single element containing the largest {@code count} elements of the input + * {@code PCollection}, in decreasing order, sorted using the + * given {@code Comparator}. The {@code Comparator} must also + * be {@code Serializable}. + * + *

If {@code count} {@code <} the number of elements in the + * input {@code PCollection}, then all the elements of the input + * {@code PCollection} will be in the resulting + * {@code List}, albeit in sorted order. + * + *

All the elements of the result's {@code List} + * must fit into the memory of a single machine. + * + *

Example of use: + *

 {@code
+   * PCollection students = ...;
+   * PCollection> top10Students =
+   *     students.apply(Top.of(10, new CompareStudentsByAvgGrade()));
+   * } 
+ * + *

By default, the {@code Coder} of the output {@code PCollection} + * is a {@code ListCoder} of the {@code Coder} of the elements of + * the input {@code PCollection}. + * + *

See also {@link #smallest} and {@link #largest}, which sort + * {@code Comparable} elements using their natural ordering. + * + *

See also {@link #perKey}, {@link #smallestPerKey}, and + * {@link #largestPerKey} which take a {@code PCollection} of + * {@code KV}s and return the top values associated with each key. + */ + public static & Serializable> + PTransform, PCollection>> of(int count, C compareFn) { + return Combine.globally(new TopCombineFn<>(count, compareFn)) + .withName("Top"); + + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a {@code PCollection>} with a + * single element containing the smallest {@code count} elements of the input + * {@code PCollection}, in increasing order, sorted according to + * their natural order. + * + *

If {@code count} {@code <} the number of elements in the + * input {@code PCollection}, then all the elements of the input + * {@code PCollection} will be in the resulting {@code PCollection}'s + * {@code List}, albeit in sorted order. + * + *

All the elements of the result {@code List} + * must fit into the memory of a single machine. + * + *

Example of use: + *

 {@code
+   * PCollection values = ...;
+   * PCollection> smallest10Values = values.apply(Top.smallest(10));
+   * } 
+ * + *

By default, the {@code Coder} of the output {@code PCollection} + * is a {@code ListCoder} of the {@code Coder} of the elements of + * the input {@code PCollection}. + * + *

See also {@link #largest}. + * + *

See also {@link #of}, which sorts using a user-specified + * {@code Comparator} function. + * + *

See also {@link #perKey}, {@link #smallestPerKey}, and + * {@link #largestPerKey} which take a {@code PCollection} of + * {@code KV}s and return the top values associated with each key. + */ + public static > + PTransform, PCollection>> smallest(int count) { + return Combine.globally(new TopCombineFn<>(count, new Smallest())) + .withName("Top.Smallest"); + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a {@code PCollection>} with a + * single element containing the largest {@code count} elements of the input + * {@code PCollection}, in decreasing order, sorted according to + * their natural order. + * + *

If {@code count} {@code <} the number of elements in the + * input {@code PCollection}, then all the elements of the input + * {@code PCollection} will be in the resulting {@code PCollection}'s + * {@code List}, albeit in sorted order. + * + *

All the elements of the result's {@code List} + * must fit into the memory of a single machine. + * + *

Example of use: + *

 {@code
+   * PCollection values = ...;
+   * PCollection> largest10Values = values.apply(Top.largest(10));
+   * } 
+ * + *

By default, the {@code Coder} of the output {@code PCollection} + * is a {@code ListCoder} of the {@code Coder} of the elements of + * the input {@code PCollection}. + * + *

See also {@link #smallest}. + * + *

See also {@link #of}, which sorts using a user-specified + * {@code Comparator} function. + * + *

See also {@link #perKey}, {@link #smallestPerKey}, and + * {@link #largestPerKey} which take a {@code PCollection} of + * {@code KV}s and return the top values associated with each key. + */ + public static > + PTransform, PCollection>> largest(int count) { + return Combine.globally(new TopCombineFn<>(count, new Largest())) + .withName("Top.Largest"); + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to the largest {@code count} values + * associated with that key in the input + * {@code PCollection>}, in decreasing order, sorted using + * the given {@code Comparator}. The + * {@code Comparator} must also be {@code Serializable}. + * + *

If there are fewer than {@code count} values associated with + * a particular key, then all those values will be in the result + * mapping for that key, albeit in sorted order. + * + *

All the values associated with a single key must fit into the + * memory of a single machine, but there can be many more + * {@code KV}s in the resulting {@code PCollection} than can fit + * into the memory of a single machine. + * + *

Example of use: + *

 {@code
+   * PCollection> studentsBySchool = ...;
+   * PCollection>> top10StudentsBySchool =
+   *     studentsBySchool.apply(
+   *         Top.perKey(10, new CompareStudentsByAvgGrade()));
+   * } 
+ * + *

By default, the {@code Coder} of the keys of the output + * {@code PCollection} is the same as that of the keys of the input + * {@code PCollection}, and the {@code Coder} of the values of the + * output {@code PCollection} is a {@code ListCoder} of the + * {@code Coder} of the values of the input {@code PCollection}. + * + *

See also {@link #smallestPerKey} and {@link #largestPerKey}, + * which sort {@code Comparable} values using their natural + * ordering. + * + *

See also {@link #of}, {@link #smallest}, and {@link #largest} + * which take a {@code PCollection} and return the top elements. + */ + public static & Serializable> + PTransform>, PCollection>>> + perKey(int count, C compareFn) { + return Combine.perKey( + new TopCombineFn<>(count, compareFn).asKeyedFn()) + .withName("Top.PerKey"); + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to the smallest {@code count} values + * associated with that key in the input + * {@code PCollection>}, in increasing order, sorted + * according to their natural order. + * + *

If there are fewer than {@code count} values associated with + * a particular key, then all those values will be in the result + * mapping for that key, albeit in sorted order. + * + *

All the values associated with a single key must fit into the + * memory of a single machine, but there can be many more + * {@code KV}s in the resulting {@code PCollection} than can fit + * into the memory of a single machine. + * + *

Example of use: + *

 {@code
+   * PCollection> keyedValues = ...;
+   * PCollection>> smallest10ValuesPerKey =
+   *     keyedValues.apply(Top.smallestPerKey(10));
+   * } 
+ * + *

By default, the {@code Coder} of the keys of the output + * {@code PCollection} is the same as that of the keys of the input + * {@code PCollection}, and the {@code Coder} of the values of the + * output {@code PCollection} is a {@code ListCoder} of the + * {@code Coder} of the values of the input {@code PCollection}. + * + *

See also {@link #largestPerKey}. + * + *

See also {@link #perKey}, which sorts values using a user-specified + * {@code Comparator} function. + * + *

See also {@link #of}, {@link #smallest}, and {@link #largest} + * which take a {@code PCollection} and return the top elements. + */ + public static > + PTransform>, PCollection>>> + smallestPerKey(int count) { + return Combine.perKey( + new TopCombineFn<>(count, new Smallest()).asKeyedFn()) + .withName("Top.SmallestPerKey"); + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection>} and returns a + * {@code PCollection>>} that contains an output + * element mapping each distinct key in the input + * {@code PCollection} to the largest {@code count} values + * associated with that key in the input + * {@code PCollection>}, in decreasing order, sorted + * according to their natural order. + * + *

If there are fewer than {@code count} values associated with + * a particular key, then all those values will be in the result + * mapping for that key, albeit in sorted order. + * + *

All the values associated with a single key must fit into the + * memory of a single machine, but there can be many more + * {@code KV}s in the resulting {@code PCollection} than can fit + * into the memory of a single machine. + * + *

Example of use: + *

 {@code
+   * PCollection> keyedValues = ...;
+   * PCollection>> largest10ValuesPerKey =
+   *     keyedValues.apply(Top.largestPerKey(10));
+   * } 
+ * + *

By default, the {@code Coder} of the keys of the output + * {@code PCollection} is the same as that of the keys of the input + * {@code PCollection}, and the {@code Coder} of the values of the + * output {@code PCollection} is a {@code ListCoder} of the + * {@code Coder} of the values of the input {@code PCollection}. + * + *

See also {@link #smallestPerKey}. + * + *

See also {@link #perKey}, which sorts values using a user-specified + * {@code Comparator} function. + * + *

See also {@link #of}, {@link #smallest}, and {@link #largest} + * which take a {@code PCollection} and return the top elements. + */ + public static > + PTransform>, PCollection>>> + largestPerKey(int count) { + return Combine.perKey( + new TopCombineFn<>(count, new Largest()).asKeyedFn()) + .withName("Top.LargestPerKey"); + } + + + //////////////////////////////////////////////////////////////////////////// + + /** + * {@code CombineFn} for {@code Top} transforms that combines a + * bunch of {@code T}s into a single {@code count}-long + * {@code List}, using {@code compareFn} to choose the largest + * {@code T}s. + * + * @param type of element being compared + */ + public static class TopCombineFn + extends AccumulatingCombineFn.Heap, List> { + + private final int count; + private final Comparator compareFn; + + public & Serializable> TopCombineFn( + int count, C compareFn) { + if (count < 0) { + throw new IllegalArgumentException("count must be >= 0"); + } + this.count = count; + this.compareFn = compareFn; + } + + class Heap + // TODO: Why do I have to fully qualify the + // Accumulator class here? + extends AccumulatingCombineFn.Heap, List> + .Accumulator { + + // Exactly one of these should be set. + private List asList; // ordered largest first + private PriorityQueue asQueue; // head is smallest + + private Heap(List asList) { + this.asList = asList; + } + + @Override + public void addInput(T value) { + addInputInternal(value); + } + + private boolean addInputInternal(T value) { + if (count == 0) { + // Don't add anything. + return false; + } + + if (asQueue == null) { + asQueue = new PriorityQueue<>(count, compareFn); + for (T item : asList) { + asQueue.add(item); + } + asList = null; + } + + if (asQueue.size() < count) { + asQueue.add(value); + return true; + } else if (compareFn.compare(value, asQueue.peek()) > 0) { + asQueue.poll(); + asQueue.add(value); + return true; + } else { + return false; + } + } + + @Override + public void mergeAccumulator(Heap accumulator) { + for (T value : accumulator.asList()) { + if (!addInputInternal(value)) { + // The list is ordered, remainder will also all be smaller. + break; + } + } + } + + @Override + public List extractOutput() { + return asList(); + } + + private List asList() { + if (asList == null) { + int index = asQueue.size(); + @SuppressWarnings("unchecked") + T[] ordered = (T[]) new Object[index]; + while (!asQueue.isEmpty()) { + index--; + ordered[index] = asQueue.poll(); + } + asList = Arrays.asList(ordered); + asQueue = null; + } + return asList; + } + } + + @Override + public Heap createAccumulator() { + return new Heap(new ArrayList()); + } + + @Override + public Coder getAccumulatorCoder( + CoderRegistry registry, Coder inputCoder) { + return new HeapCoder(inputCoder); + } + + private class HeapCoder extends CustomCoder { + private final Coder> listCoder; + + public HeapCoder(Coder inputCoder) { + listCoder = ListCoder.of(inputCoder); + } + + @Override + public void encode(Heap value, OutputStream outStream, + Context context) throws CoderException, IOException { + listCoder.encode(value.asList(), outStream, context); + } + + @Override + public Heap decode(InputStream inStream, Coder.Context context) + throws CoderException, IOException { + return new Heap(listCoder.decode(inStream, context)); + } + + @Override + public boolean isDeterministic() { + return listCoder.isDeterministic(); + } + + @Override + public boolean isRegisterByteSizeObserverCheap( + Heap value, Context context) { + return listCoder.isRegisterByteSizeObserverCheap( + value.asList(), context); + } + + @Override + public void registerByteSizeObserver( + Heap value, ElementByteSizeObserver observer, Context context) + throws Exception { + listCoder.registerByteSizeObserver(value.asList(), observer, context); + } + }; + } + + /** + * {@code Serializable} {@code Comparator} that that uses the + * compared elements' natural ordering. + */ + public static class Largest> + implements Comparator, Serializable { + @Override + public int compare(T a, T b) { + return a.compareTo(b); + } + } + + /** + * {@code Serializable} {@code Comparator} that that uses the + * reverse of the compared elements' natural ordering. + */ + public static class Smallest> + implements Comparator, Serializable { + @Override + public int compare(T a, T b) { + return b.compareTo(a); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Values.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Values.java new file mode 100644 index 0000000000000..ae008b196ad3b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Values.java @@ -0,0 +1,68 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +/** + * {@code Values} takes a {@code PCollection} of {@code KV}s and + * returns a {@code PCollection} of the values. + * + *

Example of use: + *

 {@code
+ * PCollection> wordCounts = ...;
+ * PCollection counts = wordCounts.apply(Values.create());
+ * } 
+ * + *

Each output element has the same timestamp and is in the same windows + * as its corresponding input element, and the output {@code PCollection} + * has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * associated with it as the input. + * + *

See also {@link Keys}. + * + * @param the type of the values in the input {@code PCollection}, + * and the type of the elements in the output {@code PCollection} + */ +public class Values extends PTransform>, + PCollection> { + /** + * Returns a {@code Values} {@code PTransform}. + * + * @param the type of the values in the input {@code PCollection}, + * and the type of the elements in the output {@code PCollection} + */ + public static Values create() { + return new Values<>(); + } + + private Values() { } + + @Override + public PCollection apply(PCollection> in) { + return + in.apply(ParDo.named("Values") + .of(new DoFn, V>() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().getValue()); + } + })); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java new file mode 100644 index 0000000000000..d3bb863888707 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java @@ -0,0 +1,211 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.PValueBase; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.base.Function; +import com.google.common.collect.Iterables; + +import java.util.List; +import java.util.NoSuchElementException; + +/** + * Transforms for creating {@link PCollectionView}s from {@link PCollection}s, + * for consuming the contents of those {@link PCollection}s as side inputs + * to {@link ParDo} transforms. + */ +public class View { + + // Do not instantiate + private View() { } + + /** + * Returns a {@link AsSingleton} transform that takes a singleton + * {@link PCollection} as input and produces a {@link PCollectionView} + * of the single value, to be consumed as a side input. + * + *

If the input {@link PCollection} is empty, + * throws {@link NoSuchElementException} in the consuming + * {@link DoFn}. + * + *

If the input {@link PCollection} contains more than one + * element, throws {@link IllegalArgumentException} in the + * consuming {@link DoFn}. + */ + public static AsSingleton asSingleton() { + return new AsSingleton<>(); + } + + /** + * Returns a {@link AsIterable} that takes a + * {@link PCollection} as input and produces a {@link PCollectionView} + * of the values, to be consumed as an iterable side input. + */ + public static AsIterable asIterable() { + return new AsIterable<>(); + } + + + /** + * A {@PTransform} that produces a {@link PCollectionView} of a singleton {@link PCollection} + * yielding the single element it contains. + * + *

Instantiate via {@link View.asIterable}. + */ + public static class AsIterable extends PTransform< + PCollection, + PCollectionView, Iterable>>> { + + private AsIterable() { } + + @Override + public PCollectionView, Iterable>> apply( + PCollection input) { + return input.apply( + new CreatePCollectionView, Iterable>>( + new IterablePCollectionView(input.getPipeline()))); + } + } + + /** + * A {@PTransform} that produces a {@link PCollectionView} of a singleton {@link PCollection} + * yielding the single element it contains. + * + *

Instantiate via {@link View.asIterable}. + */ + public static class AsSingleton + extends PTransform, PCollectionView>> { + + private AsSingleton() { } + + @Override + public PCollectionView> apply(PCollection input) { + return input.apply( + new CreatePCollectionView>( + new SingletonPCollectionView(input.getPipeline()))); + } + + } + + + //////////////////////////////////////////////////////////////////////////// + // Internal details below + + /** + * Creates a primitive PCollectionView. + * + *

For internal use only. + * + * @param The type of the elements of the input PCollection + * @param The type associated with the PCollectionView used as a side input + * @param The type associated with a windowed side input from the + * PCollectionView + */ + public static class CreatePCollectionView + extends PTransform, PCollectionView> { + + private PCollectionView view; + + public CreatePCollectionView(PCollectionView view) { + this.view = view; + } + + @Override + public PCollectionView apply(PCollection input) { + return view; + } + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + CreatePCollectionView.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + CreatePCollectionView transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateTyped(transform, context); + } + + private void evaluateTyped( + CreatePCollectionView transform, + DirectPipelineRunner.EvaluationContext context) { + List> elems = + context.getPCollectionWindowedValues(transform.getInput()); + context.setPCollectionView(transform.getOutput(), elems); + } + }); + } + } + + private static class SingletonPCollectionView + extends PCollectionViewBase> { + + public SingletonPCollectionView(Pipeline pipeline) { + setPipelineInternal(pipeline); + } + + @Override + public T fromIterableInternal(Iterable> contents) { + try { + return (T) Iterables.getOnlyElement(contents).getValue(); + } catch (NoSuchElementException exc) { + throw new NoSuchElementException( + "Empty PCollection accessed as a singleton view."); + } catch (IllegalArgumentException exc) { + throw new IllegalArgumentException( + "PCollection with more than one element " + + "accessed as a singleton view."); + } + } + } + + private static class IterablePCollectionView + extends PCollectionViewBase, Iterable>> { + + public IterablePCollectionView(Pipeline pipeline) { + setPipelineInternal(pipeline); + } + + @Override + public Iterable fromIterableInternal(Iterable> contents) { + return Iterables.transform(contents, new Function, T>() { + @Override + public T apply(WindowedValue input) { + return (T) input.getValue(); + } + }); + } + } + + private abstract static class PCollectionViewBase + extends PValueBase + implements PCollectionView { + + @Override + public TupleTag>> getTagInternal() { + return tag; + } + + private TupleTag>> tag = new TupleTag<>(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java new file mode 100644 index 0000000000000..1754c20a7916b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java @@ -0,0 +1,116 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.reflect.TypeToken; + +/** + * {@code WithKeys} takes a {@code PCollection}, and either a + * constant key of type {@code K} or a function from {@code V} to + * {@code K}, and returns a {@code PCollection>}, where each + * of the values in the input {@code PCollection} has been paired with + * either the constant key or a key computed from the value. + * + *

Example of use: + *

 {@code
+ * PCollection words = ...;
+ * PCollection> lengthsToWords =
+ *     words.apply(WithKeys.of(new SerializableFunction() {
+ *         public Integer apply(String s) { return s.length(); } }));
+ * } 
+ * + *

Each output element has the same timestamp and is in the same windows + * as its corresponding input element, and the output {@code PCollection} + * has the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * associated with it as the input. + * + * @param the type of the keys in the output {@code PCollection} + * @param the type of the elements in the input + * {@code PCollection} and the values in the output + * {@code PCollection} + */ +public class WithKeys extends PTransform, + PCollection>> { + /** + * Returns a {@code PTransform} that takes a {@code PCollection} + * and returns a {@code PCollection>}, where each of the + * values in the input {@code PCollection} has been paired with a + * key computed from the value by invoking the given + * {@code SerializableFunction}. + */ + public static WithKeys of(SerializableFunction fn) { + return new WithKeys<>(fn, null); + } + + /** + * Returns a {@code PTransform} that takes a {@code PCollection} + * and returns a {@code PCollection>}, where each of the + * values in the input {@code PCollection} has been paired with the + * given key. + */ + @SuppressWarnings("unchecked") + public static WithKeys of(final K key) { + return new WithKeys<>( + new SerializableFunction() { + @Override + public K apply(V value) { + return key; + } + }, + (Class) (key == null ? null : key.getClass())); + } + + + ///////////////////////////////////////////////////////////////////////////// + + private SerializableFunction fn; + private transient Class keyClass; + + private WithKeys(SerializableFunction fn, Class keyClass) { + this.fn = fn; + this.keyClass = keyClass; + } + + @Override + public PCollection> apply(PCollection in) { + Coder keyCoder; + if (keyClass == null) { + keyCoder = getCoderRegistry().getDefaultOutputCoder(fn, in.getCoder()); + } else { + keyCoder = getCoderRegistry().getDefaultCoder(TypeToken.of(keyClass)); + } + PCollection> result = + in.apply(ParDo.named("AddKeys") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + c.output(KV.of(fn.apply(c.element()), + c.element())); + } + })); + if (keyCoder != null) { + // TODO: Remove when we can set the coder inference context. + result.setCoder(KvCoder.of(keyCoder, in.getCoder())); + } + return result; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java new file mode 100644 index 0000000000000..f91d7d2ca669c --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java @@ -0,0 +1,367 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +import static com.google.cloud.dataflow.sdk.util.Structs.addObject; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.ListCoder; +import com.google.cloud.dataflow.sdk.coders.MapCoder; +import com.google.cloud.dataflow.sdk.coders.StandardCoder; +import com.google.cloud.dataflow.sdk.coders.VarIntCoder; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TupleTagList; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +/** + * A row result of a CoGroupByKey. This is a tuple of Iterables produced for + * a given key, and these can be accessed in different ways. + */ +public class CoGbkResult { + // TODO: If we keep this representation for any amount of time, + // optimize it so that the union tag does not have to be repeated in the + // values stored under the union tag key. + /** + * A map of integer union tags to a list of union objects. + * Note: the key and the embedded union tag are the same, so it is redundant + * to store it multiple times, but for now it makes encoding easier. + */ + private final Map> valueMap; + + private final CoGbkResultSchema schema; + + /** + * A row in the PCollection resulting from a CoGroupByKey transform. + * Currently, this row must fit into memory. + * + * @param schema the set of tuple tags used to refer to input tables and + * result values + * @param values the raw results from a group-by-key + */ + @SuppressWarnings("unchecked") + public CoGbkResult( + CoGbkResultSchema schema, + Iterable values) { + this.schema = schema; + valueMap = new TreeMap<>(); + for (RawUnionValue value : values) { + // Make sure the given union tag has a corresponding tuple tag in the + // schema. + int unionTag = value.getUnionTag(); + if (schema.size() <= unionTag) { + throw new IllegalStateException("union tag " + unionTag + + " has no corresponding tuple tag in the result schema"); + } + List taggedValueList = valueMap.get(unionTag); + if (taggedValueList == null) { + taggedValueList = new ArrayList<>(); + valueMap.put(unionTag, taggedValueList); + } + taggedValueList.add(value); + } + } + + public boolean isEmpty() { + return valueMap == null || valueMap.isEmpty(); + } + + /** + * Returns the schema used by this CoGbkResult. + */ + public CoGbkResultSchema getSchema() { + return schema; + } + + @Override + public String toString() { + return valueMap.toString(); + } + + /** + * Returns the values from the table represented by the given + * {@code TupleTag} as an {@code Iterable} (which may be empty if there + * are no results). + */ + public Iterable getAll(TupleTag tag) { + int index = schema.getIndex(tag); + if (index < 0) { + throw new IllegalArgumentException("TupleTag " + tag + + " is not in the schema"); + } + List unions = valueMap.get(index); + if (unions == null) { + return buildEmptyIterable(tag); + } + return new UnionValueIterable<>(unions); + } + + /** + * If there is a singleton value for the given tag, returns it. + * Otherwise, throws an IllegalArgumentException. + */ + public V getOnly(TupleTag tag) { + return innerGetOnly(tag, null, false); + } + + /** + * If there is a singleton value for the given tag, returns it. If there is + * no value for the given tag, returns the defaultValue. + * Otherwise, throws an IllegalArgumentException. + */ + public V getOnly(TupleTag tag, V defaultValue) { + return innerGetOnly(tag, defaultValue, true); + } + + /** + * A coder for CoGbkResults. + */ + public static class CoGbkResultCoder extends StandardCoder { + + private final CoGbkResultSchema schema; + private final MapCoder> mapCoder; + + /** + * Returns a CoGbkResultCoder for the given schema and unionCoder. + */ + public static CoGbkResultCoder of( + CoGbkResultSchema schema, + UnionCoder unionCoder) { + return new CoGbkResultCoder(schema, unionCoder); + } + + @JsonCreator + public static CoGbkResultCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> components, + @JsonProperty(PropertyNames.CO_GBK_RESULT_SCHEMA) CoGbkResultSchema schema) { + Preconditions.checkArgument(components.size() == 1, + "Expecting 1 component, got " + components.size()); + return new CoGbkResultCoder(schema, (MapCoder) components.get(0)); + } + + private CoGbkResultCoder( + CoGbkResultSchema tupleTags, + UnionCoder unionCoder) { + this.schema = tupleTags; + this.mapCoder = MapCoder.of(VarIntCoder.of(), + ListCoder.of(unionCoder)); + } + + private CoGbkResultCoder( + CoGbkResultSchema tupleTags, + MapCoder mapCoder) { + this.schema = tupleTags; + this.mapCoder = mapCoder; + } + + + @Override + public List> getCoderArguments() { + return null; + } + + @Override + public List> getComponents() { + return Arrays.>asList(mapCoder); + } + + @Override + public CloudObject asCloudObject() { + CloudObject result = super.asCloudObject(); + addObject(result, PropertyNames.CO_GBK_RESULT_SCHEMA, schema.asCloudObject()); + return result; + } + + @Override + public void encode( + CoGbkResult value, + OutputStream outStream, + Context context) throws CoderException, + IOException { + if (!schema.equals(value.getSchema())) { + throw new CoderException("input schema does not match coder schema"); + } + mapCoder.encode(value.valueMap, outStream, context); + } + + @Override + public CoGbkResult decode( + InputStream inStream, + Context context) + throws CoderException, IOException { + Map> map = mapCoder.decode( + inStream, context); + return new CoGbkResult(schema, map); + } + + public boolean equals(Object other) { + if (!super.equals(other)) { + return false; + } + return schema.equals(((CoGbkResultCoder) other).schema); + } + + @Override + public boolean isDeterministic() { + return mapCoder.isDeterministic(); + } + } + + + ////////////////////////////////////////////////////////////////////////////// + // Methods for testing purposes + + /** + * Returns a new CoGbkResult that contains just the given tag the given data. + */ + public static CoGbkResult of(TupleTag tag, List data) { + return CoGbkResult.empty().and(tag, data); + } + + /** + * Returns a new CoGbkResult based on this, with the given tag and given data + * added to it. + */ + public CoGbkResult and(TupleTag tag, List data) { + if (nextTestUnionId != schema.size()) { + throw new IllegalArgumentException( + "Attempting to call and() on a CoGbkResult apparently not created by" + + " of()."); + } + Map> valueMap = new TreeMap<>(this.valueMap); + valueMap.put(nextTestUnionId, + convertValueListToUnionList(nextTestUnionId, data)); + return new CoGbkResult( + new CoGbkResultSchema(schema.getTupleTagList().and(tag)), valueMap, + nextTestUnionId + 1); + } + + /** + * Returns an empty CoGbkResult. + */ + public static CoGbkResult empty() { + return new CoGbkResult(new CoGbkResultSchema(TupleTagList.empty()), + new TreeMap>()); + } + + ////////////////////////////////////////////////////////////////////////////// + + private int nextTestUnionId = 0; + + private CoGbkResult( + CoGbkResultSchema schema, + Map> valueMap, + int nextTestUnionId) { + this(schema, valueMap); + this.nextTestUnionId = nextTestUnionId; + } + + private CoGbkResult( + CoGbkResultSchema schema, + Map> valueMap) { + this.schema = schema; + this.valueMap = valueMap; + } + + private static List convertValueListToUnionList( + int unionTag, List data) { + List unionList = new ArrayList<>(); + for (V value : data) { + unionList.add(new RawUnionValue(unionTag, value)); + } + return unionList; + } + + private Iterable buildEmptyIterable(TupleTag tag) { + return new ArrayList<>(); + } + + private V innerGetOnly( + TupleTag tag, + V defaultValue, + boolean useDefault) { + int index = schema.getIndex(tag); + if (index < 0) { + throw new IllegalArgumentException("TupleTag " + tag + + " is not in the schema"); + } + List unions = valueMap.get(index); + if (unions.isEmpty()) { + if (useDefault) { + return defaultValue; + } else { + throw new IllegalArgumentException("TupleTag " + tag + + " corresponds to an empty result, and no default was provided"); + } + } + if (unions.size() != 1) { + throw new IllegalArgumentException("TupleTag " + tag + + " corresponds to a non-singleton result of size " + unions.size()); + } + return (V) unions.get(0).getValue(); + } + + /** + * Lazily converts and recasts an {@code Iterable} into an + * {@code Iterable}, where V is the type of the raw union value's contents. + */ + private static class UnionValueIterable implements Iterable { + + private final Iterable unions; + + private UnionValueIterable(Iterable unions) { + this.unions = unions; + } + + @Override + public Iterator iterator() { + final Iterator unionsIterator = unions.iterator(); + return new Iterator() { + @Override + public boolean hasNext() { + return unionsIterator.hasNext(); + } + + @Override + public V next() { + return (V) unionsIterator.next().getValue(); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultSchema.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultSchema.java new file mode 100644 index 0000000000000..93883b80750c8 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultSchema.java @@ -0,0 +1,133 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +import static com.google.cloud.dataflow.sdk.util.Structs.addList; + +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TupleTagList; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +/** + * A schema for the results of a CoGroupByKey. This maintains the full + * set of TupleTags for the results of a CoGroupByKey, and facilitates mapping + * between TupleTags and Union Tags (which are used as secondary keys in the + * CoGroupByKey). + */ +class CoGbkResultSchema implements Serializable { + + private final TupleTagList tupleTagList; + + @JsonCreator + public static CoGbkResultSchema of( + @JsonProperty(PropertyNames.TUPLE_TAGS) List> tags) { + TupleTagList tupleTags = TupleTagList.empty(); + for (TupleTag tag : tags) { + tupleTags = tupleTags.and(tag); + } + return new CoGbkResultSchema(tupleTags); + } + + /** + * Maps TupleTags to union tags. This avoids needing to encode the tags + * themselves. + */ + private final HashMap, Integer> tagMap = new HashMap<>(); + + /** + * Builds a schema from a tuple of {@code TupleTag}s. + */ + public CoGbkResultSchema(TupleTagList tupleTagList) { + this.tupleTagList = tupleTagList; + int index = -1; + for (TupleTag tag : tupleTagList.getAll()) { + index++; + tagMap.put(tag, index); + } + } + + /** + * Returns the index for the given tuple tag, if the tag is present in this + * schema, -1 if it isn't. + */ + public int getIndex(TupleTag tag) { + Integer index = tagMap.get(tag); + return index == null ? -1 : index; + } + + /** + * Returns the JoinTupleTag at the given index. + */ + public TupleTag getTag(int index) { + return tupleTagList.get(index); + } + + /** + * Returns the number of columms for this schema. + */ + public int size() { + return tupleTagList.getAll().size(); + } + + /** + * Returns the TupleTagList tuple associated with this schema. + */ + public TupleTagList getTupleTagList() { + return tupleTagList; + } + + public CloudObject asCloudObject() { + CloudObject result = CloudObject.forClass(getClass()); + List serializedTags = new ArrayList<>(tupleTagList.size()); + for (TupleTag tag : tupleTagList.getAll()) { + serializedTags.add(tag.asCloudObject()); + } + addList(result, PropertyNames.TUPLE_TAGS, serializedTags); + return result; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof CoGbkResultSchema)) { + return false; + } + CoGbkResultSchema other = (CoGbkResultSchema) obj; + return tupleTagList.getAll().equals(other.tupleTagList.getAll()); + } + + @Override + public int hashCode() { + return tupleTagList.getAll().hashCode(); + } + + @Override + public String toString() { + return "CoGbkResultSchema: " + tupleTagList.getAll(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKey.java new file mode 100644 index 0000000000000..d81c9ef707ca7 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKey.java @@ -0,0 +1,208 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Flatten; +import com.google.cloud.dataflow.sdk.transforms.GroupByKey; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult.CoGbkResultCoder; +import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple.TaggedKeyedPCollection; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; + +import java.util.ArrayList; +import java.util.List; + +/** + * A transform that performs a CoGroupByKey on a tuple of tables. A + * CoGroupByKey groups results from all tables by like keys into CoGbkResults, + * from which the results for any specific table can be accessed by the + * TupleTag supplied with the initial table. + * + *

Example of performing a CoGroupByKey followed by a ParDo that consumes + * the results: + *

 
+ * {@literal PCollection>} pt1 = ...;
+ * {@literal PCollection>} pt2 = ...;
+ *
+ * final {@literal TupleTag} t1 = new {@literal TupleTag<>()};
+ * final {@literal TupleTag} t2 = new {@literal TupleTag<>()};
+ * {@literal PCollection>} coGbkResultCollection =
+ *   KeyedPCollectionTuple.of(t1, pt1)
+ *                        .and(t2, pt2)
+ *                        .apply({@literal CoGroupByKey.create()});
+ *
+ * {@literal PCollection} finalResultCollection =
+ *   coGbkResultCollection.apply(ParDo.of(
+ *     new {@literal DoFn, T>()} {
+ *       {@literal @}Override
+ *       public void processElement(ProcessContext c) {
+ *         {@literal KV} e = c.element();
+ *         {@literal Iterable} pt1Vals = e.getValue().getAll(t1);
+ *         V2 pt2Val = e.getValue().getOnly(t2);
+ *          ... Do Something ....
+ *         c.output(...some T...);
+ *       }
+ *     }));
+ *  
+ * + * @param the type of the keys in the input and output + * {@code PCollection}s + */ +public class CoGroupByKey extends + PTransform, + PCollection>> { + /** + * Returns a {@code CoGroupByKey} {@code PTransform}. + * + * @param the type of the keys in the input and output + * {@code PCollection}s + */ + public static CoGroupByKey create() { + return new CoGroupByKey<>(); + } + + private CoGroupByKey() { } + + @Override + public PCollection> apply( + KeyedPCollectionTuple input) { + if (input.isEmpty()) { + throw new IllegalArgumentException( + "must have at least one input to a KeyedPCollections"); + } + + // First build the union coder. + // TODO: Look at better integration of union types with the + // schema specified in the input. + List> codersList = new ArrayList<>(); + for (TaggedKeyedPCollection entry : input.getKeyedCollections()) { + codersList.add(getValueCoder(entry.pCollection)); + } + UnionCoder unionCoder = UnionCoder.of(codersList); + Coder keyCoder = input.getKeyCoder(); + KvCoder kVCoder = + KvCoder.of(keyCoder, unionCoder); + + PCollectionList> unionTables = + PCollectionList.empty(getPipeline()); + + // TODO: Use the schema to order the indices rather than depending + // on the fact that the schema ordering is identical to the ordering from + // input.getJoinCollections(). + int index = -1; + for (TaggedKeyedPCollection entry : input.getKeyedCollections()) { + index++; + PCollection> unionTable = + makeUnionTable(index, entry.pCollection, kVCoder); + unionTables = unionTables.and(unionTable); + } + + PCollection> flattenedTable = + unionTables.apply(Flatten.>create()); + + PCollection>> groupedTable = + flattenedTable.apply(GroupByKey.create()); + + CoGbkResultSchema tupleTags = input.getCoGbkResultSchema(); + PCollection> result = groupedTable.apply( + ParDo.of(new ConstructCoGbkResultFn(tupleTags)) + .named("ConstructCoGbkResultFn")); + result.setCoder(KvCoder.of(keyCoder, + CoGbkResultCoder.of(tupleTags, unionCoder))); + + return result; + } + + ////////////////////////////////////////////////////////////////////////////// + + /** + * Returns the value coder for the given PCollection. Assumes that the value + * coder is an instance of {@code KvCoder}. + */ + private Coder getValueCoder(PCollection> pCollection) { + // Assumes that the PCollection uses a KvCoder. + Coder entryCoder = pCollection.getCoder(); + if (!(entryCoder instanceof KvCoder)) { + throw new IllegalArgumentException("PCollection does not use a KvCoder"); + } + @SuppressWarnings("unchecked") + KvCoder coder = (KvCoder) entryCoder; + return coder.getValueCoder(); + } + + /** + * Returns a UnionTable for the given input PCollection, using the given + * union index and the given unionTableEncoder. + */ + private PCollection> makeUnionTable( + final int index, + PCollection> pCollection, + KvCoder unionTableEncoder) { + + return pCollection.apply(ParDo.of( + new ConstructUnionTableFn(index)).named("MakeUnionTable")) + .setCoder(unionTableEncoder); + } + + /** + * A DoFn to construct a UnionTable (i.e., a + * {@code PCollection>} from a + * {@code PCollection>}. + */ + private static class ConstructUnionTableFn extends + DoFn, KV> { + + private final int index; + + public ConstructUnionTableFn(int index) { + this.index = index; + } + + @Override + public void processElement(ProcessContext c) { + KV e = c.element(); + c.output(KV.of(e.getKey(), new RawUnionValue(index, e.getValue()))); + } + } + + /** + * A DoFn to construct a CoGbkResult from an input grouped union + * table. + */ + private static class ConstructCoGbkResultFn + extends DoFn>, + KV> { + + private final CoGbkResultSchema schema; + + public ConstructCoGbkResultFn(CoGbkResultSchema schema) { + this.schema = schema; + } + + @Override + public void processElement(ProcessContext c) { + KV> e = c.element(); + c.output(KV.of(e.getKey(), new CoGbkResult(schema, e.getValue()))); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/KeyedPCollectionTuple.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/KeyedPCollectionTuple.java new file mode 100644 index 0000000000000..a9fd4b684f85b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/KeyedPCollectionTuple.java @@ -0,0 +1,217 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.POutput; +import com.google.cloud.dataflow.sdk.values.PValue; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TupleTagList; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +/** + * Represents an immutable tuple of keyed PCollections (i.e. PCollections of + * {@code KV}), with key type K. + * + * @param the type of key shared by all constituent PCollections + */ +public class KeyedPCollectionTuple implements PInput { + /** + * Returns an empty {@code KeyedPCollections} on the given pipeline. + */ + public static KeyedPCollectionTuple empty(Pipeline pipeline) { + return new KeyedPCollectionTuple<>(pipeline); + } + + /** + * Returns a new {@code KeyedPCollections} with the given tag and initial + * PCollection. + */ + public static KeyedPCollectionTuple of( + TupleTag tag, + PCollection> pc) { + return new KeyedPCollectionTuple(pc.getPipeline()).and(tag, pc); + } + + /** + * Returns a new {@code KeyedPCollections} that is the same as this, + * appended with the given PCollection. + */ + public KeyedPCollectionTuple and( + TupleTag< V> tag, + PCollection> pc) { + if (pc.getPipeline() != getPipeline()) { + throw new IllegalArgumentException( + "PCollections come from different Pipelines"); + } + TaggedKeyedPCollection wrapper = + new TaggedKeyedPCollection<>(tag, pc); + Coder myKeyCoder = keyCoder == null ? getKeyCoder(pc) : keyCoder; + List> + newKeyedCollections = + copyAddLast( + keyedCollections, + wrapper); + return new KeyedPCollectionTuple<>( + getPipeline(), + newKeyedCollections, + schema.getTupleTagList().and(tag), + myKeyCoder); + } + + public boolean isEmpty() { + return keyedCollections.isEmpty(); + } + + /** + * Returns a list of TaggedKeyedPCollections for the PCollections contained in + * this {@code KeyedPCollections}. + */ + public List> getKeyedCollections() { + return keyedCollections; + } + + /** + * Applies the given transform to this input. + */ + public O apply( + PTransform, O> transform) { + return Pipeline.applyTransform(this, transform); + } + + /** + * Expands the component PCollections, stripping off any tag-specific + * information. + */ + @Override + public Collection expand() { + List> retval = new ArrayList<>(); + for (TaggedKeyedPCollection taggedPCollection : keyedCollections) { + retval.add(taggedPCollection.pCollection); + } + return retval; + } + + /** + * Returns the KeyCoder for all PCollections in this KeyedPCollections. + */ + public Coder getKeyCoder() { + if (keyCoder == null) { + throw new IllegalStateException("cannot return null keyCoder"); + } + return keyCoder; + } + + /** + * Returns the CoGbkResultSchema associated with this + * KeyedPCollections. + */ + public CoGbkResultSchema getCoGbkResultSchema() { + return schema; + } + + @Override + public Pipeline getPipeline() { + return pipeline; + } + + @Override + public void finishSpecifying() { + for (TaggedKeyedPCollection taggedPCollection : keyedCollections) { + taggedPCollection.pCollection.finishSpecifying(); + } + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * A utility class to help ensure coherence of tag and input PCollection + * types. + */ + static class TaggedKeyedPCollection { + final TupleTag tupleTag; + final PCollection> pCollection; + + public TaggedKeyedPCollection( + TupleTag tupleTag, + PCollection> pCollection) { + this.tupleTag = tupleTag; + this.pCollection = pCollection; + } + } + + /** + * We use a List to properly track the order in which collections are added. + */ + private final List> keyedCollections; + + private final Coder keyCoder; + + private final CoGbkResultSchema schema; + + private final Pipeline pipeline; + + KeyedPCollectionTuple(Pipeline pipeline) { + this(pipeline, + new ArrayList>(), + TupleTagList.empty(), + null); + } + + KeyedPCollectionTuple( + Pipeline pipeline, + List> keyedCollections, + TupleTagList tupleTagList, + Coder keyCoder) { + this.pipeline = pipeline; + this.keyedCollections = keyedCollections; + this.schema = new CoGbkResultSchema(tupleTagList); + this.keyCoder = keyCoder; + } + + private static Coder getKeyCoder(PCollection> pc) { + // Need to run coder inference on this PCollection before inspecting it. + pc.finishSpecifying(); + + // Assumes that the PCollection uses a KvCoder. + Coder entryCoder = pc.getCoder(); + if (!(entryCoder instanceof KvCoder)) { + throw new IllegalArgumentException("PCollection does not use a KvCoder"); + } + @SuppressWarnings("unchecked") + KvCoder coder = (KvCoder) entryCoder; + return coder.getKeyCoder(); + } + + private static List> copyAddLast( + List> keyedCollections, + TaggedKeyedPCollection taggedCollection) { + List> retval = + new ArrayList<>(keyedCollections); + retval.add(taggedCollection); + return retval; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/RawUnionValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/RawUnionValue.java new file mode 100644 index 0000000000000..b52f8b3e49c2f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/RawUnionValue.java @@ -0,0 +1,51 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +// TODO: Think about making this a complete dynamic union by adding +// a schema. Type would then be defined by the corresponding schema entry. + +/** + * This corresponds to an integer union tag and value. The mapping of + * union tag to type must come from elsewhere. + */ +class RawUnionValue { + private final int unionTag; + private final Object value; + + /** + * Constructs a partial union from the given union tag and value. + */ + public RawUnionValue(int unionTag, Object value) { + this.unionTag = unionTag; + this.value = value; + } + + public int getUnionTag() { + return unionTag; + } + + public Object getValue() { + return value; + } + + @Override + public String toString() { + return unionTag + ":" + value; + } +} + diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java new file mode 100644 index 0000000000000..a6bb4bcb45860 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java @@ -0,0 +1,149 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.StandardCoder; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.VarInt; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; + +/** + * A UnionCoder encodes RawUnionValues. + */ +class UnionCoder extends StandardCoder { + // TODO: Think about how to integrate this with a schema object (i.e. + // a tuple of tuple tags). + /** + * Builds a union coder with the given list of element coders. This list + * corresponds to a mapping of union tag to Coder. Union tags start at 0. + */ + public static UnionCoder of(List> elementCoders) { + return new UnionCoder(elementCoders); + } + + @JsonCreator + public static UnionCoder jsonOf( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> elements) { + return UnionCoder.of(elements); + } + + private int getIndexForEncoding(RawUnionValue union) { + if (union == null) { + throw new IllegalArgumentException("cannot encode a null tagged union"); + } + int index = union.getUnionTag(); + if (index < 0 || index >= elementCoders.size()) { + throw new IllegalArgumentException( + "union value index " + index + " not in range [0.." + + (elementCoders.size() - 1) + "]"); + } + return index; + } + + @SuppressWarnings("unchecked") + @Override + public void encode( + RawUnionValue union, + OutputStream outStream, + Context context) + throws IOException, CoderException { + int index = getIndexForEncoding(union); + // Write out the union tag. + VarInt.encode(index, outStream); + + // Write out the actual value. + Coder coder = (Coder) elementCoders.get(index); + coder.encode( + union.getValue(), + outStream, + context); + } + + @Override + public RawUnionValue decode(InputStream inStream, Context context) + throws IOException, CoderException { + int index = VarInt.decodeInt(inStream); + Object value = elementCoders.get(index).decode(inStream, context); + return new RawUnionValue(index, value); + } + + @Override + public List> getCoderArguments() { + return null; + } + + @Override + public List> getComponents() { + return elementCoders; + } + + /** + * Since this coder uses elementCoders.get(index) and coders that are known to run in constant + * time, we defer the return value to that coder. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(RawUnionValue union, Context context) { + int index = getIndexForEncoding(union); + Coder coder = (Coder) elementCoders.get(index); + return coder.isRegisterByteSizeObserverCheap(union.getValue(), context); + } + + /** + * Notifies ElementByteSizeObserver about the byte size of the encoded value using this coder. + */ + @Override + public void registerByteSizeObserver( + RawUnionValue union, ElementByteSizeObserver observer, Context context) + throws Exception { + int index = getIndexForEncoding(union); + // Write out the union tag. + observer.update(VarInt.getLength(index)); + // Write out the actual value. + Coder coder = (Coder) elementCoders.get(index); + coder.registerByteSizeObserver(union.getValue(), observer, context); + } + + ///////////////////////////////////////////////////////////////////////////// + + private final List> elementCoders; + + private UnionCoder(List> elementCoders) { + this.elementCoders = elementCoders; + } + + @Override + public boolean isDeterministic() { + for (Coder elementCoder : elementCoders) { + if (!elementCoder.isDeterministic()) { + return false; + } + } + + return true; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/package-info.java new file mode 100644 index 0000000000000..ba907ac2cd734 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/package-info.java @@ -0,0 +1,21 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Defines the {@link com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey} transform + * for joining multiple PCollections. + */ +package com.google.cloud.dataflow.sdk.transforms.join; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/package-info.java new file mode 100644 index 0000000000000..b72e90e780ac0 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/package-info.java @@ -0,0 +1,43 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Defines {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s for transforming + * data in a pipeline. + * + *

A {@link com.google.cloud.dataflow.sdk.transforms.PTransform} is an operation that takes an + * {@code Input} (some subtype of {@link com.google.cloud.dataflow.sdk.values.PInput}) + * and produces an + * {@code Output} (some subtype of {@link com.google.cloud.dataflow.sdk.values.POutput}). + * + *

Common PTransforms include root PTransforms like + * {@link com.google.cloud.dataflow.sdk.io.TextIO.Read} and + * {@link com.google.cloud.dataflow.sdk.transforms.Create}, processing and + * conversion operations like {@link com.google.cloud.dataflow.sdk.transforms.ParDo}, + * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}, + * {@link com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey}, + * {@link com.google.cloud.dataflow.sdk.transforms.Combine}, and + * {@link com.google.cloud.dataflow.sdk.transforms.Count}, and outputting + * PTransforms like + * {@link com.google.cloud.dataflow.sdk.io.TextIO.Write}. + * + *

New PTransforms can be created by composing existing PTransforms. + * Most PTransforms in this package are composites, and users can also create composite PTransforms + * for their own application-specific logic. + * + */ +package com.google.cloud.dataflow.sdk.transforms; + diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/BoundedWindow.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/BoundedWindow.java new file mode 100644 index 0000000000000..01de83f1585d0 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/BoundedWindow.java @@ -0,0 +1,37 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import org.joda.time.Instant; + +/** + * A {@code BoundedWindow} represents a finite grouping of elements, with an + * upper bound (larger timestamps represent more recent data) on the timestamps + * of elements that can be placed in the window. This finiteness means that for + * every window, at some point in time, all data for that window will have + * arrived and can be processed together. + * + *

Windows must also implement {@link Object#equals} and + * {@link Object#hashCode} such that windows that are logically equal will + * be treated as equal by {@code equals()} and {@code hashCode()}. + */ +public abstract class BoundedWindow { + /** + * Returns the upper bound of timestamps for values in this window. + */ + public abstract Instant maxTimestamp(); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java new file mode 100644 index 0000000000000..bb0de796f86ab --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java @@ -0,0 +1,300 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import com.google.cloud.dataflow.sdk.coders.Coder; + +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.Days; +import org.joda.time.Instant; +import org.joda.time.Months; +import org.joda.time.Years; + +/** + * A collection of {@link WindowingFn}s that windows values into calendar-based + * windows such as spans of days, months, or years. + * + *

For example, to group data into quarters that change on the 15th, use + * {@code CalendarWindows.months(3).withStartingMonth(2014, 1).beginningOnDay(15)}. + */ +public class CalendarWindows { + + /** + * Returns a {@link WindowingFn} that windows elements into periods measured by days. + * + *

For example, {@code CalendarWindows.days(1)} will window elements into + * separate windows for each day. + */ + public static DaysWindows days(int number) { + return new DaysWindows(number, new DateTime(0, DateTimeZone.UTC), DateTimeZone.UTC); + } + + /** + * Returns a {@link WindowingFn} that windows elements into periods measured by weeks. + * + *

For example, {@code CalendarWindows.weeks(1, DateTimeConstants.TUESDAY)} will + * window elements into week-long windows starting on Tuesdays. + */ + public static DaysWindows weeks(int number, int startDayOfWeek) { + return new DaysWindows( + 7 * number, + new DateTime(0, DateTimeZone.UTC).withDayOfWeek(startDayOfWeek), + DateTimeZone.UTC); + } + + /** + * Returns a {@link WindowingFn} that windows elements into periods measured by months. + * + *

For example, + * {@code CalendarWindows.months(8).withStartingMonth(2014, 1).beginningOnDay(10)} + * will window elements into 8 month windows where that start on the 10th day of month, + * and the first window begins in January 2014. + */ + public static MonthsWindows months(int number) { + return new MonthsWindows(number, 1, new DateTime(0, DateTimeZone.UTC), DateTimeZone.UTC); + } + + /** + * Returns a {@link WindowingFn} that windows elements into periods measured by years. + * + *

For example, + * {@code CalendarWindows.years(1).withTimeZone(DateTimeZone.forId("America/Los_Angeles"))} + * will window elements into year-long windows that start at midnight on Jan 1, in the + * America/Los_Angeles time zone. + */ + public static YearsWindows years(int number) { + return new YearsWindows(number, 1, 1, new DateTime(0, DateTimeZone.UTC), DateTimeZone.UTC); + } + + /** + * A {@link WindowingFn} that windows elements into periods measured by days. + * + *

By default, periods of multiple days are measured starting at the + * epoch. This can be overridden with {@link #withStartingDay}. + * + *

The time zone used to determine calendar boundaries is UTC, unless this + * is overridden with the {@link #withTimeZone} method. + */ + public static class DaysWindows extends PartitioningWindowingFn { + + public DaysWindows withStartingDay(int year, int month, int day) { + return new DaysWindows( + number, new DateTime(year, month, day, 0, 0, timeZone), timeZone); + } + + public DaysWindows withTimeZone(DateTimeZone timeZone) { + return new DaysWindows( + number, startDate.withZoneRetainFields(timeZone), timeZone); + } + + //////////////////////////////////////////////////////////////////////////// + + private int number; + private DateTime startDate; + private DateTimeZone timeZone; + + private DaysWindows(int number, DateTime startDate, DateTimeZone timeZone) { + this.number = number; + this.startDate = startDate; + this.timeZone = timeZone; + } + + @Override + public IntervalWindow assignWindow(Instant timestamp) { + DateTime datetime = new DateTime(timestamp, timeZone); + + int dayOffset = Days.daysBetween(startDate, datetime).getDays() / number * number; + + DateTime begin = startDate.plusDays(dayOffset); + DateTime end = begin.plusDays(number); + + return new IntervalWindow(begin.toInstant(), end.toInstant()); + } + + @Override + public Coder windowCoder() { + return IntervalWindow.getCoder(); + } + + @Override + public boolean isCompatible(WindowingFn other) { + if (!(other instanceof DaysWindows)) { + return false; + } + DaysWindows that = (DaysWindows) other; + return number == that.number + && startDate == that.startDate + && timeZone == that.timeZone; + } + } + + /** + * A {@link WindowingFn} that windows elements into periods measured by months. + * + *

By default, periods of multiple months are measured starting at the + * epoch. This can be overridden with {@link #withStartingMonth}. + * + *

Months start on the first day of each calendar month, unless overridden by + * {@link #beginningOnDay}. + * + *

The time zone used to determine calendar boundaries is UTC, unless this + * is overridden with the {@link #withTimeZone} method. + */ + public static class MonthsWindows extends PartitioningWindowingFn { + + public MonthsWindows beginningOnDay(int dayOfMonth) { + return new MonthsWindows( + number, dayOfMonth, startDate, timeZone); + } + + public MonthsWindows withStartingMonth(int year, int month) { + return new MonthsWindows( + number, dayOfMonth, new DateTime(year, month, 1, 0, 0, timeZone), timeZone); + } + + public MonthsWindows withTimeZone(DateTimeZone timeZone) { + return new MonthsWindows( + number, dayOfMonth, startDate.withZoneRetainFields(timeZone), timeZone); + } + + //////////////////////////////////////////////////////////////////////////// + + private int number; + private int dayOfMonth; + private DateTime startDate; + private DateTimeZone timeZone; + + private MonthsWindows(int number, int dayOfMonth, DateTime startDate, DateTimeZone timeZone) { + this.number = number; + this.dayOfMonth = dayOfMonth; + this.startDate = startDate; + this.timeZone = timeZone; + } + + @Override + public IntervalWindow assignWindow(Instant timestamp) { + DateTime datetime = new DateTime(timestamp, timeZone); + + int monthOffset = + Months.monthsBetween(startDate.withDayOfMonth(dayOfMonth), datetime).getMonths() + / number * number; + + DateTime begin = startDate.withDayOfMonth(dayOfMonth).plusMonths(monthOffset); + DateTime end = begin.plusMonths(number); + + return new IntervalWindow(begin.toInstant(), end.toInstant()); + } + + @Override + public Coder windowCoder() { + return IntervalWindow.getCoder(); + } + + @Override + public boolean isCompatible(WindowingFn other) { + if (!(other instanceof MonthsWindows)) { + return false; + } + MonthsWindows that = (MonthsWindows) other; + return number == that.number + && dayOfMonth == dayOfMonth + && startDate == that.startDate + && timeZone == that.timeZone; + } + } + + /** + * A {@link WindowingFn} that windows elements into periods measured by years. + * + *

By default, periods of multiple years are measured starting at the + * epoch. This can be overridden with {@link #withStartingYear}. + * + *

Years start on the first day of each calendar year, unless overridden by + * {@link #beginningOnDay}. + * + *

The time zone used to determine calendar boundaries is UTC, unless this + * is overridden with the {@link #withTimeZone} method. + */ + public static class YearsWindows extends PartitioningWindowingFn { + + public YearsWindows beginningOnDay(int monthOfYear, int dayOfMonth) { + return new YearsWindows( + number, monthOfYear, dayOfMonth, startDate, timeZone); + } + + public YearsWindows withStartingYear(int year) { + return new YearsWindows( + number, monthOfYear, dayOfMonth, new DateTime(year, 1, 1, 0, 0, timeZone), timeZone); + } + + public YearsWindows withTimeZone(DateTimeZone timeZone) { + return new YearsWindows( + number, monthOfYear, dayOfMonth, startDate.withZoneRetainFields(timeZone), timeZone); + } + + //////////////////////////////////////////////////////////////////////////// + + private int number; + private int monthOfYear; + private int dayOfMonth; + private DateTime startDate; + private DateTimeZone timeZone; + + private YearsWindows( + int number, int monthOfYear, int dayOfMonth, DateTime startDate, DateTimeZone timeZone) { + this.number = number; + this.monthOfYear = monthOfYear; + this.dayOfMonth = dayOfMonth; + this.startDate = startDate; + this.timeZone = timeZone; + } + + @Override + public IntervalWindow assignWindow(Instant timestamp) { + DateTime datetime = new DateTime(timestamp, timeZone); + + DateTime offsetStart = startDate.withMonthOfYear(monthOfYear).withDayOfMonth(dayOfMonth); + + int yearOffset = + Years.yearsBetween(offsetStart, datetime).getYears() / number * number; + + DateTime begin = offsetStart.plusYears(yearOffset); + DateTime end = begin.plusYears(number); + + return new IntervalWindow(begin.toInstant(), end.toInstant()); + } + + @Override + public Coder windowCoder() { + return IntervalWindow.getCoder(); + } + + @Override + public boolean isCompatible(WindowingFn other) { + if (!(other instanceof YearsWindows)) { + return false; + } + YearsWindows that = (YearsWindows) other; + return number == that.number + && monthOfYear == monthOfYear + && dayOfMonth == dayOfMonth + && startDate == that.startDate + && timeZone == that.timeZone; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java new file mode 100644 index 0000000000000..ea7a22c8fc41a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java @@ -0,0 +1,93 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import com.google.cloud.dataflow.sdk.coders.Coder; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** + * A {@link WindowingFn} that windows values into fixed-size timestamp-based windows. + * + *

For example, in order to partition the data into 10 minute windows: + *

 {@code
+ * PCollection items = ...;
+ * PCollection windowedItems = items.apply(
+ *   Window.by(FixedWindows.of(Duration.standardMinutes(10))));
+ * } 
+ */ +public class FixedWindows extends PartitioningWindowingFn { + + /** + * Size of this window. + */ + private final Duration size; + + /** + * Offset of this window. Windows start at time + * N * size + offset, where 0 is the epoch. + */ + private final Duration offset; + + /** + * Partitions the timestamp space into half-open intervals of the form + * [N * size, (N + 1) * size), where 0 is the epoch. + */ + public static FixedWindows of(Duration size) { + return new FixedWindows(size, Duration.ZERO); + } + + /** + * Partitions the timestamp space into half-open intervals of the form + * [N * size + offset, (N + 1) * size + offset), + * where 0 is the epoch. + * + * @throws IllegalAgumentException if offset is not in [0, size) + */ + public FixedWindows withOffset(Duration offset) { + return new FixedWindows(size, offset); + } + + private FixedWindows(Duration size, Duration offset) { + if (offset.isShorterThan(Duration.ZERO) || !offset.isShorterThan(size)) { + throw new IllegalArgumentException( + "FixedWindows WindowingStrategies must have 0 <= offset < size"); + } + this.size = size; + this.offset = offset; + } + + @Override + public IntervalWindow assignWindow(Instant timestamp) { + long start = timestamp.getMillis() + - timestamp.plus(size).minus(offset).getMillis() % size.getMillis(); + return new IntervalWindow(new Instant(start), size); + } + + @Override + public Coder windowCoder() { + return IntervalWindow.getFixedSizeCoder(size); + } + + @Override + public boolean isCompatible(WindowingFn other) { + return (other instanceof FixedWindows) + && (size.equals(((FixedWindows) other).size)) + && (offset.equals(((FixedWindows) other).offset)); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java new file mode 100644 index 0000000000000..bfcb9c7fa1595 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java @@ -0,0 +1,84 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; + +import org.joda.time.Instant; + +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.Collection; + +/** + * Default {@link WindowingFn} where all data is in the same bucket. + */ +public class GlobalWindow + extends NonMergingWindowingFn { + @Override + public Collection assignWindows(AssignContext c) { + return Arrays.asList(Window.INSTANCE); + } + + @Override + public boolean isCompatible(WindowingFn o) { + return o instanceof GlobalWindow; + } + + @Override + public Coder windowCoder() { + return Window.Coder.INSTANCE; + } + + /** + * The default window into which all data is placed. + */ + public static class Window extends BoundedWindow { + public static Window INSTANCE = new Window(); + + @Override + public Instant maxTimestamp() { + return new Instant(Long.MAX_VALUE); + } + + private Window() {} + + /** + * {@link Coder} for encoding and decoding {@code Window}s. + */ + public static class Coder extends AtomicCoder { + public static Coder INSTANCE = new Coder(); + + @Override + public void encode(Window window, OutputStream outStream, Context context) {} + + @Override + public Window decode(InputStream inStream, Context context) { + return Window.INSTANCE; + } + + @Override + public boolean isDeterministic() { + return true; + } + + private Coder() {} + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/IntervalWindow.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/IntervalWindow.java new file mode 100644 index 0000000000000..8ac23501c97e5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/IntervalWindow.java @@ -0,0 +1,257 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import static com.google.cloud.dataflow.sdk.util.Structs.addString; +import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudDuration; +import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.InstantCoder; +import com.google.cloud.dataflow.sdk.util.CloudObject; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** + * An implementation of {@link BoundedWindow} that represents an interval from + * {@link #start} (inclusive) to {@link #end} (exclusive). + */ +public class IntervalWindow extends BoundedWindow + implements Comparable { + /** + * Start of the interval, inclusive. + */ + private final Instant start; + + /** + * End of the interval, exclusive. + */ + private final Instant end; + + /** + * Creates a new IntervalWindow that represents the half-open time + * interval [start, end). + */ + public IntervalWindow(Instant start, Instant end) { + this.start = start; + this.end = end; + } + + public IntervalWindow(Instant start, Duration size) { + this.start = start; + this.end = start.plus(size); + } + + /** + * Returns the start of this window, inclusive. + */ + public Instant start() { + return start; + } + + /** + * Returns the end of this window, exclusive. + */ + public Instant end() { + return end; + } + + /** + * Returns the largest timestamp that can be included in this window. + */ + @Override + public Instant maxTimestamp() { + // end not inclusive + return end.minus(1); + } + + /** + * Returns whether this window contains the given window. + */ + public boolean contains(IntervalWindow other) { + return !this.start.isAfter(other.start) && !this.end.isBefore(other.end); + } + + /** + * Returns whether this window is disjoint from the given window. + */ + public boolean isDisjoint(IntervalWindow other) { + return !this.end.isAfter(other.start) || !other.end.isAfter(this.start); + } + + /** + * Returns whether this window intersects the given window. + */ + public boolean intersects(IntervalWindow other) { + return !isDisjoint(other); + } + + /** + * Returns the minimal window that includes both this window and + * the given window. + */ + public IntervalWindow span(IntervalWindow other) { + return new IntervalWindow( + new Instant(Math.min(start.getMillis(), other.start.getMillis())), + new Instant(Math.max(end.getMillis(), other.end.getMillis()))); + } + + @Override + public boolean equals(Object o) { + return (o instanceof IntervalWindow) + && ((IntervalWindow) o).end.isEqual(end) + && ((IntervalWindow) o).start.isEqual(start); + } + + @Override + public int hashCode() { + // The end values are themselves likely to be arithmetic sequence, + // which is a poor distribution to use for a hashtable, so we + // add a highly non-linear transformation. + return (int) + (start.getMillis() + modInverse((int) (end.getMillis() << 1) + 1)); + } + + /** + * Compute the inverse of (odd) x mod 2^32. + */ + private int modInverse(int x) { + // Cube gives inverse mod 2^4, as x^4 == 1 (mod 2^4) for all odd x. + int inverse = x * x * x; + // Newton iteration doubles correct bits at each step. + inverse *= 2 - x * inverse; + inverse *= 2 - x * inverse; + inverse *= 2 - x * inverse; + return inverse; + } + + @Override + public String toString() { + return "[" + start + ".." + end + ")"; + } + + @Override + public int compareTo(IntervalWindow o) { + if (start.isEqual(o.start)) { + return end.compareTo(o.end); + } + return start.compareTo(o.start); + } + + /** + * Returns a Coder suitable for encoding IntervalWindows. + */ + public static Coder getCoder() { + return IntervalWindowCoder.of(); + } + + /** + * Returns a Coder for encoding interval windows of fixed size (which + * is more efficient than {@link #getCoder()} as it only needs to encode + * one endpoint). + */ + public static Coder getFixedSizeCoder(final Duration size) { + return FixedSizeIntervalWindowCoder.of(size); + } + + private static class IntervalWindowCoder extends AtomicCoder { + private static final IntervalWindowCoder INSTANCE = + new IntervalWindowCoder(); + private static final Coder instantCoder = InstantCoder.of(); + + @JsonCreator + public static IntervalWindowCoder of() { + return INSTANCE; + } + + @Override + public void encode(IntervalWindow window, + OutputStream outStream, + Context context) + throws IOException, CoderException { + instantCoder.encode(window.start, outStream, context.nested()); + instantCoder.encode(window.end, outStream, context.nested()); + } + + @Override + public IntervalWindow decode(InputStream inStream, Context context) + throws IOException, CoderException { + Instant start = instantCoder.decode(inStream, context.nested()); + Instant end = instantCoder.decode(inStream, context.nested()); + return new IntervalWindow(start, end); + } + + @Override + public boolean isDeterministic() { return true; } + } + + private static class FixedSizeIntervalWindowCoder + extends AtomicCoder { + private static final Coder instantCoder = InstantCoder.of(); + + private final Duration size; + + @JsonCreator + public static FixedSizeIntervalWindowCoder of( + @JsonProperty("duration") String duration) { + return of(fromCloudDuration(duration)); + } + + public static FixedSizeIntervalWindowCoder of(Duration size) { + return new FixedSizeIntervalWindowCoder(size); + } + + private FixedSizeIntervalWindowCoder(Duration size) { + this.size = size; + } + + @Override + public void encode(IntervalWindow window, + OutputStream outStream, + Context context) + throws IOException, CoderException { + instantCoder.encode(window.start, outStream, context); + } + + @Override + public IntervalWindow decode(InputStream inStream, Context context) + throws IOException, CoderException { + Instant start = instantCoder.decode(inStream, context); + return new IntervalWindow(start, size); + } + + @Override + public boolean isDeterministic() { return true; } + + @Override + public CloudObject asCloudObject() { + CloudObject result = super.asCloudObject(); + addString(result, "duration", toCloudDuration(Duration.millis(size.getMillis()))); + return result; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowingFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowingFn.java new file mode 100644 index 0000000000000..7ad7f29f6655a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowingFn.java @@ -0,0 +1,75 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import com.google.cloud.dataflow.sdk.coders.Coder; + +import java.util.Collection; + +/** + * A {@link WindowingFn} that represents an invalid pipeline state. + * + * @param window type + */ +public class InvalidWindowingFn extends WindowingFn { + private String cause; + private WindowingFn originalWindowingFn; + + public InvalidWindowingFn(String cause, WindowingFn originalWindowingFn) { + this.originalWindowingFn = originalWindowingFn; + this.cause = cause; + } + + /** + * Returns the reason that this {@code WindowingFn} is invalid. + */ + public String getCause() { + return cause; + } + + /** + * Returns the original windowingFn that this InvalidWindowingFn replaced. + */ + public WindowingFn getOriginalWindowingFn() { + return originalWindowingFn; + } + + @Override + public Collection assignWindows(AssignContext c) { + throw new UnsupportedOperationException(); + } + + @Override + public void mergeWindows(MergeContext c) { + throw new UnsupportedOperationException(); + } + + @Override + public Coder windowCoder() { + return originalWindowingFn.windowCoder(); + } + + /** + * {@code InvalidWindowingFn} objects with the same {@code originalWindowingFn} are compatible. + */ + @Override + public boolean isCompatible(WindowingFn other) { + return getClass() == other.getClass() + && getOriginalWindowingFn().isCompatible( + ((InvalidWindowingFn) other).getOriginalWindowingFn()); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java new file mode 100644 index 0000000000000..4d4dd8492684d --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java @@ -0,0 +1,86 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * A {@link WindowingFn} that merges overlapping {@link IntervalWindow}s. + */ +public class MergeOverlappingIntervalWindows { + + /** + * Merge overlapping intervals. + */ + public static void mergeWindows(WindowingFn.MergeContext c) throws Exception { + // Merge any overlapping windows into a single window. + // Sort the list of existing windows so we only have to + // traverse the list once rather than considering all + // O(n^2) window pairs. + List sortedWindows = new ArrayList<>(); + for (IntervalWindow window : c.windows()) { + sortedWindows.add(window); + } + Collections.sort(sortedWindows); + List merges = new ArrayList<>(); + MergeCandidate current = new MergeCandidate(); + for (IntervalWindow window : sortedWindows) { + if (current.intersects(window)) { + current.add(window); + } else { + merges.add(current); + current = new MergeCandidate(window); + } + } + merges.add(current); + for (MergeCandidate merge : merges) { + merge.apply(c); + } + } + + private static class MergeCandidate { + private IntervalWindow union; + private final List parts; + public MergeCandidate() { + parts = new ArrayList<>(); + } + public MergeCandidate(IntervalWindow window) { + union = window; + parts = new ArrayList<>(Arrays.asList(window)); + } + public boolean intersects(IntervalWindow window) { + return union == null || union.intersects(window); + } + public void add(IntervalWindow window) { + union = union == null ? window : union.span(window); + parts.add(window); + } + public void apply(WindowingFn.MergeContext c) throws Exception { + if (parts.size() > 1) { + c.merge(parts, union); + } + } + + @Override + public String toString() { + return "MergeCandidate[union=" + union + ", parts=" + parts + "]"; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowingFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowingFn.java new file mode 100644 index 0000000000000..ffeea996d60dc --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowingFn.java @@ -0,0 +1,31 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +/** + * Abstract base class for {@link WindowingFn}s that do not merge windows. + * + * @param type of elements being windowed + * @param {@link BoundedWindow} subclass used to represent the windows used by this + * {@code WindowingFn} + */ +public abstract class NonMergingWindowingFn + extends WindowingFn { + + @Override + public final void mergeWindows(MergeContext c) { } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowingFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowingFn.java new file mode 100644 index 0000000000000..6a65ba134f181 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowingFn.java @@ -0,0 +1,42 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import org.joda.time.Instant; + +import java.util.Arrays; +import java.util.Collection; + +/** + * A {@link WindowingFn} that places each value into exactly one window + * based on its timestamp and never merges windows. + * + * @param type of elements being windowed + * @param window type + */ +public abstract class PartitioningWindowingFn + extends NonMergingWindowingFn { + /** + * Returns the single window to which elements with this timestamp belong. + */ + public abstract W assignWindow(Instant timestamp); + + @Override + public final Collection assignWindows(AssignContext c) { + return Arrays.asList(assignWindow(c.timestamp())); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java new file mode 100644 index 0000000000000..47f8a08005830 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java @@ -0,0 +1,81 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import com.google.cloud.dataflow.sdk.coders.Coder; + +import org.joda.time.Duration; + +import java.util.Arrays; +import java.util.Collection; + +/** + * A WindowingFn windowing values into sessions separated by {@link #gapDuration}-long + * periods with no elements. + * + *

For example, in order to window data into session with at least 10 minute + * gaps in between them: + *

 {@code
+ * PCollection pc = ...;
+ * PCollection windowed_pc = pc.apply(
+ *   Window.by(Sessions.withGapDuration(Duration.standardMinutes(10))));
+ * } 
+ */ +public class Sessions extends WindowingFn { + + /** + * Duration of the gaps between sessions. + */ + private final Duration gapDuration; + + /** + * Creates a {@code Sessions} {@link WindowingFn} with the specified gap duration. + */ + public static Sessions withGapDuration(Duration gapDuration) { + return new Sessions(gapDuration); + } + + /** + * Creates a {@code Sessions} {@link WindowingFn} with the specified gap duration. + */ + private Sessions(Duration gapDuration) { + this.gapDuration = gapDuration; + } + + @Override + public Collection assignWindows(AssignContext c) { + // Assign each element into a window from its timestamp until gapDuration in the + // future. Overlapping windows (representing elements within gapDuration of + // each other) will be merged. + return Arrays.asList(new IntervalWindow(c.timestamp(), gapDuration)); + } + + @Override + public void mergeWindows(MergeContext c) throws Exception { + MergeOverlappingIntervalWindows.mergeWindows(c); + } + + @Override + public Coder windowCoder() { + return IntervalWindow.getCoder(); + } + + @Override + public boolean isCompatible(WindowingFn other) { + return other instanceof Sessions; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java new file mode 100644 index 0000000000000..6643289071ef5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java @@ -0,0 +1,131 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import com.google.cloud.dataflow.sdk.coders.Coder; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +/** + * A WindowingFn that windows values into possibly overlapping fixed-size + * timestamp-based windows. + * + *

For example, in order to window data into 10 minute windows that + * update every minute: + *

 {@code
+ * PCollection items = ...;
+ * PCollection windowedItems = items.apply(
+ *   Window.by(SlidingWindows.of(Duration.standardMinutes(10))));
+ * } 
+ */ +public class SlidingWindows extends NonMergingWindowingFn { + + /** + * Amount of time between generated windows. + */ + private final Duration period; + + /** + * Size of the generated windows. + */ + private final Duration size; + + /** + * Offset of the generated windows. + * Windows start at time N * start + offset, where 0 is the epoch. + */ + private final Duration offset; + + /** + * Assigns timestamps into half-open intervals of the form + * [N * period, N * period + size), where 0 is the epoch. + * + *

If {@link SlidingWindows#every} is not called, the period defaults + * to one millisecond. + */ + public static SlidingWindows of(Duration size) { + return new SlidingWindows(new Duration(1), size, Duration.ZERO); + } + + /** + * Returns a new {@code SlidingWindows} with the original size, that assigns + * timestamps into half-open intervals of the form + * [N * period, N * period + size), where 0 is the epoch. + */ + public SlidingWindows every(Duration period) { + return new SlidingWindows(period, size, offset); + } + + /** + * Assigns timestamps into half-open intervals of the form + * [N * period + offset, N * period + offset + size). + * + * @throws IllegalArgumentException if offset is not in [0, period) + */ + public SlidingWindows withOffset(Duration offset) { + return new SlidingWindows(period, size, offset); + } + + private SlidingWindows(Duration period, Duration size, Duration offset) { + if (offset.isShorterThan(Duration.ZERO) + || !offset.isShorterThan(period) + || !size.isLongerThan(Duration.ZERO)) { + throw new IllegalArgumentException( + "SlidingWindows WindowingStrategies must have 0 <= offset < period and 0 < size"); + } + this.period = period; + this.size = size; + this.offset = offset; + } + + @Override + public Coder windowCoder() { + return IntervalWindow.getFixedSizeCoder(size); + } + + @Override + public Collection assignWindows(AssignContext c) { + List windows = + new ArrayList<>((int) (size.getMillis() / period.getMillis())); + Instant timestamp = c.timestamp(); + long lastStart = timestamp.getMillis() + - timestamp.plus(period).minus(offset).getMillis() % period.getMillis(); + for (long start = lastStart; + start > timestamp.minus(size).getMillis(); + start -= period.getMillis()) { + windows.add(new IntervalWindow(new Instant(start), size)); + } + return windows; + } + + @Override + public boolean isCompatible(WindowingFn other) { + if (other instanceof SlidingWindows) { + SlidingWindows that = (SlidingWindows) other; + return period.equals(that.period) + && size.equals(that.size) + && offset.equals(that.offset); + } else { + return false; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java new file mode 100644 index 0000000000000..68796c908aba5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java @@ -0,0 +1,321 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray; +import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString; +import static com.google.cloud.dataflow.sdk.util.StringUtils.jsonStringToByteArray; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn; +import com.google.cloud.dataflow.sdk.util.DirectModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.DoFnRunner; +import com.google.cloud.dataflow.sdk.util.PTuple; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.StringUtils; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * {@code Window} logically divides up or groups the elements of a + * {@link PCollection} into finite windows according to a {@link WindowingFn}. + * The output of {@code Window} contains the same elements as input, but they + * have been logically assigned to windows. The next + * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}s, including one + * within composite transforms, will group by the combination of keys and + * windows. + + *

See {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} + * for more information about how grouping with windows works. + * + *

Windowing a {@code PCollection} allows chunks of it to be processed + * individually, before the entire {@code PCollection} is available. This is + * especially important for {@code PCollection}s with unbounded size, + * since the full {@code PCollection} is + * never available at once, since more data is continually arriving. + * For {@code PCollection}s with a bounded size (aka. conventional batch mode), + * by default, all data is implicitly in a single window, unless + * {@code Window} is applied. + * + *

For example, a simple form of windowing divides up the data into + * fixed-width time intervals, using {@link FixedWindows}. + * The following example demonstrates how to use {@code Window} in a pipeline + * that counts the number of occurrences of strings each minute: + * + *

 {@code
+ * PCollection items = ...;
+ * PCollection windowed_items = item.apply(
+ *   Window.into(FixedWindows.of(1, TimeUnit.MINUTES)));
+ * PCollection> windowed_counts = windowed_items.apply(
+ *   Count.create());
+ * } 
+ * + *

Let (data, timestamp) denote a data element along with its timestamp. + * Then, if the input to this pipeline consists of + * {("foo", 15s), ("bar", 30s), ("foo", 45s), ("foo", 1m30s)}, + * the output will be + * {(KV("foo", 2), 1m), (KV("bar", 1), 1m), (KV("foo", 1), 2m)} + * + * + *

Several predefined {@link WindowingFn}s are provided: + *

    + *
  • {@link FixedWindows} partitions the timestamps into fixed-width intervals. + *
  • {@link SlidingWindows} places data into overlapping fixed-width intervals. + *
  • {@link Sessions} groups data into sessions where each item in a window + * is separated from the next by no more than a specified gap. + *
+ * + * Additionally, custom {@link WindowingFn}s can be created, by creating new + * subclasses of {@link WindowingFn}. + */ +public class Window { + /** + * Creates a {@code Window} {@code PTransform} with the given name. + * + *

See the discussion of Naming in + * {@link com.google.cloud.dataflow.sdk.transforms.ParDo} for more explanation. + * + *

The resulting {@code PTransform} is incomplete, and its input/output + * type is not yet bound. Use {@link Window.Unbound#into} to specify the + * {@link WindowingFn} to use, which will also bind the input/output type of this + * {@code PTransform}. + */ + public static Unbound named(String name) { + return new Unbound().named(name); + } + + /** + * Creates a {@code Window} {@code PTransform} that uses the given + * {@link WindowingFn} to window the data. + * + *

The resulting {@code PTransform}'s types have been bound, with both the + * input and output being a {@code PCollection}, inferred from the types of + * the argument {@code WindowingFn}. It is ready to be applied, or further + * properties can be set on it first. + */ + public static Bound into(WindowingFn fn) { + return new Unbound().into(fn); + } + + /** + * An incomplete {@code Window} transform, with unbound input/output type. + * + *

Before being applied, {@link Window.Unbound#into} must be + * invoked to specify the {@link WindowingFn} to invoke, which will also + * bind the input/output type of this {@code PTransform}. + */ + public static class Unbound { + String name; + + Unbound() {} + + Unbound(String name) { + this.name = name; + } + + /** + * Returns a new {@code Window} transform that's like this + * transform but with the specified name. Does not modify this + * transform. The resulting transform is still incomplete. + * + *

See the discussion of Naming in + * {@link com.google.cloud.dataflow.sdk.transforms.ParDo} for more + * explanation. + */ + public Unbound named(String name) { + return new Unbound(name); + } + + /** + * Returns a new {@code Window} {@code PTransform} that's like this + * transform but which will use the given {@link WindowingFn}, and which has + * its input and output types bound. Does not modify this transform. The + * resulting {@code PTransform} is sufficiently specified to be applied, + * but more properties can still be specified. + */ + public Bound into(WindowingFn fn) { + return new Bound<>(name, fn); + } + } + + /** + * A {@code PTransform} that windows the elements of a {@code PCollection}, + * into finite windows according to a user-specified {@code WindowingFn}. + * + * @param The type of elements this {@code Window} is applied to + */ + public static class Bound extends PTransform, PCollection> { + WindowingFn fn; + + Bound(String name, WindowingFn fn) { + this.name = name; + this.fn = fn; + } + + /** + * Returns a new {@code Window} {@code PTransform} that's like this + * {@code PTransform} but with the specified name. Does not + * modify this {@code PTransform}. + * + *

See the discussion of Naming in + * {@link com.google.cloud.dataflow.sdk.transforms.ParDo} for more + * explanation. + */ + public Bound named(String name) { + return new Bound<>(name, fn); + } + + @Override + public PCollection apply(PCollection input) { + return PCollection.createPrimitiveOutputInternal(fn); + } + + @Override + protected Coder getDefaultOutputCoder() { + return getInput().getCoder(); + } + + @Override + protected String getKindString() { + return "Window.Into(" + StringUtils.approximateSimpleName(fn.getClass()) + ")"; + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Creates a {@code Window} {@code PTransform} that does not change assigned + * windows, but will cause windows to be merged again as part of the next + * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}. + */ + public static Remerge remerge() { + return new Remerge(); + } + + /** + * {@code PTransform} that does not change assigned windows, but will cause + * windows to be merged again as part of the next + * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}. + */ + public static class Remerge extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection input) { + WindowingFn windowingFn = getInput().getWindowingFn(); + WindowingFn outputWindowingFn = + (windowingFn instanceof InvalidWindowingFn) + ? ((InvalidWindowingFn) windowingFn).getOriginalWindowingFn() + : windowingFn; + + return input.apply(ParDo.named("Identity").of(new DoFn() { + @Override public void processElement(ProcessContext c) { + c.output(c.element()); + } + })).setWindowingFnInternal(outputWindowingFn); + } + } + + + ///////////////////////////////////////////////////////////////////////////// + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + Bound.class, + new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + Bound transform, + DirectPipelineRunner.EvaluationContext context) { + evaluateHelper(transform, context); + } + }); + } + + private static void evaluateHelper( + Bound transform, + DirectPipelineRunner.EvaluationContext context) { + PCollection input = transform.getInput(); + + DirectModeExecutionContext executionContext = new DirectModeExecutionContext(); + + TupleTag outputTag = new TupleTag<>(); + DoFn addWindowsDoFn = new AssignWindowsDoFn<>(transform.fn); + DoFnRunner addWindowsRunner = + DoFnRunner.createWithListOutputs( + context.getPipelineOptions(), + addWindowsDoFn, + PTuple.empty(), + outputTag, + new ArrayList>(), + executionContext.getStepContext(context.getStepName(transform)), + context.getAddCounterMutator()); + + addWindowsRunner.startBundle(); + + // Process input elements. + for (DirectPipelineRunner.ValueWithMetadata inputElem + : context.getPCollectionValuesWithMetadata(input)) { + executionContext.setKey(inputElem.getKey()); + addWindowsRunner.processElement(inputElem.getWindowedValue()); + } + + addWindowsRunner.finishBundle(); + + context.setPCollectionValuesWithMetadata( + transform.getOutput(), + executionContext.getOutput(outputTag)); + } + + + ///////////////////////////////////////////////////////////////////////////// + + static { + DataflowPipelineTranslator.registerTransformTranslator( + Bound.class, + new DataflowPipelineTranslator.TransformTranslator() { + @Override + public void translate( + Bound transform, + DataflowPipelineTranslator.TranslationContext context) { + translateHelper(transform, context); + } + }); + } + + private static void translateHelper( + Bound transform, + DataflowPipelineTranslator.TranslationContext context) { + context.addStep(transform, "Bucket"); + context.addInput(PropertyNames.PARALLEL_INPUT, transform.getInput()); + context.addOutput(PropertyNames.OUTPUT, transform.getOutput()); + + byte[] serializedBytes = serializeToByteArray(transform.fn); + String serializedJson = byteArrayToJsonString(serializedBytes); + assert Arrays.equals(serializedBytes, + jsonStringToByteArray(serializedJson)); + context.addInput(PropertyNames.SERIALIZED_FN, serializedJson); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingFn.java new file mode 100644 index 0000000000000..0f049372555bb --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingFn.java @@ -0,0 +1,117 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import com.google.cloud.dataflow.sdk.coders.Coder; + +import org.joda.time.Instant; + +import java.io.Serializable; +import java.util.Collection; + +/** + * The argument to the {@link Window} transform used to assign elements into + * windows and to determine how windows are merged. See {@link Window} for more + * information on how {@code WindowingFn}s are used and for a library of + * predefined {@code WindowingFn}s. + * + *

Users will generally want to use the predefined + * {@code WindowingFn}s, but it is also possible to create new + * subclasses. + * TODO: Describe how to properly create {@code WindowingFn}s. + * + * @param type of elements being windowed + * @param {@link BoundedWindow} subclass used to represent the + * windows used by this {@code WindowingFn} + */ +public abstract class WindowingFn + implements Serializable { + + /** + * Information available when running {@link #assignWindows}. + */ + public abstract class AssignContext { + /** + * Returns the current element. + */ + public abstract T element(); + + /** + * Returns the timestamp of the current element. + */ + public abstract Instant timestamp(); + + /** + * Returns the windows the current element was in, prior to this + * {@code AssignFn} being called. + */ + public abstract Collection windows(); + } + + /** + * Given a timestamp and element, returns the set of windows into which it + * should be placed. + */ + public abstract Collection assignWindows(AssignContext c) throws Exception; + + /** + * Information available when running {@link #mergeWindows}. + */ + public abstract class MergeContext { + /** + * Returns the current set of windows. + */ + public abstract Collection windows(); + + /** + * Signals to the framework that the windows in {@code toBeMerged} should + * be merged together to form {@code mergeResult}. + * + *

{@code toBeMerged} should be a subset of {@link #windows} + * and disjoint from the {@code toBeMerged} set of previous calls + * to {@code merge}. + * + *

{@code mergeResult} must either not be in {@link #windows} or be in + * {@code toBeMerged}. + * + * @throws IllegalArgumentException if any elements of toBeMerged are not + * in windows(), or have already been merged + */ + public abstract void merge(Collection toBeMerged, W mergeResult) + throws Exception; + } + + /** + * Does whatever merging of windows is necessary. + * + *

See {@link MergeOverlappingIntervalWindows#mergeWindows} for an + * example of how to override this method. + */ + public abstract void mergeWindows(MergeContext c) throws Exception; + + /** + * Returns whether this performs the same merging as the given + * {@code WindowingFn}. + */ + public abstract boolean isCompatible(WindowingFn other); + + /** + * Returns the {@link Coder} used for serializing the windows used + * by this windowingFn. + */ + public abstract Coder windowCoder(); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AbstractWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AbstractWindowSet.java new file mode 100644 index 0000000000000..dda2488dac346 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AbstractWindowSet.java @@ -0,0 +1,170 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.values.KV; + +import java.util.Arrays; +import java.util.Collection; + +/** + * Abstract class representing a set of active windows for a key. + */ +abstract class AbstractWindowSet { + /** + * Hook for determining how to keep track of active windows and when they + * should be marked as complete. + */ + interface ActiveWindowManager { + /** + * Notes that a window has been added to the active set. + * + *

The given window must not already be active. + */ + void addWindow(W window) throws Exception; + + /** + * Notes that a window has been explicitly removed from the active set. + * + *

The given window must currently be active. + * + *

Windows are implicitly removed from the active set when they are + * complete, and this method will not be called. This method is called when + * a window is merged into another and thus is no longer active. + */ + void removeWindow(W window) throws Exception; + } + + /** + * Wrapper around AbstractWindowSet that provides the MergeContext interface. + */ + static class WindowMergeContext + extends WindowingFn.MergeContext { + private final AbstractWindowSet windowSet; + + public WindowMergeContext( + AbstractWindowSet windowSet, + WindowingFn windowingFn) { + ((WindowingFn) windowingFn).super(); + this.windowSet = windowSet; + } + + @Override public Collection windows() { + return windowSet.windows(); + } + + @Override public void merge(Collection toBeMerged, W mergeResult) throws Exception { + windowSet.merge(toBeMerged, mergeResult); + } + } + + protected final K key; + protected final WindowingFn windowingFn; + protected final Coder inputCoder; + protected final DoFnProcessContext> context; + protected final ActiveWindowManager activeWindowManager; + + protected AbstractWindowSet( + K key, + WindowingFn windowingFn, + Coder inputCoder, + DoFnProcessContext> context, + ActiveWindowManager activeWindowManager) { + this.key = key; + this.windowingFn = windowingFn; + this.inputCoder = inputCoder; + this.context = context; + this.activeWindowManager = activeWindowManager; + } + + /** + * Returns the set of known windows. + */ + protected abstract Collection windows(); + + /** + * Returns the final value of the elements in the given window. + * + *

Illegal to call if the window does not exist in the set. + */ + protected abstract VO finalValue(W window) throws Exception; + + /** + * Adds the given value in the given window to the set. + * + *

If the window already exists, puts the element into that window. + * If not, adds the window to the set first, then puts the element + * in the window. + */ + protected abstract void put(W window, VI value) throws Exception; + + /** + * Removes the given window from the set. + * + *

Illegal to call if the window does not exist in the set. + * + *

{@code AbstractWindowSet} subclasses may throw + * {@link UnsupportedOperationException} if they do not support removing + * windows. + */ + protected abstract void remove(W window) throws Exception; + + /** + * Instructs this set to merge the windows in toBeMerged into mergeResult. + * + *

{@code toBeMerged} should be a subset of {@link #windows} + * and disjoint from the {@code toBeMerged} set of previous calls + * to {@code merge}. + * + *

{@code mergeResult} must either not be in {@link @windows} or be in + * {@code toBeMerged}. + * + *

{@code AbstractWindowSet} subclasses may throw + * {@link UnsupportedOperationException} if they do not support merging windows. + */ + protected abstract void merge(Collection toBeMerged, W mergeResult) throws Exception; + + /** + * Returns whether this window set contains the given window. + * + *

{@code AbstractWindowSet} subclasses may throw + * {@link UnsupportedOperationException} if they do not support querying for + * which windows are active. If this is the case, callers must ensure they + * do not call {@link #finalValue} on non-existent windows. + */ + protected abstract boolean contains(W window); + + /** + * Marks the window as complete, causing its elements to be emitted. + */ + public void markCompleted(W window) throws Exception { + VO value = finalValue(window); + remove(window); + context.outputWindowedValue( + KV.of(key, value), + window.maxTimestamp(), + Arrays.asList(window)); + } + + /** + * Hook for WindowSets to take action before they are deleted. + */ + protected void flush() throws Exception {} +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AggregatorImpl.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AggregatorImpl.java new file mode 100644 index 0000000000000..e71bf7f8a7f0c --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AggregatorImpl.java @@ -0,0 +1,111 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MAX; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MIN; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; + +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; +import com.google.cloud.dataflow.sdk.transforms.Max; +import com.google.cloud.dataflow.sdk.transforms.Min; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +/** + * An implementation of the {@code Aggregator} interface. + * + * @param the type of input values + * @param the type of accumulator values + * @param the type of output value + */ +public class AggregatorImpl implements Aggregator { + + private final Counter counter; + + /* + * Constructs a new aggregator with the given name and aggregation logic + * specified in the CombineFn argument. The underlying counter is + * automatically added into the provided CounterSet. + * + *

If a counter with the same name already exists, it will be + * reused, as long as it has the same type. + */ + public AggregatorImpl(String name, + CombineFn combiner, + CounterSet.AddCounterMutator addCounterMutator) { + this((Counter) constructCounter(name, combiner), addCounterMutator); + } + + /* + * Constructs a new aggregator with the given name and aggregation logic + * specified in the SerializableFunction argument. The underlying counter is + * automatically added into the provided CounterSet. + * + *

If a counter with the same name already exists, it will be + * reused, as long as it has the same type. + */ + public AggregatorImpl(String name, + SerializableFunction, VO> combiner, + CounterSet.AddCounterMutator addCounterMutator) { + this((Counter) constructCounter(name, combiner), addCounterMutator); + } + + private AggregatorImpl(Counter counter, + CounterSet.AddCounterMutator addCounterMutator) { + try { + this.counter = addCounterMutator.addCounter(counter); + } catch (IllegalArgumentException ex) { + throw new IllegalArgumentException( + "aggregator's name collides with an existing aggregator " + + "or system-provided counter of an incompatible type"); + } + } + + private static Counter constructCounter(String name, Object combiner) { + if (combiner.getClass() == Sum.SumIntegerFn.class) { + return Counter.ints(name, SUM); + } else if (combiner.getClass() == Sum.SumLongFn.class) { + return Counter.longs(name, SUM); + } else if (combiner.getClass() == Sum.SumDoubleFn.class) { + return Counter.doubles(name, SUM); + } else if (combiner.getClass() == Min.MinIntegerFn.class) { + return Counter.ints(name, MIN); + } else if (combiner.getClass() == Min.MinLongFn.class) { + return Counter.longs(name, MIN); + } else if (combiner.getClass() == Min.MinDoubleFn.class) { + return Counter.doubles(name, MIN); + } else if (combiner.getClass() == Max.MaxIntegerFn.class) { + return Counter.ints(name, MAX); + } else if (combiner.getClass() == Max.MaxLongFn.class) { + return Counter.longs(name, MAX); + } else if (combiner.getClass() == Max.MaxDoubleFn.class) { + return Counter.doubles(name, MAX); + } else { + throw new IllegalArgumentException("unsupported combiner in Aggregator: " + + combiner.getClass().getName()); + } + } + + @Override + public void addValue(VI value) { + counter.addValue(value); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ApiErrorExtractor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ApiErrorExtractor.java new file mode 100644 index 0000000000000..ad181cee40b38 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ApiErrorExtractor.java @@ -0,0 +1,104 @@ +/** + * Copyright 2013 Google Inc. All Rights Reserved. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.googleapis.json.GoogleJsonError; +import com.google.api.client.googleapis.json.GoogleJsonResponseException; +import com.google.api.client.http.HttpStatusCodes; +import com.google.common.annotations.VisibleForTesting; + +import java.io.IOException; + +/** + * Translates exceptions from API calls into higher-level meaning, while allowing injectability + * for testing how API errors are handled. + */ +public class ApiErrorExtractor { + + public static final int STATUS_CODE_CONFLICT = 409; + public static final int STATUS_CODE_RANGE_NOT_SATISFIABLE = 416; + + /** + * Determines if the given exception indicates 'item not found'. + */ + public boolean itemNotFound(IOException e) { + if (e instanceof GoogleJsonResponseException) { + return (getHttpStatusCode((GoogleJsonResponseException) e)) == + HttpStatusCodes.STATUS_CODE_NOT_FOUND; + } + return false; + } + + /** + * Determines if the given GoogleJsonError indicates 'item not found'. + */ + public boolean itemNotFound(GoogleJsonError e) { + return e.getCode() == HttpStatusCodes.STATUS_CODE_NOT_FOUND; + } + + /** + * Checks if HTTP status code indicates the error specified. + */ + private boolean hasHttpCode(IOException e, int code) { + if (e instanceof GoogleJsonResponseException) { + return (getHttpStatusCode((GoogleJsonResponseException) e)) == code; + } + return false; + } + + /** + * Determines if the given exception indicates 'conflict' (already exists). + */ + public boolean alreadyExists(IOException e) { + return hasHttpCode(e, STATUS_CODE_CONFLICT); + } + + /** + * Determines if the given exception indicates 'range not satisfiable'. + */ + public boolean rangeNotSatisfiable(IOException e) { + return hasHttpCode(e, STATUS_CODE_RANGE_NOT_SATISFIABLE); + } + + /** + * Determines if the given exception indicates 'access denied'. + */ + public boolean accessDenied(GoogleJsonResponseException e) { + return getHttpStatusCode(e) == HttpStatusCodes.STATUS_CODE_FORBIDDEN; + } + + /** + * Determines if the given exception indicates 'access denied', recursively checking inner + * getCause() if outer exception isn't an instance of the correct class. + */ + public boolean accessDenied(IOException e) { + return (e.getCause() != null) && + (e.getCause() instanceof GoogleJsonResponseException) && + accessDenied((GoogleJsonResponseException) e.getCause()); + } + + /** + * Returns HTTP status code from the given exception. + * + * Note: GoogleJsonResponseException.getStatusCode() method is marked final therefore + * it cannot be mocked using Mockito. We use this helper so that we can override it in tests. + */ + @VisibleForTesting + int getHttpStatusCode(GoogleJsonResponseException e) { + return e.getStatusCode(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AppEngineEnvironment.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AppEngineEnvironment.java new file mode 100644 index 0000000000000..f3b57a4508b0e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AppEngineEnvironment.java @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import java.lang.reflect.InvocationTargetException; + +/** Stores whether we are running within AppEngine or not. */ +public class AppEngineEnvironment { + /** + * True if running inside of AppEngine, false otherwise. + */ + @Deprecated + public static final boolean IS_APP_ENGINE = isAppEngine(); + + /** + * Attempts to detect whether we are inside of AppEngine. + *

+ * Purposely copied and left private from private code.google.common.util.concurrent.MoreExecutors#isAppEngine. + * + * @return true if we are inside of AppEngine, false otherwise. + */ + static boolean isAppEngine() { + if (System.getProperty("com.google.appengine.runtime.environment") == null) { + return false; + } + try { + // If the current environment is null, we're not inside AppEngine. + return Class.forName("com.google.apphosting.api.ApiProxy") + .getMethod("getCurrentEnvironment") + .invoke(null) != null; + } catch (ClassNotFoundException e) { + // If ApiProxy doesn't exist, we're not on AppEngine at all. + return false; + } catch (InvocationTargetException e) { + // If ApiProxy throws an exception, we're not in a proper AppEngine environment. + return false; + } catch (IllegalAccessException e) { + // If the method isn't accessible, we're not on a supported version of AppEngine; + return false; + } catch (NoSuchMethodException e) { + // If the method doesn't exist, we're not on a supported version of AppEngine; + return false; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AssignWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AssignWindowsDoFn.java new file mode 100644 index 0000000000000..7649a8c637248 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AssignWindowsDoFn.java @@ -0,0 +1,64 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; + +import org.joda.time.Instant; + +import java.util.Collection; + +/** + * {@link DoFn} that tags elements of a PCollection with windows, according + * to the provided {@link WindowingFn}. + * @param Type of elements being windowed + * @param Window type + */ +public class AssignWindowsDoFn extends DoFn { + private WindowingFn fn; + + public AssignWindowsDoFn(WindowingFn fn) { + this.fn = fn; + } + + @Override + public void processElement(ProcessContext c) throws Exception { + final DoFnProcessContext context = (DoFnProcessContext) c; + Collection windows = + ((WindowingFn) fn).assignWindows( + ((WindowingFn) fn).new AssignContext() { + @Override + public T element() { + return context.element(); + } + + @Override + public Instant timestamp() { + return context.timestamp(); + } + + @Override + public Collection windows() { + return context.windows(); + } + }); + + context.outputWindowedValue(context.element(), context.timestamp(), windows); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOff.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOff.java new file mode 100644 index 0000000000000..78e8e0538b824 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOff.java @@ -0,0 +1,82 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.util.BackOff; +import com.google.common.base.Preconditions; + +/** + * Implementation of {@link BackOff} that increases the back off period for each retry attempt + * using a randomization function that grows exponentially. + *

+ * Example: The initial interval is .5 seconds and the maximum number of retries is 10. + * For 10 tries the sequence will be (values in seconds): + *

+ * + *
+   retry#      retry_interval     randomized_interval
+   1             0.5                [0.25,   0.75]
+   2             0.75               [0.375,  1.125]
+   3             1.125              [0.562,  1.687]
+   4             1.687              [0.8435, 2.53]
+   5             2.53               [1.265,  3.795]
+   6             3.795              [1.897,  5.692]
+   7             5.692              [2.846,  8.538]
+   8             8.538              [4.269, 12.807]
+   9            12.807              [6.403, 19.210]
+   10           {@link BackOff#STOP}
+ * 
+ * + *

+ * Implementation is not thread-safe. + *

+ */ +public class AttemptBoundedExponentialBackOff implements BackOff { + public static final double DEFAULT_MULTIPLIER = 1.5; + public static final double DEFAULT_RANDOMIZATION_FACTOR = 0.5; + private final int maximumNumberOfAttempts; + private final long initialIntervalMillis; + private int currentAttempt; + + public AttemptBoundedExponentialBackOff(int maximumNumberOfAttempts, long initialIntervalMillis) { + Preconditions.checkArgument(maximumNumberOfAttempts > 0, + "Maximum number of attempts must be greater than zero."); + Preconditions.checkArgument(initialIntervalMillis > 0, + "Initial interval must be greater than zero."); + this.maximumNumberOfAttempts = maximumNumberOfAttempts; + this.initialIntervalMillis = initialIntervalMillis; + reset(); + } + + @Override + public void reset() { + currentAttempt = 1; + } + + @Override + public long nextBackOffMillis() { + if (currentAttempt >= maximumNumberOfAttempts) { + return BackOff.STOP; + } + double currentIntervalMillis = initialIntervalMillis + * Math.pow(DEFAULT_MULTIPLIER, currentAttempt - 1); + double randomOffset = (Math.random() * 2 - 1) + * DEFAULT_RANDOMIZATION_FACTOR * currentIntervalMillis; + currentAttempt += 1; + return Math.round(currentIntervalMillis + randomOffset); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Base64Utils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Base64Utils.java new file mode 100644 index 0000000000000..0ea25102e1321 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Base64Utils.java @@ -0,0 +1,30 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +/** + * Utilities related to Base64 encoding. + */ +public class Base64Utils { + /** + * Returns an upper bound of the length of non-chunked Base64 encoded version + * of the string of the given length. + */ + public static int getBase64Length(int length) { + return 4 * ((length + 2) / 3); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java new file mode 100644 index 0000000000000..2d42407c94377 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java @@ -0,0 +1,157 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; +import com.google.cloud.dataflow.sdk.values.CodedTupleTagMap; + +import org.joda.time.Instant; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * {@link ExecutionContext} for use in batch mode. + */ +public class BatchModeExecutionContext extends ExecutionContext { + private Object key; + private Map> timers = new HashMap<>(); + + /** + * Create a new {@link ExecutionContext.StepContext}. + */ + @Override + public ExecutionContext.StepContext createStepContext(String stepName) { + return new StepContext(stepName); + } + + /** + * Sets the key of the work currently being processed. + */ + public void setKey(Object key) { + this.key = key; + } + + /** + * Returns the key of the work currently being processed. + * + *

If there is not a currently defined key, returns null. + */ + public Object getKey() { + return key; + } + + @Override + public void setTimer(String timer, Instant timestamp) { + Map keyTimers = timers.get(getKey()); + if (keyTimers == null) { + keyTimers = new HashMap<>(); + timers.put(getKey(), keyTimers); + } + keyTimers.put(timer, timestamp); + } + + @Override + public void deleteTimer(String timer) { + Map keyTimers = timers.get(getKey()); + if (keyTimers != null) { + keyTimers.remove(timer); + } + } + + public List> getAllTimers() { + List> result = new ArrayList<>(); + for (Map.Entry> keyTimers : timers.entrySet()) { + for (Map.Entry timer : keyTimers.getValue().entrySet()) { + result.add(TimerOrElement.timer(timer.getKey(), timer.getValue(), keyTimers.getKey())); + } + } + return result; + } + + /** + * {@link ExecutionContext.StepContext} used in batch mode. + */ + class StepContext extends ExecutionContext.StepContext { + private Map, Object>> state = new HashMap<>(); + private Map, List>> tagLists = new HashMap<>(); + + StepContext(String stepName) { + super(stepName); + } + + @Override + public void store(CodedTupleTag tag, T value) { + Map, Object> perKeyState = state.get(getKey()); + if (perKeyState == null) { + perKeyState = new HashMap<>(); + state.put(getKey(), perKeyState); + } + perKeyState.put(tag, value); + } + + @Override + public CodedTupleTagMap lookup(List> tags) { + Map, Object> perKeyState = state.get(getKey()); + Map, Object> map = new HashMap<>(); + if (perKeyState != null) { + for (CodedTupleTag tag : tags) { + map.put(tag, perKeyState.get(tag)); + } + } + return CodedTupleTagMap.of(map); + } + + @Override + public void writeToTagList(CodedTupleTag tag, T value, Instant timestamp) { + Map, List> perKeyTagLists = tagLists.get(getKey()); + if (perKeyTagLists == null) { + perKeyTagLists = new HashMap<>(); + tagLists.put(getKey(), perKeyTagLists); + } + List tagList = perKeyTagLists.get(tag); + if (tagList == null) { + tagList = new ArrayList<>(); + perKeyTagLists.put(tag, tagList); + } + tagList.add(value); + } + + @Override + public void deleteTagList(CodedTupleTag tag) { + Map, List> perKeyTagLists = tagLists.get(getKey()); + if (perKeyTagLists != null) { + perKeyTagLists.remove(tag); + } + } + + @Override + public Iterable readTagList(CodedTupleTag tag) { + Map, List> perKeyTagLists = tagLists.get(getKey()); + if (perKeyTagLists == null || perKeyTagLists.get(tag) == null) { + return new ArrayList(); + } + List result = new ArrayList(); + for (Object element : perKeyTagLists.get(tag)) { + result.add((T) element); + } + return result; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableInserter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableInserter.java new file mode 100644 index 0000000000000..c241ee2f25912 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableInserter.java @@ -0,0 +1,240 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableDataInsertAllRequest; +import com.google.api.services.bigquery.model.TableDataInsertAllResponse; +import com.google.api.services.bigquery.model.TableDataList; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition; +import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.WriteDisposition; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * Inserts rows into BigQuery. + */ +public class BigQueryTableInserter { + private static final Logger LOG = LoggerFactory.getLogger(BigQueryTableInserter.class); + + // Approximate amount of table data to upload per InsertAll request. + private static final long UPLOAD_BATCH_SIZE = 64 * 1024; + + private final Bigquery client; + private final TableReference ref; + + /** + * Constructs a new row inserter. + * + * @param client a BigQuery client + * @param ref identifies the table to insert into + */ + public BigQueryTableInserter(Bigquery client, TableReference ref) { + this.client = client; + this.ref = ref; + } + + /** + * Insert all rows from the given iterator. + */ + public void insertAll(Iterator rowIterator) throws IOException { + insertAll(rowIterator, null); + } + + /** + * Insert all rows from the given iterator using specified insertIds if not null. + */ + public void insertAll(Iterator rowIterator, + @Nullable Iterator insertIdIterator) throws IOException { + // Upload in batches. + List rows = new LinkedList<>(); + int numInserted = 0; + int dataSize = 0; + while (rowIterator.hasNext()) { + TableRow row = rowIterator.next(); + TableDataInsertAllRequest.Rows out = new TableDataInsertAllRequest.Rows(); + if (insertIdIterator != null) { + if (insertIdIterator.hasNext()) { + out.setInsertId(insertIdIterator.next()); + } else { + throw new AssertionError("If insertIdIterator is not null it needs to have at least " + + "as many elements as rowIterator"); + } + } + out.setJson(row.getUnknownKeys()); + rows.add(out); + + dataSize += row.toString().length(); + if (dataSize >= UPLOAD_BATCH_SIZE || !rowIterator.hasNext()) { + TableDataInsertAllRequest content = new TableDataInsertAllRequest(); + content.setRows(rows); + + LOG.info("Number of rows in BigQuery insert: {}", rows.size()); + numInserted += rows.size(); + + Bigquery.Tabledata.InsertAll insert = client.tabledata() + .insertAll(ref.getProjectId(), ref.getDatasetId(), ref.getTableId(), + content); + TableDataInsertAllResponse response = insert.execute(); + List errors = response + .getInsertErrors(); + if (errors != null && !errors.isEmpty()) { + throw new IOException("Insert failed: " + errors); + } + + dataSize = 0; + rows.clear(); + } + } + + LOG.info("Number of rows written to BigQuery: {}", numInserted); + } + + /** + * Retrieves or creates the table. + *

+ * The table is checked to conform to insertion requirements as specified + * by WriteDisposition and CreateDisposition. + *

+ * If table truncation is requested (WriteDisposition.WRITE_TRUNCATE), then + * this will re-create the table if necessary to ensure it is empty. + *

+ * If an empty table is required (WriteDisposition.WRITE_EMPTY), then this + * will fail if the table exists and is not empty. + *

+ * When constructing a table, a {@code TableSchema} must be available. If a + * schema is provided, then it will be used. If no schema is provided, but + * an existing table is being cleared (WRITE_TRUNCATE option above), then + * the existing schema will be re-used. If no schema is available, then an + * {@code IOException} is thrown. + */ + public Table getOrCreateTable( + WriteDisposition writeDisposition, + CreateDisposition createDisposition, + @Nullable TableSchema schema) throws IOException { + // Check if table already exists. + Bigquery.Tables.Get get = client.tables() + .get(ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); + Table table = null; + try { + table = get.execute(); + } catch (IOException e) { + ApiErrorExtractor errorExtractor = new ApiErrorExtractor(); + if (!errorExtractor.itemNotFound(e) || + createDisposition != CreateDisposition.CREATE_IF_NEEDED) { + // Rethrow. + throw e; + } + } + + // If we want an empty table, and it isn't, then delete it first. + if (table != null) { + if (writeDisposition == WriteDisposition.WRITE_APPEND) { + return table; + } + + boolean empty = isEmpty(); + if (empty) { + if (writeDisposition == WriteDisposition.WRITE_TRUNCATE) { + LOG.info("Empty table found, not removing {}", BigQueryIO.toTableSpec(ref)); + } + return table; + + } else if (writeDisposition == WriteDisposition.WRITE_EMPTY) { + throw new IOException("WriteDisposition is WRITE_EMPTY, " + + "but table is not empty"); + } + + // Reuse the existing schema if none was provided. + if (schema == null) { + schema = table.getSchema(); + } + + // Delete table and fall through to re-creating it below. + LOG.info("Deleting table {}", BigQueryIO.toTableSpec(ref)); + Bigquery.Tables.Delete delete = client.tables() + .delete(ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); + delete.execute(); + } + + if (schema == null) { + throw new IllegalArgumentException( + "Table schema required for new table."); + } + + // Create the table. + return tryCreateTable(schema); + } + + /** + * Checks if a table is empty. + */ + public boolean isEmpty() throws IOException { + Bigquery.Tabledata.List list = client.tabledata() + .list(ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); + list.setMaxResults(1L); + TableDataList dataList = list.execute(); + + return dataList.getRows() == null || dataList.getRows().isEmpty(); + } + + /** + * Tries to create the BigQuery table. + * If a table with the same name already exists in the dataset, the table + * creation fails, and the function returns null. In such a case, + * the existing table doesn't necessarily have the same schema as specified + * by the parameter. + * + * @param schema Schema of the new BigQuery table. + * @return The newly created BigQuery table information, or null if the table + * with the same name already exists. + * @throws IOException if other error than already existing table occurs. + */ + @Nullable + public Table tryCreateTable(TableSchema schema) throws IOException { + LOG.info("Trying to create BigQuery table: {}", BigQueryIO.toTableSpec(ref)); + + Table content = new Table(); + content.setTableReference(ref); + content.setSchema(schema); + + try { + return client.tables() + .insert(ref.getProjectId(), ref.getDatasetId(), content) + .execute(); + } catch (IOException e) { + if (new ApiErrorExtractor().alreadyExists(e)) { + LOG.info("The BigQuery table already exists."); + return null; + } + throw e; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableRowIterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableRowIterator.java new file mode 100644 index 0000000000000..a6ea658ae3f42 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableRowIterator.java @@ -0,0 +1,201 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.util.Data; +import com.google.api.client.util.Preconditions; +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableCell; +import com.google.api.services.bigquery.model.TableDataList; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Objects; + +/** + * Iterates over all rows in a table. + */ +public class BigQueryTableRowIterator implements Iterator, Closeable { + + private final Bigquery client; + private final TableReference ref; + private TableSchema schema; + private String pageToken; + private Iterator rowIterator; + // Set true when the final page is seen from the service. + private boolean lastPage = false; + + public BigQueryTableRowIterator(Bigquery client, TableReference ref) { + this.client = client; + this.ref = ref; + } + + @Override + public boolean hasNext() { + try { + if (!isOpen()) { + open(); + } + + if (!rowIterator.hasNext() && !lastPage) { + readNext(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + + return rowIterator.hasNext(); + } + + /** + * Adjusts a field returned from the API to + * match the type that will be seen when run on the + * backend service. The end result is: + * + *

    + *
  • Nulls are {@code null}. + *
  • Repeated fields are lists. + *
  • Record columns are {@link TableRow}s. + *
  • {@code BOOLEAN} columns are JSON booleans, hence Java {@link Boolean}s. + *
  • {@code FLOAT} columns are JSON floats, hence Java {@link Double}s. + *
  • Every other atomic type is a {@link String}. + *

+ * + *

Note that currently integers are encoded as strings to match + * the behavior of the backend service. + */ + private Object getTypedCellValue(TableFieldSchema fieldSchema, Object v) { + // In the input from the BQ API, atomic types all come in as + // strings, while on the Dataflow service they have more precise + // types. + + if (Data.isNull(v)) { + return null; + } + + if (Objects.equals(fieldSchema.getMode(), "REPEATED")) { + TableFieldSchema elementSchema = fieldSchema.clone().setMode("REQUIRED"); + List rawValues = (List) v; + List values = new ArrayList(rawValues.size()); + for (Object element : rawValues) { + values.add(getTypedCellValue(elementSchema, element)); + } + return values; + } + + if (fieldSchema.getType().equals("RECORD")) { + return getTypedTableRow(fieldSchema.getFields(), (TableRow) v); + } + + if (fieldSchema.getType().equals("FLOAT")) { + return Double.parseDouble((String) v); + } + + if (fieldSchema.getType().equals("BOOLEAN")) { + return Boolean.parseBoolean((String) v); + } + + return v; + } + + private TableRow getTypedTableRow(List fields, TableRow rawRow) { + List cells = rawRow.getF(); + Preconditions.checkState(cells.size() == fields.size()); + + Iterator cellIt = cells.iterator(); + Iterator fieldIt = fields.iterator(); + + TableRow row = new TableRow(); + while (cellIt.hasNext()) { + TableCell cell = cellIt.next(); + TableFieldSchema fieldSchema = fieldIt.next(); + row.set(fieldSchema.getName(), getTypedCellValue(fieldSchema, cell.getV())); + } + return row; + } + + @Override + public TableRow next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + // Embed schema information into the raw row, so that values have an + // associated key. This matches how rows are read when using the + // DataflowPipelineRunner. + return getTypedTableRow(schema.getFields(), rowIterator.next()); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + private void readNext() throws IOException { + Bigquery.Tabledata.List list = client.tabledata() + .list(ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); + if (pageToken != null) { + list.setPageToken(pageToken); + } + + TableDataList result = list.execute(); + pageToken = result.getPageToken(); + rowIterator = result.getRows() != null ? result.getRows().iterator() : + Collections.emptyIterator(); + + // The server may return a page token indefinitely on a zero-length table. + if (pageToken == null || + result.getTotalRows() != null && result.getTotalRows() == 0) { + lastPage = true; + } + } + + @Override + public void close() throws IOException { + // Prevent any further requests. + lastPage = true; + } + + private boolean isOpen() { + return schema != null; + } + + /** + * Opens the table for read. + * @throws IOException on failure + */ + private void open() throws IOException { + // Get table schema. + Bigquery.Tables.Get get = client.tables() + .get(ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); + Table table = get.execute(); + schema = table.getSchema(); + + // Read the first page of results. + readNext(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java new file mode 100644 index 0000000000000..4801d6d64c3c3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java @@ -0,0 +1,193 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.WindowUtils.bufferTag; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.MapCoder; +import com.google.cloud.dataflow.sdk.coders.SetCoder; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; +import com.google.cloud.dataflow.sdk.values.KV; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * A WindowSet allowing windows to be merged and deleted. + */ +class BufferingWindowSet + extends AbstractWindowSet, W> { + /** + * Tag for storing the merge tree, the data structure that keeps + * track of which windows have been merged together. + */ + private final CodedTupleTag>> mergeTreeTag = + CodedTupleTag.of( + "mergeTree", + MapCoder.of( + windowingFn.windowCoder(), + SetCoder.of(windowingFn.windowCoder()))); + + /** + * A map of live windows to windows that were merged into them. + * + *

The keys of the map correspond to the set of (merged) windows and the values + * are the no-longer-present windows that were merged into the keys. A given + * window can appear in both the key and value of a single entry, but other at + * most once across all keys and values. + */ + private final Map> mergeTree; + + /** + * Used to determine if writing the mergeTree (which is relatively stable) + * is necessary. + */ + private final Map> originalMergeTree; + + protected BufferingWindowSet( + K key, + WindowingFn windowingFn, + Coder inputCoder, + DoFnProcessContext>> context, + ActiveWindowManager activeWindowManager) throws Exception { + super(key, windowingFn, inputCoder, context, activeWindowManager); + + mergeTree = emptyIfNull( + context.context.stepContext.lookup(Arrays.asList(mergeTreeTag)) + .get(mergeTreeTag)); + + originalMergeTree = deepCopy(mergeTree); + } + + @Override + public void put(W window, V value) throws Exception { + context.context.stepContext.writeToTagList( + bufferTag(window, windowingFn.windowCoder(), inputCoder), + value, + context.timestamp()); + if (!mergeTree.containsKey(window)) { + mergeTree.put(window, new HashSet()); + activeWindowManager.addWindow(window); + } + } + + @Override + public void remove(W window) throws Exception { + mergeTree.remove(window); + activeWindowManager.removeWindow(window); + } + + @Override + public void merge(Collection otherWindows, W newWindow) throws Exception { + Set subWindows = mergeTree.get(newWindow); + if (subWindows == null) { + subWindows = new HashSet<>(); + } + for (W other : otherWindows) { + if (!mergeTree.containsKey(other)) { + throw new IllegalArgumentException("Tried to merge a non-existent window: " + other); + } + subWindows.addAll(mergeTree.get(other)); + subWindows.add(other); + remove(other); + } + mergeTree.put(newWindow, subWindows); + activeWindowManager.addWindow(newWindow); + } + + @Override + public Collection windows() { + return Collections.unmodifiableSet(mergeTree.keySet()); + } + + @Override + public boolean contains(W window) { + return mergeTree.containsKey(window); + } + + @Override + protected Iterable finalValue(W window) throws Exception { + if (!contains(window)) { + throw new IllegalStateException("finalValue called for non-existent window"); + } + + List toEmit = new ArrayList<>(); + // This is the set of windows that we're currently emitting. + Set curWindows = new HashSet<>(); + curWindows.add(window); + curWindows.addAll(mergeTree.get(window)); + + // This is the set of unflushed windows (for preservation detection). + Set otherWindows = new HashSet<>(); + for (Map.Entry> entry : mergeTree.entrySet()) { + if (!entry.getKey().equals(window)) { + otherWindows.add(entry.getKey()); + otherWindows.addAll(entry.getValue()); + } + } + + for (W curWindow : curWindows) { + Iterable items = context.context.stepContext.readTagList(bufferTag( + curWindow, windowingFn.windowCoder(), inputCoder)); + for (V item : items) { + toEmit.add(item); + } + context.context.stepContext.deleteTagList(bufferTag( + curWindow, windowingFn.windowCoder(), inputCoder)); + } + + return toEmit; + } + + @Override + public void flush() throws Exception { + if (!mergeTree.equals(originalMergeTree)) { + context.context.stepContext.store(mergeTreeTag, mergeTree); + } + } + + private static Map> emptyIfNull(Map> input) { + if (input == null) { + return new HashMap<>(); + } else { + for (Map.Entry> entry : input.entrySet()) { + if (entry.getValue() == null) { + entry.setValue(new HashSet()); + } + } + return input; + } + } + + private Map> deepCopy(Map> mergeTree) { + Map> newMergeTree = new HashMap<>(); + for (Map.Entry> entry : mergeTree.entrySet()) { + newMergeTree.put(entry.getKey(), new HashSet(entry.getValue())); + } + return newMergeTree; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudCounterUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudCounterUtils.java new file mode 100644 index 0000000000000..f96ba486f24da --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudCounterUtils.java @@ -0,0 +1,104 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.services.dataflow.model.MetricStructuredName; +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Utilities for working with CloudCounters. + */ +public class CloudCounterUtils { + private static final Logger LOG = LoggerFactory.getLogger(CloudCounterUtils.class); + + public static List extractCounters( + CounterSet counters, boolean delta) { + synchronized (counters) { + List cloudCounters = new ArrayList<>(counters.size()); + for (Counter counter : counters) { + try { + MetricUpdate cloudCounter = extractCounter(counter, delta); + if (cloudCounter != null) { + cloudCounters.add(cloudCounter); + } + } catch (IllegalArgumentException exn) { + LOG.warn("Error extracting counter value: ", exn); + } + } + return cloudCounters; + } + } + + public static MetricUpdate extractCounter(Counter counter, boolean delta) { + // TODO: Omit no-op counter updates, for counters whose + // values haven't changed since the last time we sent them. + synchronized (counter) { + MetricStructuredName name = new MetricStructuredName(); + name.setName(counter.getName()); + MetricUpdate metricUpdate = new MetricUpdate() + .setName(name) + .setKind(counter.getKind().name()) + .setCumulative(!delta); + switch (counter.getKind()) { + case SUM: + case MAX: + case MIN: + case AND: + case OR: + metricUpdate.setScalar(CloudObject.forKnownType(counter.getAggregate(delta))); + break; + case MEAN: { + long countUpdate = counter.getCount(delta); + if (countUpdate <= 0) { + return null; + } + metricUpdate.setMeanSum(CloudObject.forKnownType(counter.getAggregate(delta))); + metricUpdate.setMeanCount(CloudObject.forKnownType(countUpdate)); + break; + } + case SET: { + Set values = counter.getSet(delta); + if (values.isEmpty()) { + return null; + } + Set encodedSet = new HashSet(values.size()); + for (Object value : values) { + encodedSet.add(CloudObject.forKnownType(value)); + } + metricUpdate.setSet(encodedSet); + break; + } + default: + throw new IllegalArgumentException("unexpected kind of counter"); + } + if (delta) { + counter.resetDelta(); + } + return metricUpdate; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudKnownType.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudKnownType.java new file mode 100644 index 0000000000000..ad57b99536313 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudKnownType.java @@ -0,0 +1,138 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import javax.annotation.Nullable; + +/** A utility for manipulating well-known cloud types. */ +enum CloudKnownType { + TEXT("http://schema.org/Text", String.class) { + @Override + public T parse(Object value, Class clazz) { + return clazz.cast(value); + } + }, + BOOLEAN("http://schema.org/Boolean", Boolean.class) { + @Override + public T parse(Object value, Class clazz) { + return clazz.cast(value); + } + }, + INTEGER("http://schema.org/Integer", Long.class, Integer.class) { + @Override + public T parse(Object value, Class clazz) { + Object result = null; + if (value.getClass() == clazz) { + result = value; + } else if (clazz == Long.class) { + if (value instanceof Integer) { + result = ((Integer) value).longValue(); + } else if (value instanceof String) { + result = Long.valueOf((String) value); + } + } else if (clazz == Integer.class) { + if (value instanceof Long) { + result = ((Long) value).intValue(); + } else if (value instanceof String) { + result = Integer.valueOf((String) value); + } + } + return clazz.cast(result); + } + }, + FLOAT("http://schema.org/Float", Double.class, Float.class) { + @Override + public T parse(Object value, Class clazz) { + Object result = null; + if (value.getClass() == clazz) { + result = value; + } else if (clazz == Double.class) { + if (value instanceof Float) { + result = ((Float) value).doubleValue(); + } else if (value instanceof String) { + result = Double.valueOf((String) value); + } + } else if (clazz == Float.class) { + if (value instanceof Double) { + result = ((Double) value).floatValue(); + } else if (value instanceof String) { + result = Float.valueOf((String) value); + } + } + return clazz.cast(result); + } + }; + + private final String uri; + private final Class[] classes; + + private CloudKnownType(String uri, Class... classes) { + this.uri = uri; + this.classes = classes; + } + + public String getUri() { + return uri; + } + + public abstract T parse(Object value, Class clazz); + + public Class defaultClass() { + return classes[0]; + } + + private static final Map typesByUri = + Collections.unmodifiableMap(buildTypesByUri()); + + private static Map buildTypesByUri() { + Map result = new HashMap<>(); + for (CloudKnownType ty : CloudKnownType.values()) { + result.put(ty.getUri(), ty); + } + return result; + } + + @Nullable + public static CloudKnownType forUri(@Nullable String uri) { + if (uri == null) { + return null; + } + return typesByUri.get(uri); + } + + private static final Map, CloudKnownType> typesByClass = + Collections.unmodifiableMap(buildTypesByClass()); + + private static Map, CloudKnownType> buildTypesByClass() { + Map, CloudKnownType> result = new HashMap<>(); + for (CloudKnownType ty : CloudKnownType.values()) { + for (Class clazz : ty.classes) { + result.put(clazz, ty); + } + } + return result; + } + + @Nullable + public static CloudKnownType forClass(Class clazz) { + return typesByClass.get(clazz); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudMetricUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudMetricUtils.java new file mode 100644 index 0000000000000..da99e5b3c3851 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudMetricUtils.java @@ -0,0 +1,73 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.services.dataflow.model.MetricStructuredName; +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.cloud.dataflow.sdk.util.common.Metric; +import com.google.cloud.dataflow.sdk.util.common.Metric.DoubleMetric; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Utilities for working with Dataflow API Metrics. + */ +public class CloudMetricUtils { + // Do not instantiate. + private CloudMetricUtils() {} + + /** + * Returns a List of {@link MetricUpdate}s representing the given Metrics. + */ + public static List extractCloudMetrics( + Collection> metrics, + String workerId) { + List cloudMetrics = new ArrayList<>(metrics.size()); + for (Metric metric : metrics) { + cloudMetrics.add(extractCloudMetric(metric, workerId)); + } + return cloudMetrics; + } + + /** + * Returns a {@link MetricUpdate} representing the given Metric. + */ + public static MetricUpdate extractCloudMetric(Metric metric, String workerId) { + if (metric instanceof DoubleMetric) { + return extractCloudMetric( + metric, + ((DoubleMetric) metric).getValue(), + workerId); + } else { + throw new IllegalArgumentException("unexpected kind of Metric"); + } + } + + private static MetricUpdate extractCloudMetric( + Metric metric, Double value, String workerId) { + MetricStructuredName name = new MetricStructuredName(); + name.setName(metric.getName()); + Map context = new HashMap<>(); + context.put("workerId", workerId); + name.setContext(context); + return new MetricUpdate().setName(name).setScalar(CloudObject.forFloat(value)); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudObject.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudObject.java new file mode 100644 index 0000000000000..973fe5ab7707e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudObject.java @@ -0,0 +1,184 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.api.client.util.Preconditions.checkNotNull; + +import com.google.api.client.json.GenericJson; +import com.google.api.client.util.Key; + +import java.util.Map; + +import javax.annotation.Nullable; + +/** + * A representation of an arbitrary Java object to be instantiated by Dataflow + * workers. + *

+ * Typically, an object to be written by the SDK to the Dataflow service will + * implement a method (typically called {@code asCloudObject()}) which returns a + * {@code CloudObject} to represent the object in the protocol. Once the + * {@code CloudObject} is constructed, the method should explicitly add + * additional properties to be presented during deserialization, representing + * child objects by building additional {@code CloudObject}s. + */ +public final class CloudObject extends GenericJson { + /** + * Constructs a {@code CloudObject} by copying the supplied serialized object spec, + * which must represent an SDK object serialized for transport via the + * Dataflow API. + *

+ * The most common use of this method is during deserialization on the worker, + * where it's used as a binding type during instance construction. + * + * @param spec supplies the serialized form of the object as a nested map + * @throws RuntimeException if the supplied map does not represent an SDK object + */ + public static CloudObject fromSpec(Map spec) { + CloudObject result = new CloudObject(); + result.putAll(spec); + if (result.className == null) { + throw new RuntimeException("Unable to create an SDK object from " + spec + + ": Object class not specified (missing \"" + + PropertyNames.OBJECT_TYPE_NAME + "\" field)"); + } + return result; + } + + /** + * Constructs a {@code CloudObject} to be used for serializing an instance of + * the supplied class for transport via the Dataflow API. The instance + * parameters to be serialized must be supplied explicitly after the + * {@code CloudObject} is created, by using {@link CloudObject#put}. + * + * @param cls the class to use when deserializing the object on the worker + */ + public static CloudObject forClass(Class cls) { + CloudObject result = new CloudObject(); + result.className = checkNotNull(cls).getName(); + return result; + } + + /** + * Constructs a {@code CloudObject} to be used for serializing data to be + * deserialized using the supplied class name the supplied class name for + * transport via the Dataflow API. The instance parameters to be serialized + * must be supplied explicitly after the {@code CloudObject} is created, by + * using {@link CloudObject#put}. + * + * @param className the class to use when deserializing the object on the worker + */ + public static CloudObject forClassName(String className) { + CloudObject result = new CloudObject(); + result.className = checkNotNull(className); + return result; + } + + /** + * Constructs a {@code CloudObject} representing the given value. + * @param value the scalar value to represent. + */ + public static CloudObject forString(String value) { + CloudObject result = forClassName(CloudKnownType.TEXT.getUri()); + result.put(PropertyNames.SCALAR_FIELD_NAME, value); + return result; + } + + /** + * Constructs a {@code CloudObject} representing the given value. + * @param value the scalar value to represent. + */ + public static CloudObject forBoolean(Boolean value) { + CloudObject result = forClassName(CloudKnownType.BOOLEAN.getUri()); + result.put(PropertyNames.SCALAR_FIELD_NAME, value); + return result; + } + + /** + * Constructs a {@code CloudObject} representing the given value. + * @param value the scalar value to represent. + */ + public static CloudObject forInteger(Long value) { + CloudObject result = forClassName(CloudKnownType.INTEGER.getUri()); + result.put(PropertyNames.SCALAR_FIELD_NAME, value); + return result; + } + + /** + * Constructs a {@code CloudObject} representing the given value. + * @param value the scalar value to represent. + */ + public static CloudObject forInteger(Integer value) { + CloudObject result = forClassName(CloudKnownType.INTEGER.getUri()); + result.put(PropertyNames.SCALAR_FIELD_NAME, value); + return result; + } + + /** + * Constructs a {@code CloudObject} representing the given value. + * @param value the scalar value to represent. + */ + public static CloudObject forFloat(Float value) { + CloudObject result = forClassName(CloudKnownType.FLOAT.getUri()); + result.put(PropertyNames.SCALAR_FIELD_NAME, value); + return result; + } + + /** + * Constructs a {@code CloudObject} representing the given value. + * @param value the scalar value to represent. + */ + public static CloudObject forFloat(Double value) { + CloudObject result = forClassName(CloudKnownType.FLOAT.getUri()); + result.put(PropertyNames.SCALAR_FIELD_NAME, value); + return result; + } + + /** + * Constructs a {@code CloudObject} representing the given value of a + * well-known cloud object type. + * @param value the scalar value to represent. + * @throw RuntimeException if the value does not have a {@link CloudKnownType} + * mapping + */ + public static CloudObject forKnownType(Object value) { + @Nullable CloudKnownType ty = CloudKnownType.forClass(value.getClass()); + if (ty == null) { + throw new RuntimeException("Unable to represent value via the Dataflow API: " + value); + } + CloudObject result = forClassName(ty.getUri()); + result.put(PropertyNames.SCALAR_FIELD_NAME, value); + return result; + } + + @Key(PropertyNames.OBJECT_TYPE_NAME) + private String className; + + private CloudObject() {} + + /** + * Gets the name of the Java class which this CloudObject represents. + */ + public String getClassName() { + return className; + } + + @Override + public CloudObject clone() { + return (CloudObject) super.clone(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtils.java new file mode 100644 index 0000000000000..7d97948af437e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtils.java @@ -0,0 +1,80 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.runners.worker.SourceFactory; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Utilities for working with Source Dataflow API definitions and {@link Source} + * objects. + */ +public class CloudSourceUtils { + /** + * Returns a copy of the source with {@code baseSpecs} flattened into {@code spec}. + * On conflict for a parameter name, values in {@code spec} override values in {@code baseSpecs}, + * and later values in {@code baseSpecs} override earlier ones. + */ + public static com.google.api.services.dataflow.model.Source + flattenBaseSpecs(com.google.api.services.dataflow.model.Source source) { + if (source.getBaseSpecs() == null) { + return source; + } + Map params = new HashMap<>(); + for (Map baseSpec : source.getBaseSpecs()) { + params.putAll(baseSpec); + } + params.putAll(source.getSpec()); + + com.google.api.services.dataflow.model.Source result = source.clone(); + result.setSpec(params); + result.setBaseSpecs(null); + return result; + } + + /** Reads all elements from the given {@link Source}. */ + public static List readElemsFromSource(Source source) { + List elems = new ArrayList<>(); + try (Source.SourceIterator it = source.iterator()) { + while (it.hasNext()) { + elems.add(it.next()); + } + } catch (IOException e) { + throw new RuntimeException("Failed to read from source: " + source, e); + } + return elems; + } + + /** + * Creates a {@link Source} from the given Dataflow Source API definition and + * reads all elements from it. + */ + public static List readElemsFromSource( + com.google.api.services.dataflow.model.Source source) { + try { + return readElemsFromSource(SourceFactory.create(null, source, null)); + } catch (Exception e) { + throw new RuntimeException("Failed to read from source: " + source.toString(), e); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CoderUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CoderUtils.java new file mode 100644 index 0000000000000..c77f35a45da2a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CoderUtils.java @@ -0,0 +1,202 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.Structs.addList; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoderBase; +import com.google.cloud.dataflow.sdk.coders.MapCoder; +import com.google.cloud.dataflow.sdk.coders.MapCoderBase; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeInfo.As; +import com.fasterxml.jackson.annotation.JsonTypeInfo.Id; +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.annotation.JsonTypeIdResolver; +import com.fasterxml.jackson.databind.jsontype.impl.TypeIdResolverBase; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.type.TypeFactory; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.lang.reflect.TypeVariable; + +/** + * Utilities for working with Coders. + */ +public final class CoderUtils { + private CoderUtils() {} // Non-instantiable + + /** + * Coder class-name alias for a key-value type. + */ + public static final String KIND_PAIR = "kind:pair"; + + /** + * Coder class-name alias for a stream type. + */ + public static final String KIND_STREAM = "kind:stream"; + + /** + * Encodes the given value using the specified Coder, and returns + * the encoded bytes. + * + * @throws CoderException if there are errors during encoding + */ + public static byte[] encodeToByteArray(Coder coder, T value) + throws CoderException { + try { + try (ByteArrayOutputStream os = new ByteArrayOutputStream()) { + coder.encode(value, os, Coder.Context.OUTER); + return os.toByteArray(); + } + } catch (IOException exn) { + throw new RuntimeException("unexpected IOException", exn); + } + } + + /** + * Decodes the given bytes using the specified Coder, and returns + * the resulting decoded value. + * + * @throws CoderException if there are errors during decoding + */ + public static T decodeFromByteArray(Coder coder, byte[] encodedValue) + throws CoderException { + try { + try (ByteArrayInputStream is = new ByteArrayInputStream(encodedValue)) { + T result = coder.decode(is, Coder.Context.OUTER); + if (is.available() != 0) { + throw new CoderException( + is.available() + " unexpected extra bytes after decoding " + + result); + } + return result; + } + } catch (IOException exn) { + throw new RuntimeException("unexpected IOException", exn); + } + } + + public static CloudObject makeCloudEncoding( + String type, + CloudObject... componentSpecs) { + CloudObject encoding = CloudObject.forClassName(type); + if (componentSpecs.length > 0) { + addList(encoding, PropertyNames.COMPONENT_ENCODINGS, componentSpecs); + } + return encoding; + } + + /** + * A {@link com.fasterxml.jackson.databind.module.Module} which adds the type + * resolver needed for Coder definitions created by the Dataflow service. + */ + static final class Jackson2Module extends SimpleModule { + /** + * The Coder custom type resolver. + *

+ * This resolver resolves coders. If the Coder ID is a particular + * well-known identifier supplied by the Dataflow service, it's replaced + * with the corresponding class. All other Coder instances are resolved + * by class name, using the package com.google.cloud.dataflow.sdk.coders + * if there are no "."s in the ID. + */ + private static final class Resolver extends TypeIdResolverBase { + public Resolver() { + super(TypeFactory.defaultInstance().constructType(Coder.class), + TypeFactory.defaultInstance()); + } + + @Override + public JavaType typeFromId(String id) { + Class clazz = getClassForId(id); + if (clazz == KvCoder.class) { + clazz = KvCoderBase.class; + } + if (clazz == MapCoder.class) { + clazz = MapCoderBase.class; + } + TypeVariable[] tvs = clazz.getTypeParameters(); + JavaType[] types = new JavaType[tvs.length]; + for (int lupe = 0; lupe < tvs.length; lupe++) { + types[lupe] = TypeFactory.unknownType(); + } + return _typeFactory.constructSimpleType(clazz, types); + } + + private Class getClassForId(String id) { + try { + if (id.contains(".")) { + return Class.forName(id); + } + + if (id.equals(KIND_STREAM)) { + return IterableCoder.class; + } else if (id.equals(KIND_PAIR)) { + return KvCoder.class; + } + + // Otherwise, see if the ID is the name of a class in + // com.google.cloud.dataflow.sdk.coders. We do this via creating + // the class object so that class loaders have a chance to get + // involved -- and since we need the class object anyway. + return Class.forName("com.google.cloud.dataflow.sdk.coders." + id); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Unable to convert coder ID " + id + " to class", e); + } + } + + @Override + public String idFromValueAndType(Object o, Class clazz) { + return clazz.getName(); + } + + @Override + public String idFromValue(Object o) { + return o.getClass().getName(); + } + + @Override + public JsonTypeInfo.Id getMechanism() { + return JsonTypeInfo.Id.CUSTOM; + } + } + + /** + * The mixin class defining how Coders are handled by the deserialization + * {@link ObjectMapper}. + *

+ * This is done via a mixin so that this resolver is only used + * during deserialization requested by the Dataflow SDK. + */ + @JsonTypeIdResolver(Resolver.class) + @JsonTypeInfo(use = Id.CUSTOM, include = As.PROPERTY, property = PropertyNames.OBJECT_TYPE_NAME) + private static final class Mixin {} + + public Jackson2Module() { + super("DataflowCoders"); + setMixInAnnotation(Coder.class, Mixin.class); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java new file mode 100644 index 0000000000000..2a24a76fde9f3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java @@ -0,0 +1,244 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.auth.oauth2.Credential; +import com.google.api.client.extensions.java6.auth.oauth2.AbstractPromptReceiver; +import com.google.api.client.extensions.java6.auth.oauth2.AuthorizationCodeInstalledApp; +import com.google.api.client.googleapis.auth.oauth2.GoogleAuthorizationCodeFlow; +import com.google.api.client.googleapis.auth.oauth2.GoogleClientSecrets; +import com.google.api.client.googleapis.auth.oauth2.GoogleCredential; +import com.google.api.client.googleapis.auth.oauth2.GoogleOAuthConstants; +import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport; +import com.google.api.client.http.HttpTransport; +import com.google.api.client.json.JsonFactory; +import com.google.api.client.json.jackson2.JacksonFactory; +import com.google.api.client.util.Preconditions; +import com.google.api.client.util.Strings; +import com.google.api.client.util.store.FileDataStoreFactory; +import com.google.cloud.dataflow.sdk.options.GcpOptions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +/** + * Provides support for loading credentials. + */ +public class Credentials { + + private static final Logger LOG = LoggerFactory.getLogger(Credentials.class); + + /** OAuth 2.0 scopes used by a local worker (not on GCE). + * The scope cloud-platform provides access to all Cloud Platform resources. + * cloud-platform isn't sufficient yet for talking to datastore so we request + * those resources separately. + * + * Note that trusted scope relationships don't apply to OAuth tokens, so for + * services we access directly (GCS) as opposed to through the backend + * (BigQuery, GCE), we need to explicitly request that scope. + */ + private static final List WORKER_SCOPES = Arrays.asList( + "https://www.googleapis.com/auth/cloud-platform", + "https://www.googleapis.com/auth/devstorage.full_control", + "https://www.googleapis.com/auth/userinfo.email", + "https://www.googleapis.com/auth/datastore"); + + private static final List USER_SCOPES = Arrays.asList( + "https://www.googleapis.com/auth/cloud-platform", + "https://www.googleapis.com/auth/devstorage.full_control", + "https://www.googleapis.com/auth/userinfo.email", + "https://www.googleapis.com/auth/datastore"); + + private static class PromptReceiver extends AbstractPromptReceiver { + @Override + public String getRedirectUri() { + return GoogleOAuthConstants.OOB_REDIRECT_URI; + } + } + + /** + * Initializes OAuth2 credential for a worker, using the + * + * application default credentials, or from a local key file when running outside of GCE. + */ + public static Credential getWorkerCredential(GcpOptions options) + throws IOException { + String keyFile = options.getServiceAccountKeyfile(); + String accountName = options.getServiceAccountName(); + + if (keyFile != null && accountName != null) { + try { + return getCredentialFromFile(keyFile, accountName, WORKER_SCOPES); + } catch (GeneralSecurityException e) { + LOG.warn("Unable to obtain credentials from file {}", keyFile); + // Fall through.. + } + } + + return GoogleCredential.getApplicationDefault().createScoped(WORKER_SCOPES); + } + + /** + * Initializes OAuth2 credential for an interactive user program. + * + * This can use 4 different mechanisms for obtaining a credential: + *

    + *
  1. + * It can fetch the + * + * application default credentials. + *
  2. + *
  3. + * It can run the gcloud tool in a subprocess to obtain a credential. + * This is the preferred mechanism. The property "gcloud_path" can be + * used to specify where we search for gcloud data. + *
  4. + *
  5. + * The user can specify a client secrets file and go through the OAuth2 + * webflow. The credential will then be cached in the user's home + * directory for reuse. Provide the property "secrets_file" to use this + * mechanism. + *
  6. + *
  7. + * The user can specify a file containing a service account. + * Provide the properties "service_account_keyfile" and + * "service_account_name" to use this mechanism. + *
  8. + *
+ * The default mechanism is to use the + * + * application default credentials falling back to gcloud. The other options can be + * used by providing the corresponding properties. + */ + public static Credential getUserCredential(GcpOptions options) + throws IOException, GeneralSecurityException { + String keyFile = options.getServiceAccountKeyfile(); + String accountName = options.getServiceAccountName(); + + if (keyFile != null && accountName != null) { + try { + return getCredentialFromFile(keyFile, accountName, USER_SCOPES); + } catch (GeneralSecurityException e) { + throw new IOException("Unable to obtain credentials from file", e); + } + } + + if (options.getSecretsFile() != null) { + return getCredentialFromClientSecrets(options, USER_SCOPES); + } + + try { + return GoogleCredential.getApplicationDefault().createScoped(USER_SCOPES); + } catch (IOException e) { + LOG.info("Failed to get application default credentials, falling back to gcloud."); + } + + String gcloudPath = options.getGCloudPath(); + return getCredentialFromGCloud(gcloudPath); + } + + /** + * Loads OAuth2 credential from a local file. + */ + private static Credential getCredentialFromFile( + String keyFile, String accountId, Collection scopes) + throws IOException, GeneralSecurityException { + GoogleCredential credential = new GoogleCredential.Builder() + .setTransport(Transport.getTransport()) + .setJsonFactory(Transport.getJsonFactory()) + .setServiceAccountId(accountId) + .setServiceAccountScopes(scopes) + .setServiceAccountPrivateKeyFromP12File(new File(keyFile)) + .build(); + + LOG.info("Created credential from file {}", keyFile); + return credential; + } + + /** + * Loads OAuth2 credential from GCloud utility. + */ + private static Credential getCredentialFromGCloud(String gcloudPath) + throws IOException, GeneralSecurityException { + GCloudCredential credential; + HttpTransport transport = GoogleNetHttpTransport.newTrustedTransport(); + if (Strings.isNullOrEmpty(gcloudPath)) { + credential = new GCloudCredential(transport); + } else { + credential = new GCloudCredential(gcloudPath, transport); + } + + try { + credential.refreshToken(); + } catch (IOException e) { + throw new RuntimeException("Could not obtain credential using gcloud", e); + } + + LOG.info("Got credential from GCloud"); + return credential; + } + + /** + * Loads OAuth2 credential from client secrets, which may require an + * interactive authorization prompt. + */ + private static Credential getCredentialFromClientSecrets( + GcpOptions options, Collection scopes) + throws IOException, GeneralSecurityException { + String clientSecretsFile = options.getSecretsFile(); + + Preconditions.checkArgument(clientSecretsFile != null); + HttpTransport httpTransport = GoogleNetHttpTransport.newTrustedTransport(); + + JsonFactory jsonFactory = JacksonFactory.getDefaultInstance(); + GoogleClientSecrets clientSecrets; + + try { + clientSecrets = GoogleClientSecrets.load(jsonFactory, + new FileReader(clientSecretsFile)); + } catch (IOException e) { + throw new RuntimeException( + "Could not read the client secrets from file: " + clientSecretsFile, + e); + } + + FileDataStoreFactory dataStoreFactory = + new FileDataStoreFactory(new java.io.File(options.getCredentialDir())); + + GoogleAuthorizationCodeFlow flow = new GoogleAuthorizationCodeFlow.Builder( + httpTransport, jsonFactory, clientSecrets, scopes) + .setDataStoreFactory(dataStoreFactory) + .build(); + + // The credentialId identifies the credential if we're using a persistent + // credential store. + Credential credential = + new AuthorizationCodeInstalledApp(flow, new PromptReceiver()) + .authorize(options.getCredentialId()); + + LOG.info("Got credential from client secret"); + return credential; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowReleaseInfo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowReleaseInfo.java new file mode 100644 index 0000000000000..ab7e0de6a8e0f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowReleaseInfo.java @@ -0,0 +1,87 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.json.GenericJson; +import com.google.api.client.util.Key; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Properties; + +/** + * Utilities for working with the Dataflow distribution. + */ +public final class DataflowReleaseInfo extends GenericJson { + private static final Logger LOG = LoggerFactory.getLogger(DataflowReleaseInfo.class); + + private static final String DATAFLOW_PROPERTIES_PATH = + "/com/google/cloud/dataflow/sdk/sdk.properties"; + + private static class LazyInit { + private static final DataflowReleaseInfo INSTANCE = + new DataflowReleaseInfo(DATAFLOW_PROPERTIES_PATH); + } + + /** + * Returns an instance of DataflowReleaseInfo. + */ + public static DataflowReleaseInfo getReleaseInfo() { + return LazyInit.INSTANCE; + } + + @Key private String name = "Google Cloud Dataflow Java SDK"; + @Key private String version = "Unknown"; + + /** Provides the SDK name. */ + public String getName() { + return name; + } + + /** Provides the SDK version. */ + public String getVersion() { + return version; + } + + private DataflowReleaseInfo(String resourcePath) { + Properties properties = new Properties(); + + InputStream in = DataflowReleaseInfo.class.getResourceAsStream( + DATAFLOW_PROPERTIES_PATH); + if (in == null) { + LOG.warn("Dataflow properties resource not found: {}", resourcePath); + return; + } + + try { + properties.load(in); + } catch (IOException e) { + LOG.warn("Error loading Dataflow properties resource: ", e); + } + + for (String name : properties.stringPropertyNames()) { + if (name.equals("name")) { + // We don't allow the properties to override the SDK name. + continue; + } + put(name, properties.getProperty(name)); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DirectModeExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DirectModeExecutionContext.java new file mode 100644 index 0000000000000..a157ceefa57ca --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DirectModeExecutionContext.java @@ -0,0 +1,68 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.ValueWithMetadata; + +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * {@link ExecutionContext} for use in direct mode. + */ +public class DirectModeExecutionContext extends BatchModeExecutionContext { + List output = new ArrayList<>(); + Map, List> sideOutputs = new HashMap<>(); + + @Override + public ExecutionContext.StepContext createStepContext(String stepName) { + return new StepContext(stepName); + } + + @Override + public void noteOutput(WindowedValue outputElem) { + output.add(ValueWithMetadata.of(outputElem) + .withKey(getKey())); + } + + @Override + public void noteSideOutput(TupleTag tag, WindowedValue outputElem) { + List output = sideOutputs.get(tag); + if (output == null) { + output = new ArrayList<>(); + sideOutputs.put(tag, output); + } + output.add(ValueWithMetadata.of(outputElem) + .withKey(getKey())); + } + + public List> getOutput(TupleTag tag) { + return (List) output; + } + + public List> getSideOutput(TupleTag tag) { + if (sideOutputs.containsKey(tag)) { + return (List) sideOutputs.get(tag); + } else { + return new ArrayList<>(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java new file mode 100644 index 0000000000000..80d8f34edd04d --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java @@ -0,0 +1,193 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.util.DoFnRunner.OutputManager; +import com.google.cloud.dataflow.sdk.util.ExecutionContext.StepContext; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import org.joda.time.Instant; + +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * A concrete implementation of {@link DoFn.Context} used for running + * a {@link DoFn}. + * + * @param the type of the DoFn's (main) input elements + * @param the type of the DoFn's (main) output elements + * @param the type of object which receives outputs + */ +class DoFnContext extends DoFn.Context { + private static final int MAX_SIDE_OUTPUTS = 1000; + + final PipelineOptions options; + final DoFn fn; + final PTuple sideInputs; + final OutputManager outputManager; + final Map outputMap; + final TupleTag mainOutputTag; + final StepContext stepContext; + final CounterSet.AddCounterMutator addCounterMutator; + + public DoFnContext(PipelineOptions options, + DoFn fn, + PTuple sideInputs, + OutputManager outputManager, + TupleTag mainOutputTag, + List> sideOutputTags, + StepContext stepContext, + CounterSet.AddCounterMutator addCounterMutator) { + fn.super(); + this.options = options; + this.fn = fn; + this.sideInputs = sideInputs; + this.outputManager = outputManager; + this.mainOutputTag = mainOutputTag; + this.outputMap = new HashMap<>(); + outputMap.put(mainOutputTag, outputManager.initialize(mainOutputTag)); + for (TupleTag sideOutputTag : sideOutputTags) { + outputMap.put(sideOutputTag, outputManager.initialize(sideOutputTag)); + } + this.stepContext = stepContext; + this.addCounterMutator = addCounterMutator; + } + + public R getReceiver(TupleTag tag) { + R receiver = outputMap.get(tag); + if (receiver == null) { + throw new IllegalArgumentException( + "calling getReceiver() with unknown tag " + tag); + } + return receiver; + } + + ////////////////////////////////////////////////////////////////////////////// + + @Override + public PipelineOptions getPipelineOptions() { + return options; + } + + @Override + public T sideInput(PCollectionView view) { + TupleTag tag = view.getTagInternal(); + if (!sideInputs.has(tag)) { + throw new IllegalArgumentException( + "calling sideInput() with unknown view; " + + "did you forget to pass the view in " + + "ParDo.withSideInputs()?"); + } + return view.fromIterableInternal((Iterable>) sideInputs.get(tag)); + } + + void outputWindowedValue( + O output, + Instant timestamp, + Collection windows) { + WindowedValue windowedElem = WindowedValue.of(output, timestamp, windows); + outputManager.output(outputMap.get(mainOutputTag), windowedElem); + if (stepContext != null) { + stepContext.noteOutput(windowedElem); + } + } + + protected void sideOutputWindowedValue(TupleTag tag, + T output, + Instant timestamp, + Collection windows) { + R receiver = outputMap.get(tag); + if (receiver == null) { + // This tag wasn't declared nor was it seen before during this execution. + // Thus, this must be a new, undeclared and unconsumed output. + + // To prevent likely user errors, enforce the limit on the number of side + // outputs. + if (outputMap.size() >= MAX_SIDE_OUTPUTS) { + throw new IllegalArgumentException( + "the number of side outputs has exceeded a limit of " + + MAX_SIDE_OUTPUTS); + } + + // Register the new TupleTag with outputManager and add an entry for it in + // the outputMap. + receiver = outputManager.initialize(tag); + outputMap.put(tag, receiver); + } + + WindowedValue windowedElem = WindowedValue.of(output, timestamp, windows); + outputManager.output(receiver, windowedElem); + if (stepContext != null) { + stepContext.noteSideOutput(tag, windowedElem); + } + } + + // Following implementations of output, outputWithTimestamp, and sideOutput + // are only accessible in DoFn.startBundle and DoFn.finishBundle, and will be shadowed by + // ProcessContext's versions in DoFn.processElement. + // TODO: it seems wrong to use Long.MIN_VALUE, since it will violate all our rules about + // DoFns preserving watermarks. + @Override + public void output(O output) { + outputWindowedValue(output, + new Instant(Long.MIN_VALUE), + Arrays.asList(GlobalWindow.Window.INSTANCE)); + } + + @Override + public void outputWithTimestamp(O output, Instant timestamp) { + outputWindowedValue(output, timestamp, Arrays.asList(GlobalWindow.Window.INSTANCE)); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + sideOutputWindowedValue(tag, + output, + new Instant(Long.MIN_VALUE), + Arrays.asList(GlobalWindow.Window.INSTANCE)); + } + + private String generateInternalAggregatorName(String userName) { + return "user-" + stepContext.getStepName() + "-" + userName; + } + + @Override + public Aggregator createAggregator( + String name, Combine.CombineFn combiner) { + return new AggregatorImpl<>(generateInternalAggregatorName(name), combiner, addCounterMutator); + } + + @Override + public Aggregator createAggregator( + String name, SerializableFunction, AO> combiner) { + return new AggregatorImpl, AO>( + generateInternalAggregatorName(name), combiner, addCounterMutator); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnProcessContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnProcessContext.java new file mode 100644 index 0000000000000..d393e6f0b8b66 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnProcessContext.java @@ -0,0 +1,136 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.DoFn.KeyedState; +import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresKeyedState; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import org.joda.time.Instant; + +import java.util.Collection; + +/** + * A concrete implementation of {@link DoFn.ProcessContext} used for running + * a {@link DoFn} over a single element. + * + * @param the type of the DoFn's (main) input elements + * @param the type of the DoFn's (main) output elements + */ +class DoFnProcessContext extends DoFn.ProcessContext { + + final DoFn fn; + final DoFnContext context; + final WindowedValue windowedValue; + + public DoFnProcessContext(DoFn fn, + DoFnContext context, + WindowedValue windowedValue) { + fn.super(); + this.fn = fn; + this.context = context; + this.windowedValue = windowedValue; + } + + @Override + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); + } + + @Override + public I element() { + return windowedValue.getValue(); + } + + @Override + public KeyedState keyedState() { + if (!(fn instanceof RequiresKeyedState) + || (element() != null && !(element() instanceof KV))) { + throw new UnsupportedOperationException( + "Keyed state is only available in the context of a keyed DoFn marked as requiring state"); + } + + return context.stepContext; + } + + @Override + public T sideInput(PCollectionView view) { + return context.sideInput(view); + } + + @Override + public void output(O output) { + context.outputWindowedValue(output, windowedValue.getTimestamp(), windowedValue.getWindows()); + } + + @Override + public void outputWithTimestamp(O output, Instant timestamp) { + Instant originalTimestamp = windowedValue.getTimestamp(); + + if (originalTimestamp != null) { + Preconditions.checkArgument( + !timestamp.isBefore(originalTimestamp.minus(fn.getAllowedTimestampSkew()))); + } + context.outputWindowedValue(output, timestamp, windowedValue.getWindows()); + } + + void outputWindowedValue( + O output, + Instant timestamp, + Collection windows) { + context.outputWindowedValue(output, timestamp, windows); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + context.sideOutputWindowedValue(tag, + output, + windowedValue.getTimestamp(), + windowedValue.getWindows()); + } + + @Override + public Aggregator createAggregator( + String name, Combine.CombineFn combiner) { + return context.createAggregator(name, combiner); + } + + @Override + public Aggregator createAggregator( + String name, SerializableFunction, AO> combiner) { + return context.createAggregator(name, combiner); + } + + @Override + public Instant timestamp() { + return windowedValue.getTimestamp(); + } + + @Override + public Collection windows() { + return windowedValue.getWindows(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java new file mode 100644 index 0000000000000..975af472a4b6d --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java @@ -0,0 +1,147 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.util.ExecutionContext.StepContext; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import java.util.ArrayList; +import java.util.List; + +/** + * Runs a DoFn by constructing the appropriate contexts and passing them in. + * + * @param the type of the DoFn's (main) input elements + * @param the type of the DoFn's (main) output elements + * @param the type of object which receives outputs + */ +public class DoFnRunner { + + /** Information about how to create output receivers and output to them. */ + public interface OutputManager { + + /** Returns the receiver to use for a given tag. */ + public R initialize(TupleTag tag); + + /** Outputs a single element to the provided receiver. */ + public void output(R receiver, WindowedValue output); + + } + + /** The DoFn being run. */ + public final DoFn fn; + + /** The context used for running the DoFn. */ + public final DoFnContext context; + + private DoFnRunner(PipelineOptions options, + DoFn fn, + PTuple sideInputs, + OutputManager outputManager, + TupleTag mainOutputTag, + List> sideOutputTags, + StepContext stepContext, + CounterSet.AddCounterMutator addCounterMutator) { + this.fn = fn; + this.context = new DoFnContext<>(options, fn, sideInputs, outputManager, + mainOutputTag, sideOutputTags, stepContext, + addCounterMutator); + } + + public static DoFnRunner create( + PipelineOptions options, + DoFn fn, + PTuple sideInputs, + OutputManager outputManager, + TupleTag mainOutputTag, + List> sideOutputTags, + StepContext stepContext, + CounterSet.AddCounterMutator addCounterMutator) { + return new DoFnRunner<>( + options, fn, sideInputs, outputManager, + mainOutputTag, sideOutputTags, stepContext, addCounterMutator); + } + + public static DoFnRunner createWithListOutputs( + PipelineOptions options, + DoFn fn, + PTuple sideInputs, + TupleTag mainOutputTag, + List> sideOutputTags, + StepContext stepContext, + CounterSet.AddCounterMutator addCounterMutator) { + return create( + options, fn, sideInputs, + new OutputManager() { + @Override + public List initialize(TupleTag tag) { + return new ArrayList<>(); + } + @Override + public void output(List list, WindowedValue output) { + list.add(output); + } + }, + mainOutputTag, sideOutputTags, stepContext, addCounterMutator); + } + + /** Calls {@link DoFn#startBundle}. */ + public void startBundle() { + // This can contain user code. Wrap it in case it throws an exception. + try { + fn.startBundle(context); + } catch (Throwable t) { + // Exception in user code. + throw new UserCodeException(t); + } + } + + /** + * Calls {@link DoFn#processElement} with a ProcessContext containing + * the current element. + */ + public void processElement(WindowedValue elem) { + DoFnProcessContext processContext = new DoFnProcessContext(fn, context, elem); + + // This can contain user code. Wrap it in case it throws an exception. + try { + fn.processElement(processContext); + } catch (Throwable t) { + // Exception in user code. + throw new UserCodeException(t); + } + } + + /** Calls {@link DoFn#finishBundle}. */ + public void finishBundle() { + // This can contain user code. Wrap it in case it throws an exception. + try { + fn.finishBundle(context); + } catch (Throwable t) { + // Exception in user code. + throw new UserCodeException(t); + } + } + + /** Returns the receiver who gets outputs with the provided tag. */ + public R getReceiver(TupleTag tag) { + return context.getReceiver(tag); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ExecutionContext.java new file mode 100644 index 0000000000000..12d0745b67b6c --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ExecutionContext.java @@ -0,0 +1,168 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; +import com.google.cloud.dataflow.sdk.values.CodedTupleTagMap; +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Context about the current execution. This is guaranteed to exist during processing, + * but does not necessarily persist between different batches of work. + */ +public abstract class ExecutionContext { + private Map cachedStepContexts = new HashMap<>(); + + /** + * Returns the {@link StepContext} associated with the given step. + */ + public StepContext getStepContext(String stepName) { + StepContext context = cachedStepContexts.get(stepName); + if (context == null) { + context = createStepContext(stepName); + cachedStepContexts.put(stepName, context); + } + return context; + } + + /** + * Returns a collection view of all of the {@link StepContext}s. + */ + public Collection getAllStepContexts() { + return cachedStepContexts.values(); + } + + /** + * Implementations should override this to create the specific type + * of {@link StepContext} they neeed. + */ + public abstract StepContext createStepContext(String stepName); + + /** + * Writes out a timer to be fired when the watermark reaches the given + * timestamp. Timers are identified by their name, and can be moved + * by calling {@code setTimer} again, or deleted with + * {@link ExecutionContext#deleteTimer}. + */ + public abstract void setTimer(String timer, Instant timestamp); + + /** + * Deletes the given timer. + */ + public abstract void deleteTimer(String timer); + + /** + * Hook for subclasses to implement that will be called whenever + * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#output} + * is called. + */ + public void noteOutput(WindowedValue output) {} + + /** + * Hook for subclasses to implement that will be called whenever + * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#sideOutput} + * is called. + */ + public void noteSideOutput(TupleTag tag, WindowedValue output) {} + + /** + * Per-step, per-key context used for retrieving state. + */ + public abstract class StepContext implements DoFn.KeyedState { + private final String stepName; + + public StepContext(String stepName) { + this.stepName = stepName; + } + + public String getStepName() { + return stepName; + } + + public ExecutionContext getExecutionContext() { + return ExecutionContext.this; + } + + public void noteOutput(WindowedValue output) { + ExecutionContext.this.noteOutput(output); + } + + public void noteSideOutput(TupleTag tag, WindowedValue output) { + ExecutionContext.this.noteSideOutput(tag, output); + } + + /** + * Stores the provided value in per-{@link com.google.cloud.dataflow.sdk.transforms.DoFn}, + * per-key state. This state is in the form of a map from tags to arbitrary + * encodable values. + * + * @throws IOException if encoding the given value fails + */ + public abstract void store(CodedTupleTag tag, T value) throws IOException; + + /** + * Loads the values from the per-{@link com.google.cloud.dataflow.sdk.transforms.DoFn}, + * per-key state corresponding to the given tags. + * + * @throws IOException if decoding any of the requested values fails + */ + public abstract CodedTupleTagMap lookup(List> tags) + throws IOException; + + /** + * Loads the value from the per-{@link com.google.cloud.dataflow.sdk.transforms.DoFn}, + * per-key state corresponding to the given tag. + * + * @throws IOException if decoding the value fails + */ + public T lookup(CodedTupleTag tag) throws IOException { + return lookup(Arrays.asList(tag)).get(tag); + } + + /** + * Writes the provided value to the list of values in stored state corresponding to the + * provided tag. + * + * @throws IOException if encoding the given value fails + */ + public abstract void writeToTagList(CodedTupleTag tag, T value, Instant timestamp) + throws IOException; + + /** + * Deletes the list corresponding to the given tag. + */ + public abstract void deleteTagList(CodedTupleTag tag); + + /** + * Reads the elements of the list in stored state corresponding to the provided tag. + * + * @throws IOException if decoding any of the requested values fails + */ + public abstract Iterable readTagList(CodedTupleTag tag) + throws IOException; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java new file mode 100644 index 0000000000000..71f66ed2f6db5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java @@ -0,0 +1,91 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedOutputStream; +import java.io.File; +import java.io.FileFilter; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.channels.Channels; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.nio.file.PathMatcher; +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; + +/** + * Implements IOChannelFactory for local files. + */ +public class FileIOChannelFactory implements IOChannelFactory { + private static final Logger LOG = LoggerFactory.getLogger(FileIOChannelFactory.class); + + // This implementation only allows for wildcards in the file name. + // The directory portion must exist as-is. + @Override + public Collection match(String spec) throws IOException { + File file = new File(spec); + + File parent = file.getParentFile(); + if (!parent.exists()) { + throw new IOException("Unable to find parent directory of " + spec); + } + + final PathMatcher matcher = + FileSystems.getDefault().getPathMatcher("glob:" + spec); + File[] files = parent.listFiles(new FileFilter() { + @Override + public boolean accept(File pathname) { + return matcher.matches(pathname.toPath()); + } + }); + + List result = new LinkedList<>(); + for (File match : files) { + result.add(match.getPath()); + } + + return result; + } + + @Override + public ReadableByteChannel open(String spec) throws IOException { + LOG.debug("opening file {}", spec); + FileInputStream inputStream = new FileInputStream(spec); + return inputStream.getChannel(); + } + + @Override + public WritableByteChannel create(String spec, String mimeType) + throws IOException { + LOG.debug("creating file {}", spec); + return Channels.newChannel( + new BufferedOutputStream(new FileOutputStream(spec))); + } + + @Override + public long getSizeBytes(String spec) throws IOException { + return Files.size(FileSystems.getDefault().getPath(spec)); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GCloudCredential.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GCloudCredential.java new file mode 100644 index 0000000000000..a3a3fd2eb5bf1 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GCloudCredential.java @@ -0,0 +1,113 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.auth.oauth2.BearerToken; +import com.google.api.client.auth.oauth2.Credential; +import com.google.api.client.auth.oauth2.TokenResponse; +import com.google.api.client.http.HttpTransport; +import com.google.api.client.util.IOUtils; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; + +/** + * A credential object which uses the GCloud command line tool to get + * an access token. + */ +public class GCloudCredential extends Credential { + private static final String DEFAULT_GCLOUD_BINARY = "gcloud"; + private final String binary; + + public GCloudCredential(HttpTransport transport) { + this(DEFAULT_GCLOUD_BINARY, transport); + } + + /** + * Path to the GCloud binary. + */ + public GCloudCredential(String binary, HttpTransport transport) { + super(new Builder(BearerToken.authorizationHeaderAccessMethod()) + .setTransport(transport)); + + this.binary = binary; + } + + private String readStream(InputStream stream) throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + IOUtils.copy(stream, baos); + return baos.toString("UTF-8"); + } + + @Override + protected TokenResponse executeRefreshToken() throws IOException { + TokenResponse response = new TokenResponse(); + + ProcessBuilder builder = new ProcessBuilder(); + // ProcessBuilder will search the path automatically for the binary + // GCLOUD_BINARY. + builder.command(Arrays.asList(binary, "auth", "print-access-token")); + Process process = builder.start(); + + try { + process.waitFor(); + } catch (InterruptedException e) { + throw new RuntimeException( + "Could not obtain an access token using gcloud; timed out waiting " + + "for gcloud."); + } + + if (process.exitValue() != 0) { + String output; + try { + output = readStream(process.getErrorStream()); + } catch (IOException e) { + throw new RuntimeException( + "Could not obtain an access token using gcloud."); + } + + throw new RuntimeException( + "Could not obtain an access token using gcloud. Result of " + + "invoking gcloud was:\n" + output); + } + + String output; + try { + output = readStream(process.getInputStream()); + } catch (IOException e) { + throw new RuntimeException( + "Could not obtain an access token using gcloud. We encountered an " + + "an error trying to read stdout.", e); + } + String[] lines = output.split("\n"); + + if (lines.length != 1) { + throw new RuntimeException( + "Could not obtain an access token using gcloud. Result of " + + "invoking gcloud was:\n" + output); + } + + // Access token should be good for 5 minutes. + Long expiresInSeconds = 5L * 60; + response.setExpiresInSeconds(expiresInSeconds); + response.setAccessToken(output.trim()); + + return response; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsIOChannelFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsIOChannelFactory.java new file mode 100644 index 0000000000000..9ff133261e601 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsIOChannelFactory.java @@ -0,0 +1,75 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.options.GcsOptions; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; + +import java.io.IOException; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; + +/** + * Implements IOChannelFactory for GCS. + */ +public class GcsIOChannelFactory implements IOChannelFactory { + + private final GcsOptions options; + + public GcsIOChannelFactory(GcsOptions options) { + this.options = options; + } + + @Override + public Collection match(String spec) throws IOException { + GcsPath path = GcsPath.fromUri(spec); + GcsUtil util = options.getGcsUtil(); + List matched = util.expand(path); + + List specs = new LinkedList<>(); + for (GcsPath match : matched) { + specs.add(match.toString()); + } + + return specs; + } + + @Override + public ReadableByteChannel open(String spec) throws IOException { + GcsPath path = GcsPath.fromUri(spec); + GcsUtil util = options.getGcsUtil(); + return util.open(path); + } + + @Override + public WritableByteChannel create(String spec, String mimeType) + throws IOException { + GcsPath path = GcsPath.fromUri(spec); + GcsUtil util = options.getGcsUtil(); + return util.create(path, mimeType); + } + + @Override + public long getSizeBytes(String spec) throws IOException { + GcsPath path = GcsPath.fromUri(spec); + GcsUtil util = options.getGcsUtil(); + return util.fileSize(path); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java new file mode 100644 index 0000000000000..c3edd2ac2c338 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java @@ -0,0 +1,277 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.util.Preconditions; +import com.google.api.services.storage.Storage; +import com.google.api.services.storage.model.Objects; +import com.google.api.services.storage.model.StorageObject; +import com.google.cloud.dataflow.sdk.options.DefaultValueFactory; +import com.google.cloud.dataflow.sdk.options.GcsOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.cloud.dataflow.sdk.util.gcsio.GoogleCloudStorageReadChannel; +import com.google.cloud.dataflow.sdk.util.gcsio.GoogleCloudStorageWriteChannel; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.channels.SeekableByteChannel; +import java.nio.channels.WritableByteChannel; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Provides operations on GCS. + * + * TODO: re-implement as a FileSystemProvider? + */ +public class GcsUtil { + /** + * This is a {@link DefaultValueFactory} able to create a {@link GcsUtil} using + * any transport flags specified on the {@link PipelineOptions}. + */ + public static class GcsUtilFactory implements DefaultValueFactory { + /** + * Returns an instance of {@link GcsUtil} based on the + * {@link PipelineOptions}. + *

+ * If no instance has previously been created, one is created and the value + * stored in {@code options}. + */ + @Override + public GcsUtil create(PipelineOptions options) { + GcsOptions gcsOptions = options.as(GcsOptions.class); + LOG.debug("Creating new GcsUtil"); + return new GcsUtil(Transport.newStorageClient(gcsOptions).build(), + gcsOptions.getExecutorService()); + } + } + + private static final Logger LOG = LoggerFactory.getLogger(GcsUtil.class); + + /** Maximum number of items to retrieve per Objects.List request. */ + private static final long MAX_LIST_ITEMS_PER_CALL = 1024; + + /** Matches a glob containing a wildcard, capturing the portion before the first wildcard. */ + private static final Pattern GLOB_PREFIX = Pattern.compile("(?[^*?]*)[*?].*"); + + private static final String WILDCARD = "[\\[\\]*?]"; + private static final String NON_WILDCARD = "[^\\[\\]*?]"; + private static final String NON_DELIMITER = "[^/]"; + private static final String OPTIONAL_WILDCARD_AND_SUFFIX = "(" + WILDCARD + NON_DELIMITER + "*)?"; + + /** + * A {@link Pattern} that matches globs in which every wildcard is interpreted as such, + * assuming a delimiter of {@code '/'}. + * + *

Most importantly, if a {@code '*'} or {@code '?'} occurs before the + * final delimiter it will not be interpreted as a wildcard. + */ + public static final Pattern GCS_READ_PATTERN = Pattern.compile( + NON_WILDCARD + "*" + OPTIONAL_WILDCARD_AND_SUFFIX); + + ///////////////////////////////////////////////////////////////////////////// + + /** Client for the GCS API */ + private final Storage storage; + + // Helper delegate for turning IOExceptions from API calls into higher-level semantics. + private final ApiErrorExtractor errorExtractor = new ApiErrorExtractor(); + + // Exposed for testing. + final ExecutorService executorService; + + private GcsUtil(Storage storageClient, ExecutorService executorService) { + storage = storageClient; + this.executorService = executorService; + } + + /** + * Expands a pattern into matched paths. The input path may contain + * globs (in the last component only!), which are expanded in the result. + * + * TODO: add support for full path matching. + */ + public List expand(GcsPath path) throws IOException { + if (!GCS_READ_PATTERN.matcher(path.getObject()).matches()) { + throw new IllegalArgumentException( + "Unsupported wildcard usage in \"" + path + "\": " + + " all wildcards must occur after the final '/' delimiter."); + } + + Matcher m = GLOB_PREFIX.matcher(path.getObject()); + if (!m.matches()) { + return Arrays.asList(path); + } + + String prefix = m.group("PREFIX"); + Pattern p = Pattern.compile(globToRegexp(path.getObject())); + LOG.info("matching files in bucket {}, prefix {} against pattern {}", + path.getBucket(), prefix, p.toString()); + + Storage.Objects.List listObject = storage.objects().list(path.getBucket()); + listObject.setMaxResults(MAX_LIST_ITEMS_PER_CALL); + listObject.setDelimiter("/"); + listObject.setPrefix(prefix); + + String pageToken = null; + List results = new LinkedList<>(); + do { + if (pageToken != null) { + listObject.setPageToken(pageToken); + } + + Objects objects = listObject.execute(); + Preconditions.checkNotNull(objects); + + if (objects.getItems() == null) { + break; + } + + // Filter + for (StorageObject o : objects.getItems()) { + String name = o.getName(); + // Skip directories, which end with a slash. + if (p.matcher(name).matches() && !name.endsWith("/")) { + LOG.debug("Matched object: {}", name); + results.add(GcsPath.fromObject(o)); + } + } + + pageToken = objects.getNextPageToken(); + } while (pageToken != null); + + return results; + } + + /** + * Returns the file size from GCS, or -1 if the file does not exist. + */ + public long fileSize(GcsPath path) throws IOException { + try { + Storage.Objects.Get getObject = + storage.objects().get(path.getBucket(), path.getObject()); + + StorageObject object = getObject.execute(); + return object.getSize().longValue(); + } catch (IOException e) { + if (errorExtractor.itemNotFound(e)) { + return -1; + } + + // Re-throw any other error. + throw e; + } + } + + /** + * Opens an object in GCS. + * + * Returns a SeekableByteChannel which provides access to data in the bucket. + * + * @param path the GCS filename to read from + * @return a SeekableByteChannel which can read the object data + * @throws IOException + */ + public SeekableByteChannel open(GcsPath path) + throws IOException { + return new GoogleCloudStorageReadChannel(storage, path.getBucket(), + path.getObject(), errorExtractor); + } + + /** + * Creates an object in GCS. + * + * Returns a WritableByteChannel which can be used to write data to the + * object. + * + * @param path the GCS file to write to + * @param type the type of object, eg "text/plain". + * @return a Callable object which encloses the operation. + * @throws IOException + */ + public WritableByteChannel create(GcsPath path, + String type) throws IOException { + return new GoogleCloudStorageWriteChannel( + executorService, + storage, + path.getBucket(), + path.getObject(), + type); + } + + /** + * Expands glob expressions to regular expressions. + * + * @param globExp the glob expression to expand + * @return a string with the regular expression this glob expands to + */ + static String globToRegexp(String globExp) { + StringBuilder dst = new StringBuilder(); + char[] src = globExp.toCharArray(); + int i = 0; + while (i < src.length) { + char c = src[i++]; + switch (c) { + case '*': + dst.append("[^/]*"); + break; + case '?': + dst.append("[^/]"); + break; + case '.': + case '+': + case '{': + case '}': + case '(': + case ')': + case '|': + case '^': + case '$': + // These need to be escaped in regular expressions + dst.append('\\').append(c); + break; + case '\\': + i = doubleSlashes(dst, src, i); + break; + default: + dst.append(c); + break; + } + } + return dst.toString(); + } + + private static int doubleSlashes(StringBuilder dst, char[] src, int i) { + // Emit the next character without special interpretation + dst.append('\\'); + if ((i - 1) != src.length) { + dst.append(src[i]); + i++; + } else { + // A backslash at the very end is treated like an escaped backslash + dst.append('\\'); + } + return i; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java new file mode 100644 index 0000000000000..62ae4875f9651 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java @@ -0,0 +1,359 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.NonMergingWindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.util.common.PeekingReiterator; +import com.google.cloud.dataflow.sdk.util.common.Reiterable; +import com.google.cloud.dataflow.sdk.util.common.Reiterator; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ListMultimap; + +import org.joda.time.Instant; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.PriorityQueue; + +/** + * DoFn that merges windows and groups elements in those windows, optionally + * combining values. + * + * @param key type + * @param input value element type + * @param window type + */ +public class GroupAlsoByWindowsDoFn + extends DoFn>>, KV>> { + // TODO: Add back RequiresKeyed state once that is supported. + + protected WindowingFn windowingFn; + protected Coder inputCoder; + + public GroupAlsoByWindowsDoFn( + WindowingFn windowingFn, + Coder inputCoder) { + this.windowingFn = windowingFn; + this.inputCoder = inputCoder; + } + + @Override + public void processElement(ProcessContext processContext) throws Exception { + DoFnProcessContext>>, KV>> context = + (DoFnProcessContext>>, KV>>) processContext; + + if (windowingFn instanceof NonMergingWindowingFn) { + processElementViaIterators(context); + } else { + processElementViaWindowSet(context); + } + } + + private void processElementViaWindowSet( + DoFnProcessContext>>, KV>> context) + throws Exception { + + K key = context.element().getKey(); + BatchActiveWindowManager activeWindowManager = new BatchActiveWindowManager<>(); + AbstractWindowSet, W> windowSet = + new BufferingWindowSet(key, windowingFn, inputCoder, context, activeWindowManager); + + for (WindowedValue e : context.element().getValue()) { + for (BoundedWindow window : e.getWindows()) { + windowSet.put((W) window, e.getValue()); + } + ((WindowingFn) windowingFn) + .mergeWindows(new AbstractWindowSet.WindowMergeContext(windowSet, windowingFn)); + + maybeOutputWindows(activeWindowManager, windowSet, windowingFn, e.getTimestamp()); + } + + maybeOutputWindows(activeWindowManager, windowSet, windowingFn, null); + + windowSet.flush(); + } + + /** + * Outputs any windows that are complete, with their corresponding elemeents. + * If there are potentially complete windows, try merging windows first. + */ + private void maybeOutputWindows( + BatchActiveWindowManager activeWindowManager, + AbstractWindowSet windowSet, + WindowingFn windowingFn, + Instant nextTimestamp) throws Exception { + if (activeWindowManager.hasMoreWindows() + && (nextTimestamp == null + || activeWindowManager.nextTimestamp().isBefore(nextTimestamp))) { + // There is at least one window ready to emit. Merge now in case that window should be merged + // into a not yet completed one. + ((WindowingFn) windowingFn) + .mergeWindows(new AbstractWindowSet.WindowMergeContext(windowSet, windowingFn)); + } + + while (activeWindowManager.hasMoreWindows() + && (nextTimestamp == null + || activeWindowManager.nextTimestamp().isBefore(nextTimestamp))) { + W window = activeWindowManager.getWindow(); + if (windowSet.contains(window)) { + windowSet.markCompleted(window); + } + } + } + + private void processElementViaIterators( + DoFnProcessContext>>, KV>> context) + throws Exception { + K key = context.element().getKey(); + Iterable> value = context.element().getValue(); + PeekingReiterator> iterator; + + if (value instanceof Collection) { + iterator = new PeekingReiterator<>(new ListReiterator>( + new ArrayList>((Collection>) value), 0)); + } else if (value instanceof Reiterable) { + iterator = new PeekingReiterator(((Reiterable>) value).iterator()); + } else { + throw new IllegalArgumentException( + "Input to GroupAlsoByWindowsDoFn must be a Collection or Reiterable"); + } + + // This ListMultimap is a map of window maxTimestamps to the list of active + // windows with that maxTimestamp. + ListMultimap windows = ArrayListMultimap.create(); + + while (iterator.hasNext()) { + WindowedValue e = iterator.peek(); + for (BoundedWindow window : e.getWindows()) { + // If this window is not already in the active set, emit a new WindowReiterable + // corresponding to this window, starting at this element in the input Reiterable. + if (!windows.containsEntry(window.maxTimestamp(), window)) { + // Iterating through the WindowReiterable may advance iterator as an optimization + // for as long as it detects that there are no new windows. + windows.put(window.maxTimestamp(), window); + context.outputWindowedValue( + KV.of(key, (Iterable) new WindowReiterable(iterator, window)), + window.maxTimestamp(), + Arrays.asList((W) window)); + } + } + // Copy the iterator in case the next DoFn cached its version of the iterator instead + // of immediately iterating through it. + // And, only advance the iterator if the consuming operation hasn't done so. + iterator = iterator.copy(); + if (iterator.hasNext() && iterator.peek() == e) { + iterator.next(); + } + + // Remove all windows with maxTimestamp behind the current timestamp. + Iterator windowIterator = windows.keys().iterator(); + while (windowIterator.hasNext() + && windowIterator.next().isBefore(e.getTimestamp())) { + windowIterator.remove(); + } + } + } + + /** + * {@link Reiterable} representing a view of all elements in a base + * {@link Reiterator} that are in a given window. + */ + private static class WindowReiterable implements Reiterable { + private PeekingReiterator> baseIterator; + private BoundedWindow window; + + public WindowReiterable( + PeekingReiterator> baseIterator, BoundedWindow window) { + this.baseIterator = baseIterator; + this.window = window; + } + + @Override + public Reiterator iterator() { + // We don't copy the baseIterator when creating the first WindowReiterator + // so that the WindowReiterator can advance the baseIterator. We have to + // make a copy afterwards so that future calls to iterator() will start + // at the right spot. + Reiterator result = new WindowReiterator(baseIterator, window); + baseIterator = baseIterator.copy(); + return result; + } + + @Override + public String toString() { + StringBuilder result = new StringBuilder(); + result.append("WR{"); + for (V v : this) { + result.append(v.toString()).append(','); + } + result.append("}"); + return result.toString(); + } + } + + /** + * The {@link Reiterator} used by {@link WindowReiterable}. + */ + private static class WindowReiterator implements Reiterator { + private PeekingReiterator> iterator; + private BoundedWindow window; + + public WindowReiterator(PeekingReiterator> iterator, BoundedWindow window) { + this.iterator = iterator; + this.window = window; + } + + @Override + public Reiterator copy() { + return new WindowReiterator(iterator.copy(), window); + } + + @Override + public boolean hasNext() { + skipToValidElement(); + return (iterator.hasNext() && iterator.peek().getWindows().contains(window)); + } + + @Override + public V next() { + skipToValidElement(); + WindowedValue next = iterator.next(); + if (!next.getWindows().contains(window)) { + throw new NoSuchElementException("No next item in window"); + } + return next.getValue(); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + /** + * Moves the underlying iterator forward until it either points to the next + * element in the correct window, or is past the end of the window. + */ + private void skipToValidElement() { + while (iterator.hasNext()) { + WindowedValue peek = iterator.peek(); + if (!peek.getTimestamp().isBefore(window.maxTimestamp())) { + // We are past the end of this window, so there can't be any more + // elements in this iterator. + break; + } + if (!(peek.getWindows().size() == 1 && peek.getWindows().contains(window))) { + // We have reached new windows; we need to copy the iterator so we don't + // keep advancing the outer loop in processElement. + iterator = iterator.copy(); + } + if (!peek.getWindows().contains(window)) { + // The next element is not in the right window: skip it. + iterator.next(); + } else { + // The next element is in the right window. + break; + } + } + } + } + + /** + * {@link Reiterator} that wraps a {@link List}. + */ + private static class ListReiterator implements Reiterator { + private List list; + private int index; + + public ListReiterator(List list, int index) { + this.list = list; + this.index = index; + } + + @Override + public T next() { + return list.get(index++); + } + + @Override + public boolean hasNext() { + return index < list.size(); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public Reiterator copy() { + return new ListReiterator(list, index); + } + } + + private static class BatchActiveWindowManager + implements AbstractWindowSet.ActiveWindowManager { + // Sort the windows by their end timestamps so that we can efficiently + // ask for the next window that will be completed. + PriorityQueue windows = new PriorityQueue<>(11, new Comparator() { + @Override + public int compare(W w1, W w2) { + return w1.maxTimestamp().compareTo(w2.maxTimestamp()); + } + }); + + @Override + public void addWindow(W window) { + windows.add(window); + } + + @Override + public void removeWindow(W window) { + windows.remove(window); + } + + /** + * Returns whether there are more windows. + */ + public boolean hasMoreWindows() { + return windows.peek() != null; + } + + /** + * Returns the timestamp of the next window + */ + public Instant nextTimestamp() { + return windows.peek().maxTimestamp(); + } + + /** + * Returns and removes the next window. + */ + public W getWindow() { + return windows.poll(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/IOChannelFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/IOChannelFactory.java new file mode 100644 index 0000000000000..683ca76efa5d5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/IOChannelFactory.java @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import java.io.IOException; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; +import java.util.Collection; + +/** + * Defines a factory for working with read and write channels. + * + * Channels provide an abstract API for IO operations. + * + * See FACTORY_MAP = + Collections.synchronizedMap(new HashMap()); + + // Pattern which matches shard placeholders within a shard template. + private static final Pattern SHARD_FORMAT_RE = Pattern.compile("(S+|N+)"); + + /** + * Associates a scheme with an {@link IOChannelFactory}. + * + * The given factory is used to construct read and write channels when + * a URI is provided with the given scheme. + * + * For example, when reading from "gs://bucket/path", the scheme "gs" is + * used to lookup the appropriate factory. + */ + public static void setIOFactory(String scheme, IOChannelFactory factory) { + FACTORY_MAP.put(scheme, factory); + } + + /** + * Registers standard factories globally. This requires {@link PipelineOptions} + * to provide e.g. credentials for GCS. + */ + public static void registerStandardIOFactories(PipelineOptions options) { + setIOFactory("gs", new GcsIOChannelFactory(options.as(GcsOptions.class))); + } + + /** + * Creates a write channel for the given filename. + */ + public static WritableByteChannel create(String filename, String mimeType) + throws IOException { + return getFactory(filename).create(filename, mimeType); + } + + /** + * Creates a write channel for the given file components. + * + *

If numShards is specified, then a ShardingWritableByteChannel is + * returned. + * + *

Shard numbers are 0 based, meaning they start with 0 and end at the + * number of shards - 1. + */ + public static WritableByteChannel create(String prefix, String shardTemplate, + String suffix, int numShards, String mimeType) throws IOException { + if (numShards == 1) { + return create(constructName(prefix, shardTemplate, suffix, 0, 1), + mimeType); + } + + ShardingWritableByteChannel shardingChannel = + new ShardingWritableByteChannel(); + + Set outputNames = new HashSet<>(); + for (int i = 0; i < numShards; i++) { + String outputName = + constructName(prefix, shardTemplate, suffix, i, numShards); + if (!outputNames.add(outputName)) { + throw new IllegalArgumentException( + "Shard name collision detected for: " + outputName); + } + WritableByteChannel channel = create(outputName, mimeType); + shardingChannel.addChannel(channel); + } + + return shardingChannel; + } + + /** + * Constructs a fully qualified name from components. + * + *

The name is built from a prefix, shard template (with shard numbers + * applied), and a suffix. All components are required, but may be empty + * strings. + * + *

Within a shard template, repeating sequences of the letters "S" or "N" + * are replaced with the shard number, or number of shards respectively. The + * numbers are formatted with leading zeros to match the length of the + * repeated sequence of letters. + * + *

For example, if prefix = "output", shardTemplate = "-SSS-of-NNN", and + * suffix = ".txt", with shardNum = 1 and numShards = 100, the following is + * produced: "output-001-of-100.txt". + */ + public static String constructName(String prefix, + String shardTemplate, String suffix, int shardNum, int numShards) { + // Matcher API works with StringBuffer, rather than StringBuilder. + StringBuffer sb = new StringBuffer(); + sb.append(prefix); + + Matcher m = SHARD_FORMAT_RE.matcher(shardTemplate); + while (m.find()) { + boolean isShardNum = (m.group(1).charAt(0) == 'S'); + + char[] zeros = new char[m.end() - m.start()]; + Arrays.fill(zeros, '0'); + DecimalFormat df = new DecimalFormat(String.valueOf(zeros)); + String formatted = df.format(isShardNum + ? shardNum + : numShards); + m.appendReplacement(sb, formatted); + } + m.appendTail(sb); + + sb.append(suffix); + return sb.toString(); + } + + private static final Pattern URI_SCHEME_PATTERN = Pattern.compile( + "(?[a-zA-Z][-a-zA-Z0-9+.]*)://.*"); + + /** + * Returns the IOChannelFactory associated with an input specification. + */ + public static IOChannelFactory getFactory(String spec) throws IOException { + // The spec is almost, but not quite, a URI. In particular, + // the reserved characters '[', ']', and '?' have meanings that differ + // from their use in the URI spec. ('*' is not reserved). + // Here, we just need the scheme, which is so circumscribed as to be + // very easy to extract with a regex. + Matcher matcher = URI_SCHEME_PATTERN.matcher(spec); + + if (!matcher.matches()) { + return new FileIOChannelFactory(); + } + + String scheme = matcher.group("scheme"); + IOChannelFactory ioFactory = FACTORY_MAP.get(scheme); + if (ioFactory != null) { + return ioFactory; + } + + throw new IOException("Unable to find handler for " + spec); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/InstanceBuilder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/InstanceBuilder.java new file mode 100644 index 0000000000000..8712855a86220 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/InstanceBuilder.java @@ -0,0 +1,259 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.util.Preconditions; +import com.google.common.reflect.TypeToken; + +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.util.LinkedList; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * Utility for creating objects dynamically. + * + * @param type type of object returned by this instance builder + */ +public class InstanceBuilder { + + /** + * Create an InstanceBuilder for the given type. + *

+ * The specified type is the type returned by {@link #build}, which is + * typically the common base type or interface of the instance being + * constructed. + */ + public static InstanceBuilder ofType(Class type) { + return new InstanceBuilder<>(type); + } + + /** + * Create an InstanceBuilder for the given type. + *

+ * The specified type is the type returned by {@link #build}, which is + * typically the common base type or interface for the instance to be + * constructed. + *

+ * The TypeToken argument allows specification of generic types. For example, + * a {@code List} return type can be specified as + * {@code ofType(new TypeToken>(){})}. + */ + public static InstanceBuilder ofType(TypeToken token) { + @SuppressWarnings("unchecked") + Class type = (Class) token.getRawType(); + return new InstanceBuilder<>(type); + } + + /** + * Sets the class name to be constructed. + *

+ * If the name is a simple name (ie {@link Class#getSimpleName()}), then + * the package of the return type is added as a prefix. + *

+ * The default class is the return type, specified in {@link #ofType}. + *

+ * Modifies and returns the {@code InstanceBuilder} for chaining. + * + * @throws ClassNotFoundException if no class can be found by the given name + */ + public InstanceBuilder fromClassName(String name) + throws ClassNotFoundException { + Preconditions.checkArgument(factoryClass == null, + "Class name may only be specified once"); + if (name.indexOf('.') == -1) { + name = type.getPackage().getName() + "." + name; + } + + try { + factoryClass = Class.forName(name); + } catch (ClassNotFoundException e) { + throw new ClassNotFoundException( + String.format("Could not find class: %s", name), e); + } + return this; + } + + /** + * Sets the factory class to use for instance construction. + *

+ * Modifies and returns the {@code InstanceBuilder} for chaining. + */ + public InstanceBuilder fromClass(Class factoryClass) { + this.factoryClass = factoryClass; + return this; + } + + /** + * Sets the name of the factory method used to construct the instance. + *

+ * The default, if no factory method was specified, is to look for a class + * constructor. + *

+ * Modifies and returns the {@code InstanceBuilder} for chaining. + */ + public InstanceBuilder fromFactoryMethod(String methodName) { + Preconditions.checkArgument(this.methodName == null, + "Factory method name may only be specified once"); + this.methodName = methodName; + return this; + } + + /** + * Adds an argument to be passed to the factory method. + *

+ * The argument type is used to lookup the factory method. This type may be + * a supertype of the argument value's class. + *

+ * Modifies and returns the {@code InstanceBuilder} for chaining. + */ + public InstanceBuilder withArg(Class argType, A value) { + parameterTypes.add(argType); + arguments.add(value); + return this; + } + + /** + * Creates the instance by calling the factory method with the given + * arguments. + *

+ *

Defaults

+ *
    + *
  • factory class: defaults to the output type class, overridden + * via {@link #fromClassName(String)}. + *
  • factory method: defaults to using a constructor on the factory + * class, overridden via {@link #fromFactoryMethod(String)}. + *
+ * + * @throws RuntimeException if the method does not exist, on type mismatch, + * or if the method cannot be made accessible. + */ + public T build() { + if (factoryClass == null) { + factoryClass = type; + } + + Class[] types = parameterTypes + .toArray(new Class[parameterTypes.size()]); + + // TODO: cache results, to speed repeated type lookups? + if (methodName != null) { + return buildFromMethod(types); + } else { + return buildFromConstructor(types); + } + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Type of object to construct. + */ + private final Class type; + + /** + * Types of parameters for Method lookup. + * + * @see Class#getDeclaredMethod(String, Class[]) + */ + private final List> parameterTypes = new LinkedList<>(); + + /** + * Arguments to factory method {@link Method#invoke(Object, Object...)}. + */ + private final List arguments = new LinkedList<>(); + + /** + * Name of factory method, or null to invoke the constructor. + */ + @Nullable private String methodName; + + /** + * Factory class, or null to instantiate {@code type}. + */ + @Nullable private Class factoryClass; + + private InstanceBuilder(Class type) { + this.type = type; + } + + private T buildFromMethod(Class[] types) { + Preconditions.checkState(factoryClass != null); + Preconditions.checkState(methodName != null); + + try { + Method method = factoryClass.getDeclaredMethod(methodName, types); + + Preconditions.checkState(Modifier.isStatic(method.getModifiers()), + "Factory method must be a static method for " + + factoryClass.getName() + "#" + method.getName() + ); + + Preconditions.checkState(type.isAssignableFrom(method.getReturnType()), + "Return type for " + factoryClass.getName() + "#" + method.getName() + + " must be assignable to " + type.getSimpleName()); + + if (!method.isAccessible()) { + method.setAccessible(true); + } + + Object[] args = arguments.toArray(new Object[arguments.size()]); + return type.cast(method.invoke(null, args)); + + } catch (NoSuchMethodException e) { + throw new RuntimeException("Unable to find factory method " + + factoryClass.getName() + "#" + methodName); + + } catch (IllegalAccessException | InvocationTargetException e) { + throw new RuntimeException("Failed to construct instance from " + + "factory method " + factoryClass.getName() + "#" + methodName, e); + } + } + + private T buildFromConstructor(Class[] types) { + Preconditions.checkState(factoryClass != null); + + try { + Constructor constructor = factoryClass.getDeclaredConstructor(types); + + Preconditions.checkState(type.isAssignableFrom(factoryClass), + "Instance type " + factoryClass.getName() + + " must be assignable to " + type.getSimpleName()); + + if (!constructor.isAccessible()) { + constructor.setAccessible(true); + } + + Object[] args = arguments.toArray(new Object[arguments.size()]); + return type.cast(constructor.newInstance(args)); + + } catch (NoSuchMethodException e) { + throw new RuntimeException("Unable to find constructor for " + + factoryClass.getName()); + + } catch (InvocationTargetException | + InstantiationException | + IllegalAccessException e) { + throw new RuntimeException("Failed to construct instance from " + + "constructor " + factoryClass.getName(), e); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/MimeTypes.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/MimeTypes.java new file mode 100644 index 0000000000000..3318a150662ac --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/MimeTypes.java @@ -0,0 +1,23 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +/** Constants representing various mime types. */ +public class MimeTypes { + public static final String TEXT = "text/plain"; + public static final String BINARY = "application/octet-stream"; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/MonitoringUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/MonitoringUtil.java new file mode 100644 index 0000000000000..89df25c391112 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/MonitoringUtil.java @@ -0,0 +1,230 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime; + +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.Dataflow.V1b3.Projects.Jobs.Messages; +import com.google.api.services.dataflow.model.JobMessage; +import com.google.api.services.dataflow.model.ListJobMessagesResponse; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.PrintStream; +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import javax.annotation.Nullable; + +/** + * A helper class for monitoring jobs submitted to the service. + */ +public final class MonitoringUtil { + private String projectId; + private Messages messagesClient; + + /** Named constants for common values for the job state. */ + public static enum JobState { + UNKNOWN ("JOB_STATE_UNKNOWN", false), + STOPPED ("JOB_STATE_STOPPED", false), + RUNNING ("JOB_STATE_RUNNING", false), + DONE ("JOB_STATE_DONE", true), + FAILED ("JOB_STATE_FAILED", true), + CANCELLED("JOB_STATE_CANCELLED", true); + + private final String stateName; + private final boolean terminal; + + private JobState(String stateName, boolean terminal) { + this.stateName = stateName; + this.terminal = terminal; + } + + public final String getStateName() { + return stateName; + } + + public final boolean isTerminal() { + return terminal; + } + + private static final Map statesByName = + Collections.unmodifiableMap(buildStatesByName()); + + private static Map buildStatesByName() { + Map result = new HashMap<>(); + for (JobState state : JobState.values()) { + result.put(state.getStateName(), state); + } + return result; + } + + public static JobState toState(String stateName) { + @Nullable JobState state = statesByName.get(stateName); + if (state == null) { + state = UNKNOWN; + } + return state; + } + } + + /** + * An interface which can be used for defining callbacks to receive a list + * of JobMessages containing monitoring information. + */ + public interface JobMessagesHandler { + /** Process the rows. */ + void process(List messages); + } + + /** A handler which prints monitoring messages to a stream. */ + public static class PrintHandler implements JobMessagesHandler { + private PrintStream out; + + /** + * Construct the handler. + * + * @param stream The stream to write the messages to. + */ + public PrintHandler(PrintStream stream) { + out = stream; + } + + @Override + public void process(List messages) { + for (JobMessage message : messages) { + StringBuilder sb = new StringBuilder(); + if (message.getMessageText() != null && !message.getMessageText().isEmpty()) { + if (message.getMessageImportance() != null) { + if (message.getMessageImportance().equals("ERROR")) { + sb.append("Error: "); + } else if (message.getMessageImportance().equals("WARNING")) { + sb.append("Warning: "); + } + } + // TODO: Allow filtering out overly detailed messages. + sb.append(message.getMessageText()); + } + if (sb.length() > 0) { + @Nullable Instant time = fromCloudTime(message.getTime()); + if (time == null) { + out.print("UNKNOWN TIMESTAMP: "); + } else { + out.print(time + ": "); + } + out.println(sb.toString()); + } + } + out.flush(); + } + } + + /** Construct a helper for monitoring. */ + public MonitoringUtil(String projectId, Dataflow dataflow) { + this(projectId, dataflow.v1b3().projects().jobs().messages()); + } + + // @VisibleForTesting + MonitoringUtil(String projectId, Messages messagesClient) { + this.projectId = projectId; + this.messagesClient = messagesClient; + } + + /** + * Comparator for sorting rows in increasing order based on timestamp. + */ + public static class TimeStampComparator implements Comparator { + @Override + public int compare(JobMessage o1, JobMessage o2) { + @Nullable Instant t1 = fromCloudTime(o1.getTime()); + if (t1 == null) { + return -1; + } + @Nullable Instant t2 = fromCloudTime(o2.getTime()); + if (t2 == null) { + return 1; + } + return t1.compareTo(t2); + } + } + + /** + * Return job messages sorted in ascending order by timestamp. + * @param jobId The id of the job to get the messages for. + * @param startTimestampMs Return only those messages with a + * timestamp greater than this value. + * @return collection of messages + * @throws IOException + */ + public ArrayList getJobMessages( + String jobId, long startTimestampMs) throws IOException { + Instant startTimestamp = new Instant(startTimestampMs); + ArrayList allMessages = new ArrayList<>(); + String pageToken = null; + while (true) { + Messages.List listRequest = messagesClient.list(projectId, jobId); + if (pageToken != null) { + listRequest.setPageToken(pageToken); + } + ListJobMessagesResponse response = listRequest.execute(); + + if (response == null || response.getJobMessages() == null) { + return allMessages; + } + + for (JobMessage m : response.getJobMessages()) { + @Nullable Instant timestamp = fromCloudTime(m.getTime()); + if (timestamp == null) { + continue; + } + if (timestamp.isAfter(startTimestamp)) { + allMessages.add(m); + } + } + + if (response.getNextPageToken() == null) { + break; + } else { + pageToken = response.getNextPageToken(); + } + } + + Collections.sort(allMessages, new TimeStampComparator()); + return allMessages; + } + + public static String getJobMonitoringPageURL(String projectName, String jobId) { + try { + // Project name is allowed in place of the project id: the user will be redirected to a URL + // that has the project name replaced with project id. + return String.format( + "https://console.developers.google.com/project/%s/dataflow/job/%s", + URLEncoder.encode(projectName, "UTF-8"), + URLEncoder.encode(jobId, "UTF-8")); + } catch (UnsupportedEncodingException e) { + // Should never happen. + throw new AssertionError("UTF-8 encoding is not supported by the environment", e); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/OutputReference.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/OutputReference.java new file mode 100644 index 0000000000000..eade03d252041 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/OutputReference.java @@ -0,0 +1,42 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.api.client.util.Preconditions.checkNotNull; + +import com.google.api.client.json.GenericJson; +import com.google.api.client.util.Key; + +/** + * A representation used by {@link com.google.api.services.dataflow.model.Step}s + * to reference the output of other {@code Step}s. + */ +public final class OutputReference extends GenericJson { + @Key("@type") + public final String type = "OutputReference"; + + @Key("step_name") + private final String stepName; + + @Key("output_name") + private final String outputName; + + public OutputReference(String stepName, String outputName) { + this.stepName = checkNotNull(stepName); + this.outputName = checkNotNull(outputName); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PTuple.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PTuple.java new file mode 100644 index 0000000000000..98fe4606807a5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PTuple.java @@ -0,0 +1,152 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * A {@code PTuple} is an immutable tuple of + * heterogeneously-typed values, "keyed" by {@link TupleTag}s. + * + *

PTuples can be created and accessed like follows: + *

 {@code
+ * String v1 = ...;
+ * Integer v2 = ...;
+ * Iterable v3 = ...;
+ *
+ * // Create TupleTags for each of the values to put in the
+ * // PTuple (the type of the TupleTag enables tracking the
+ * // static type of each of the values in the PTuple):
+ * TupleTag tag1 = new TupleTag<>();
+ * TupleTag tag2 = new TupleTag<>();
+ * TupleTag> tag3 = new TupleTag<>();
+ *
+ * // Create a PTuple with three values:
+ * PTuple povs =
+ *     PTuple.of(tag1, v1)
+ *         .and(tag2, v2)
+ *         .and(tag3, v3);
+ *
+ * // Create an empty PTuple:
+ * Pipeline p = ...;
+ * PTuple povs2 = PTuple.empty(p);
+ *
+ * // Get values out of a PTuple, using the same tags
+ * // that were used to put them in:
+ * Integer vX = povs.get(tag2);
+ * String vY = povs.get(tag1);
+ * Iterable vZ = povs.get(tag3);
+ *
+ * // Get a map of all values in a PTuple:
+ * Map, ?> allVs = povs.getAll();
+ * } 
+ */ +public class PTuple { + /** + * Returns an empty PTuple. + * + *

Longer PTuples can be created by calling + * {@link #and} on the result. + */ + public static PTuple empty() { + return new PTuple(); + } + + /** + * Returns a singleton PTuple containing the given + * value keyed by the given TupleTag. + * + *

Longer PTuples can be created by calling + * {@link #and} on the result. + */ + public static PTuple of(TupleTag tag, V value) { + return empty().and(tag, value); + } + + /** + * Returns a new PTuple that has all the values and + * tags of this PTuple plus the given value and tag. + * + *

The given TupleTag should not already be mapped to a + * value in this PTuple. + */ + public PTuple and(TupleTag tag, V value) { + Map, Object> newMap = new LinkedHashMap, Object>(); + newMap.putAll(valueMap); + newMap.put(tag, value); + return new PTuple(newMap); + } + + /** + * Returns whether this PTuple contains a value with + * the given tag. + */ + public boolean has(TupleTag tag) { + return valueMap.containsKey(tag); + } + + /** + * Returns the value with the given tag in this + * PTuple. Throws IllegalArgumentException if there is no + * such value, i.e., {@code !has(tag)}. + */ + public V get(TupleTag tag) { + if (!has(tag)) { + throw new IllegalArgumentException( + "TupleTag not found in this PTuple"); + } + @SuppressWarnings("unchecked") + V value = (V) valueMap.get(tag); + return value; + } + + /** + * Returns an immutable Map from TupleTag to corresponding + * value, for all the members of this PTuple. + */ + public Map, ?> getAll() { + return valueMap; + } + + + ///////////////////////////////////////////////////////////////////////////// + // Internal details below here. + + private final Map, ?> valueMap; + + private PTuple() { + this(new LinkedHashMap()); + } + + private PTuple(Map, ?> valueMap) { + this.valueMap = Collections.unmodifiableMap(valueMap); + } + + /** + * Returns a PTuple with each of the given tags mapping + * to the corresponding value. + * + *

For internal use only. + */ + public static PTuple ofInternal(Map, ?> valueMap) { + return new PTuple(valueMap); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java new file mode 100644 index 0000000000000..c108ceb4f1571 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java @@ -0,0 +1,307 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.Sleeper; +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.common.collect.TreeTraverser; +import com.google.common.hash.Funnels; +import com.google.common.hash.Hasher; +import com.google.common.hash.Hashing; +import com.google.common.io.ByteStreams; +import com.google.common.io.CountingOutputStream; +import com.google.common.io.Files; + +import com.fasterxml.jackson.core.Base64Variants; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.zip.ZipEntry; +import java.util.zip.ZipOutputStream; + +/** Helper routines for packages. */ +public class PackageUtil { + private static final Logger LOG = LoggerFactory.getLogger(PackageUtil.class); + /** + * The initial interval to use between package staging attempts. + */ + private static final long INITIAL_BACKOFF_INTERVAL_MS = 5000L; + /** + * The maximum number of attempts when staging a file. + */ + private static final int MAX_ATTEMPTS = 5; + + /** + * Creates a DataflowPackage containing information about how a classpath element should be + * staged. + * + * @param classpathElement The local path for the classpath element. + * @param stagingDirectory The base location in GCS for staged classpath elements. + * @param overridePackageName If non-null, use the given value as the package name + * instead of generating one automatically. + * @return The package. + */ + public static DataflowPackage createPackage(String classpathElement, + GcsPath stagingDirectory, String overridePackageName) { + try { + File file = new File(classpathElement); + String contentHash = computeContentHash(file); + + // Drop the directory prefixes, and form the filename + hash + extension. + String uniqueName = getUniqueContentName(file, contentHash); + + GcsPath stagingPath = stagingDirectory.resolve(uniqueName); + + DataflowPackage target = new DataflowPackage(); + target.setName(overridePackageName != null ? overridePackageName : uniqueName); + target.setLocation(stagingPath.toResourceName()); + return target; + } catch (IOException e) { + throw new RuntimeException("Package setup failure for " + classpathElement, e); + } + } + + /** + * Transfers the classpath elements to GCS. + * + * @param gcsUtil GCS utility. + * @param classpathElements The elements to stage onto GCS. + * @param gcsStaging The path on GCS to stage the classpath elements to. + * @return A list of cloud workflow packages, each representing a classpath element. + */ + public static List stageClasspathElementsToGcs( + GcsUtil gcsUtil, + Collection classpathElements, + GcsPath gcsStaging) { + return stageClasspathElementsToGcs(gcsUtil, classpathElements, gcsStaging, Sleeper.DEFAULT); + } + + // Visible for testing. + static List stageClasspathElementsToGcs( + GcsUtil gcsUtil, + Collection classpathElements, + GcsPath gcsStaging, + Sleeper retrySleeper) { + ArrayList packages = new ArrayList<>(); + + if (gcsStaging == null) { + throw new IllegalArgumentException( + "Can't stage classpath elements on GCS because no GCS location has been provided"); + } + + for (String classpathElement : classpathElements) { + String packageName = null; + if (classpathElement.contains("=")) { + String[] components = classpathElement.split("=", 2); + packageName = components[0]; + classpathElement = components[1]; + } + + DataflowPackage workflowPackage = createPackage( + classpathElement, gcsStaging, packageName); + + packages.add(workflowPackage); + GcsPath target = GcsPath.fromResourceName(workflowPackage.getLocation()); + + // TODO: Should we attempt to detect the Mime type rather than + // always using MimeTypes.BINARY? + try { + long remoteLength = gcsUtil.fileSize(target); + if (remoteLength >= 0 && remoteLength == getClasspathElementLength(classpathElement)) { + LOG.info("Skipping classpath element already on gcs: {} at {}", classpathElement, target); + continue; + } + + // Upload file, retrying on failure. + BackOff backoff = new AttemptBoundedExponentialBackOff( + MAX_ATTEMPTS, + INITIAL_BACKOFF_INTERVAL_MS); + while (true) { + try { + LOG.info("Uploading classpath element {} to {}", classpathElement, target); + try (WritableByteChannel writer = gcsUtil.create(target, MimeTypes.BINARY)) { + copyContent(classpathElement, writer); + } + break; + } catch (IOException e) { + if (BackOffUtils.next(retrySleeper, backoff)) { + LOG.warn("Upload attempt failed, will retry staging of classpath: {}", + classpathElement, e); + } else { + // Rethrow last error, to be included as a cause in the catch below. + LOG.error("Upload failed, will NOT retry staging of classpath: {}", + classpathElement, e); + throw e; + } + } + } + } catch (Exception e) { + throw new RuntimeException("Could not stage classpath element: " + classpathElement, e); + } + } + + return packages; + } + + /** + * If classpathElement is a file, then the files length is returned, otherwise the length + * of the copied stream is returned. + * + * @param classpathElement The local path for the classpath element. + * @return The length of the classpathElement. + */ + private static long getClasspathElementLength(String classpathElement) throws IOException { + File file = new File(classpathElement); + if (file.isFile()) { + return file.length(); + } + + CountingOutputStream countingOutputStream = + new CountingOutputStream(ByteStreams.nullOutputStream()); + try (WritableByteChannel channel = Channels.newChannel(countingOutputStream)) { + copyContent(classpathElement, channel); + } + return countingOutputStream.getCount(); + } + + /** + * Returns a unique name for a file with a given content hash. + *

+ * Directory paths are removed. Example: + *

+   * dir="a/b/c/d", contentHash="f000" => d-f000.zip
+   * file="a/b/c/d.txt", contentHash="f000" => d-f000.txt
+   * file="a/b/c/d", contentHash="f000" => d-f000
+   * 
+ */ + static String getUniqueContentName(File classpathElement, String contentHash) { + String fileName = Files.getNameWithoutExtension(classpathElement.getAbsolutePath()); + String fileExtension = Files.getFileExtension(classpathElement.getAbsolutePath()); + if (classpathElement.isDirectory()) { + return fileName + "-" + contentHash + ".zip"; + } else if (fileExtension.isEmpty()) { + return fileName + "-" + contentHash; + } + return fileName + "-" + contentHash + "." + fileExtension; + } + + /** + * Computes a message digest of the file/directory contents, returning a base64 string which is + * suitable for use in URLs. + */ + private static String computeContentHash(File classpathElement) throws IOException { + TreeTraverser files = Files.fileTreeTraverser(); + Hasher hasher = Hashing.md5().newHasher(); + for (File currentFile : files.preOrderTraversal(classpathElement)) { + String relativePath = relativize(currentFile, classpathElement); + hasher.putString(relativePath, StandardCharsets.UTF_8); + if (currentFile.isDirectory()) { + hasher.putLong(-1L); + continue; + } + hasher.putLong(currentFile.length()); + Files.asByteSource(currentFile).copyTo(Funnels.asOutputStream(hasher)); + } + return Base64Variants.MODIFIED_FOR_URL.encode(hasher.hash().asBytes()); + } + + /** + * Copies the contents of the classpathElement to the output channel. + *

+ * If the classpathElement is a directory, a Zip stream is constructed on the fly, + * otherwise the file contents are copied as-is. + *

+ * The output channel is not closed. + */ + private static void copyContent(String classpathElement, WritableByteChannel outputChannel) + throws IOException { + final File classpathElementFile = new File(classpathElement); + if (!classpathElementFile.isDirectory()) { + Files.asByteSource(classpathElementFile).copyTo(Channels.newOutputStream(outputChannel)); + return; + } + + ZipOutputStream zos = new ZipOutputStream(Channels.newOutputStream(outputChannel)); + zipDirectoryRecursive(classpathElementFile, classpathElementFile, zos); + zos.finish(); + } + + /** + * Private helper function for zipping files. This one goes recursively through the input + * directory and all of its subdirectories and adds the single zip entries. + * + * @param file the file or directory to be added to the zip file. + * @param root each file uses the root directory to generate its relative path within the zip. + * @param zos the zipstream to write to. + * @throws IOException the zipping failed, e.g. because the output was not writable. + */ + private static void zipDirectoryRecursive(File file, File root, ZipOutputStream zos) + throws IOException { + final String entryName = relativize(file, root); + if (file.isDirectory()) { + // We are hitting a directory. Start the recursion. + // Add the empty entry if it is a subdirectory and the subdirectory has no children. + // Don't add it otherwise, as this is incompatible with certain implementations of unzip. + if (file.list().length == 0 && !file.equals(root)) { + ZipEntry entry = new ZipEntry(entryName + "/"); + zos.putNextEntry(entry); + } else { + // loop through the directory content, and zip the files + for (File currentFile : file.listFiles()) { + zipDirectoryRecursive(currentFile, root, zos); + } + } + } else { + // Put the next zip-entry into the zipoutputstream. + ZipEntry entry = new ZipEntry(entryName); + zos.putNextEntry(entry); + Files.asByteSource(file).copyTo(zos); + } + } + + /** + * Constructs a relative path between file and root. + *

+ * This function will attempt to use {@link java.nio.file.Path#relativize} and + * will fallback to using {@link java.net.URI#relativize} in AppEngine. + * + * @param file The file for which the relative path is being constructed for. + * @param root The root from which the relative path should be constructed. + * @return The relative path between the file and root. + */ + private static String relativize(File file, File root) { + if (AppEngineEnvironment.IS_APP_ENGINE) { + // AppEngine doesn't allow for java.nio.file.Path to be used so we rely on + // using URIs, but URIs are broken for UNC paths which AppEngine doesn't + // use. See for more details: http://wiki.eclipse.org/Eclipse/UNC_Paths + return root.toURI().relativize(file.toURI()).getPath(); + } + return root.toPath().relativize(file.toPath()).toString(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java new file mode 100644 index 0000000000000..96b2ece5cf987 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java @@ -0,0 +1,87 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.WindowUtils.bufferTag; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; +import com.google.cloud.dataflow.sdk.values.KV; + +import java.util.Collection; + +/** + * A WindowSet where each value is placed in exactly one window, + * and windows are never merged, deleted, or flushed early, and the + * WindowSet itself is never exposed to user code, allowing + * a much simpler (and cheaper) implementation. + * + * This WindowSet only works with {@link StreamingGroupAlsoByWindowsDoFn}. + */ +class PartitionBufferingWindowSet + extends AbstractWindowSet, W> { + PartitionBufferingWindowSet( + K key, + WindowingFn windowingFn, + Coder inputCoder, + DoFnProcessContext>> context, + ActiveWindowManager activeWindowManager) { + super(key, windowingFn, inputCoder, context, activeWindowManager); + } + + @Override + public void put(W window, V value) throws Exception { + context.context.stepContext.writeToTagList( + bufferTag(window, windowingFn.windowCoder(), inputCoder), value, context.timestamp()); + // Adds the window even if it is already present, relying on the streaming backend to + // de-deduplicate. + activeWindowManager.addWindow(window); + } + + @Override + public void remove(W window) throws Exception { + CodedTupleTag tag = bufferTag(window, windowingFn.windowCoder(), inputCoder); + context.context.stepContext.deleteTagList(tag); + } + + @Override + public void merge(Collection otherWindows, W newWindow) { + throw new UnsupportedOperationException(); + } + + @Override + public Collection windows() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean contains(W window) { + throw new UnsupportedOperationException(); + } + + @Override + protected Iterable finalValue(W window) throws Exception { + CodedTupleTag tag = bufferTag(window, windowingFn.windowCoder(), inputCoder); + Iterable result = context.context.stepContext.readTagList(tag); + if (result == null) { + throw new IllegalStateException("finalValue called for non-existent window"); + } + return result; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java new file mode 100644 index 0000000000000..85a81cdeff9c9 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java @@ -0,0 +1,87 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +/** + * Constant property names used by the SDK in CloudWorkflow specifications. + */ +public class PropertyNames { + public static final String APPEND_TRAILING_NEWLINES = "append_trailing_newlines"; + public static final String BIGQUERY_CREATE_DISPOSITION = "create_disposition"; + public static final String BIGQUERY_DATASET = "dataset"; + public static final String BIGQUERY_PROJECT = "project"; + public static final String BIGQUERY_SCHEMA = "schema"; + public static final String BIGQUERY_TABLE = "table"; + public static final String BIGQUERY_WRITE_DISPOSITION = "write_disposition"; + public static final String CO_GBK_RESULT_SCHEMA = "co_gbk_result_schema"; + public static final String COMBINE_FN = "combine_fn"; + public static final String COMPONENT_ENCODINGS = "component_encodings"; + public static final String CUSTOM_SOURCE_FORMAT = "custom_source"; + public static final String CUSTOM_SOURCE_STEP_INPUT = "custom_source_step_input"; + public static final String CUSTOM_SOURCE_SPEC = "spec"; + public static final String CUSTOM_SOURCE_METADATA = "metadata"; + public static final String CUSTOM_SOURCE_DOES_NOT_NEED_SPLITTING = "does_not_need_splitting"; + public static final String CUSTOM_SOURCE_PRODUCES_SORTED_KEYS = "produces_sorted_keys"; + public static final String CUSTOM_SOURCE_IS_INFINITE = "is_infinite"; + public static final String CUSTOM_SOURCE_ESTIMATED_SIZE_BYTES = "estimated_size_bytes"; + public static final String ELEMENT = "element"; + public static final String ELEMENTS = "elements"; + public static final String ENCODING = "encoding"; + public static final String END_INDEX = "end_index"; + public static final String END_OFFSET = "end_offset"; + public static final String END_SHUFFLE_POSITION = "end_shuffle_position"; + public static final String ENVIRONMENT_VERSION_JOB_TYPE_KEY = "job_type"; + public static final String ENVIRONMENT_VERSION_MAJOR_KEY = "major"; + public static final String FILENAME = "filename"; + public static final String FILENAME_PREFIX = "filename_prefix"; + public static final String FILENAME_SUFFIX = "filename_suffix"; + public static final String FILEPATTERN = "filepattern"; + public static final String FOOTER = "footer"; + public static final String FORMAT = "format"; + public static final String HEADER = "header"; + public static final String INPUTS = "inputs"; + public static final String INPUT_CODER = "input_coder"; + public static final String IS_GENERATED = "is_generated"; + public static final String IS_PAIR_LIKE = "is_pair_like"; + public static final String IS_STREAM_LIKE = "is_stream_like"; + public static final String IS_WRAPPER = "is_wrapper"; + public static final String NON_PARALLEL_INPUTS = "non_parallel_inputs"; + public static final String NUM_SHARDS = "num_shards"; + public static final String OBJECT_TYPE_NAME = "@type"; + public static final String OUTPUT = "output"; + public static final String OUTPUT_INFO = "output_info"; + public static final String OUTPUT_NAME = "output_name"; + public static final String PARALLEL_INPUT = "parallel_input"; + public static final String PHASE = "phase"; + public static final String PUBSUB_SUBSCRIPTION = "pubsub_subscription"; + public static final String PUBSUB_TOPIC = "pubsub_topic"; + public static final String SCALAR_FIELD_NAME = "value"; + public static final String SERIALIZED_FN = "serialized_fn"; + public static final String SHARD_NAME_TEMPLATE = "shard_template"; + public static final String SHUFFLE_KIND = "shuffle_kind"; + public static final String SHUFFLE_READER_CONFIG = "shuffle_reader_config"; + public static final String SHUFFLE_WRITER_CONFIG = "shuffle_writer_config"; + public static final String START_INDEX = "start_index"; + public static final String START_OFFSET = "start_offset"; + public static final String START_SHUFFLE_POSITION = "start_shuffle_position"; + public static final String STRIP_TRAILING_NEWLINES = "strip_trailing_newlines"; + public static final String TUPLE_TAGS = "tuple_tags"; + public static final String USER_FN = "user_fn"; + public static final String USER_NAME = "user_name"; + public static final String USES_KEYED_STATE = "uses_keyed_state"; + public static final String VALUE = "value"; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java new file mode 100644 index 0000000000000..34d40f1470793 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java @@ -0,0 +1,165 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.http.HttpBackOffIOExceptionHandler; +import com.google.api.client.http.HttpBackOffUnsuccessfulResponseHandler; +import com.google.api.client.http.HttpRequest; +import com.google.api.client.http.HttpRequestInitializer; +import com.google.api.client.http.HttpResponse; +import com.google.api.client.http.HttpUnsuccessfulResponseHandler; +import com.google.api.client.util.BackOff; +import com.google.api.client.util.ExponentialBackOff; +import com.google.api.client.util.NanoClock; +import com.google.api.client.util.Sleeper; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +import javax.annotation.Nullable; + +/** + * Implements a request initializer which adds retry handlers to all + * HttpRequests. + * + * This allows chaining through to another HttpRequestInitializer, since + * clients have exactly one HttpRequestInitializer, and Credential is also + * a required HttpRequestInitializer. + */ +public class RetryHttpRequestInitializer implements HttpRequestInitializer { + + private static final Logger LOG = LoggerFactory.getLogger(RetryHttpRequestInitializer.class); + + /** + * Http response codes that should be silently ignored. + */ + private static final Set IGNORED_RESPONSE_CODES = new HashSet<>( + Arrays.asList(307 /* Redirect, handled by Apiary client */, + 308 /* Resume Incomplete, handled by Apiary client */)); + + /** + * Http response timeout to use for hanging gets. + */ + private static final int HANGING_GET_TIMEOUT_SEC = 80; + + private static class LoggingHttpBackOffIOExceptionHandler + extends HttpBackOffIOExceptionHandler { + public LoggingHttpBackOffIOExceptionHandler(BackOff backOff) { + super(backOff); + } + + @Override + public boolean handleIOException(HttpRequest request, boolean supportsRetry) + throws IOException { + boolean willRetry = super.handleIOException(request, supportsRetry); + if (willRetry) { + LOG.info("Request failed with IOException, will retry: {}", request.getUrl()); + } else { + LOG.info("Request failed with IOException, will NOT retry: {}", request.getUrl()); + } + return willRetry; + } + } + + private static class LoggingHttpBackoffUnsuccessfulResponseHandler + implements HttpUnsuccessfulResponseHandler { + private final HttpBackOffUnsuccessfulResponseHandler handler; + + public LoggingHttpBackoffUnsuccessfulResponseHandler(BackOff backoff, + Sleeper sleeper) { + handler = new HttpBackOffUnsuccessfulResponseHandler(backoff); + handler.setSleeper(sleeper); + handler.setBackOffRequired( + new HttpBackOffUnsuccessfulResponseHandler.BackOffRequired() { + @Override + public boolean isRequired(HttpResponse response) { + int statusCode = response.getStatusCode(); + return (statusCode / 100 == 5) || // 5xx: server error + statusCode == 429; // 429: Too many requests + } + }); + } + + @Override + public boolean handleResponse(HttpRequest request, HttpResponse response, + boolean supportsRetry) throws IOException { + boolean retry = handler.handleResponse(request, response, supportsRetry); + if (retry) { + LOG.info("Request failed with code {} will retry: {}", + response.getStatusCode(), request.getUrl()); + + } else if (!IGNORED_RESPONSE_CODES.contains(response.getStatusCode())) { + LOG.info("Request failed with code {}, will NOT retry: {}", + response.getStatusCode(), request.getUrl()); + } + + return retry; + } + } + + private final HttpRequestInitializer chained; + + private final NanoClock nanoClock; // used for testing + + private final Sleeper sleeper; // used for testing + + /** + * @param chained a downstream HttpRequestInitializer, which will also be + * applied to HttpRequest initialization. May be null. + */ + public RetryHttpRequestInitializer(@Nullable HttpRequestInitializer chained) { + this(chained, NanoClock.SYSTEM, Sleeper.DEFAULT); + } + + public RetryHttpRequestInitializer(@Nullable HttpRequestInitializer chained, + NanoClock nanoClock, Sleeper sleeper) { + this.chained = chained; + this.nanoClock = nanoClock; + this.sleeper = sleeper; + } + + @Override + public void initialize(HttpRequest request) throws IOException { + if (chained != null) { + chained.initialize(request); + } + + // Set a timeout for hanging-gets. + // TODO: Do this exclusively for work requests. + request.setReadTimeout(HANGING_GET_TIMEOUT_SEC * 1000); + + // Back off on retryable http errors. + request.setUnsuccessfulResponseHandler( + // A back-off multiplier of 2 raises the maximum request retrying time + // to approximately 5 minutes (keeping other back-off parameters to + // their default values). + new LoggingHttpBackoffUnsuccessfulResponseHandler( + new ExponentialBackOff.Builder().setNanoClock(nanoClock) + .setMultiplier(2).build(), + sleeper)); + + // Retry immediately on IOExceptions. + LoggingHttpBackOffIOExceptionHandler loggingBackoffHandler = + new LoggingHttpBackOffIOExceptionHandler(BackOff.ZERO_BACKOFF); + request.setIOExceptionHandler(loggingBackoffHandler); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SerializableUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SerializableUtils.java new file mode 100644 index 0000000000000..9ee09c8608ab7 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SerializableUtils.java @@ -0,0 +1,145 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.CoderUtils.decodeFromByteArray; +import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.Arrays; + +/** + * Utilities for working with Serializables. + */ +public class SerializableUtils { + /** + * Serializes the argument into an array of bytes, and returns it. + * + * @throws IllegalArgumentException if there are errors when serializing + */ + public static byte[] serializeToByteArray(Serializable value) { + try { + ByteArrayOutputStream buffer = new ByteArrayOutputStream(); + try (ObjectOutputStream oos = new ObjectOutputStream(buffer)) { + oos.writeObject(value); + } + return buffer.toByteArray(); + } catch (IOException exn) { + throw new IllegalArgumentException( + "unable to serialize " + value, + exn); + } + } + + /** + * Deserializes an object from the given array of bytes, e.g., as + * serialized using {@link #serializeToByteArray}, and returns it. + * + * @throws IllegalArgumentException if there are errors when + * deserializing, using the provided description to identify what + * was being deserialized + */ + public static Object deserializeFromByteArray(byte[] encodedValue, + String description) { + try { + try (ObjectInputStream ois = new ObjectInputStream( + new ByteArrayInputStream(encodedValue))) { + return ois.readObject(); + } + } catch (IOException | ClassNotFoundException exn) { + throw new IllegalArgumentException( + "unable to deserialize " + description, + exn); + } + } + + public static T ensureSerializable(T value) { + @SuppressWarnings("unchecked") + T copy = (T) deserializeFromByteArray(serializeToByteArray(value), + value.toString()); + return copy; + } + + /** + * Serializes a Coder and verifies that it can be correctly deserialized. + *

+ * Throws a RuntimeException if serialized Coder cannot be deserialized, or + * if the deserialized instance is not equal to the original. + *

+ * @return the serialized Coder, as a {@link CloudObject} + */ + public static CloudObject ensureSerializable(Coder coder) { + CloudObject cloudObject = coder.asCloudObject(); + + Coder decoded; + try { + decoded = Serializer.deserialize(cloudObject, Coder.class); + } catch (RuntimeException e) { + throw new RuntimeException( + String.format("Unable to deserialize Coder: %s. " + + "Check that a suitable constructor is defined. " + + "See Coder for details.", coder), e + ); + } + Preconditions.checkState(coder.equals(decoded), + String.format("Coder not equal to original after serialization, " + + "indicating that the Coder may not implement serialization " + + "correctly. Before: %s, after: %s, cloud encoding: %s", + coder, decoded, cloudObject)); + + return cloudObject; + } + + /** + * Serializes an arbitrary T with the given Coder and verifies + * that it can be correctly deserialized. + */ + public static T ensureSerializableByCoder( + Coder coder, T value, String errorContext) { + byte[] encodedValue; + try { + encodedValue = encodeToByteArray(coder, value); + } catch (CoderException exn) { + // TODO: Put in better element printing: + // truncate if too long. + throw new IllegalArgumentException( + errorContext + ": unable to encode value " + + value + " using " + coder, + exn); + } + try { + return decodeFromByteArray(coder, encodedValue); + } catch (CoderException exn) { + // TODO: Put in better encoded byte array printing: + // use printable chars with escapes instead of codes, and + // truncate if too long. + throw new IllegalArgumentException( + errorContext + ": unable to decode " + Arrays.toString(encodedValue) + + ", encoding of value " + value + ", using " + coder, + exn); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Serializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Serializer.java new file mode 100644 index 0000000000000..42071ec467ee3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Serializer.java @@ -0,0 +1,152 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import javax.annotation.Nullable; + +/** + * Utility for converting objects between Java and Cloud representations. + */ +public final class Serializer { + // Delay initialization of statics until the first call to Serializer. + private static class SingletonHelper { + static final ObjectMapper OBJECT_MAPPER = createObjectMapper(); + static final ObjectMapper TREE_MAPPER = createTreeMapper(); + + /** + * Creates the object mapper which will be used for serializing Google API + * client maps into Jackson trees. + */ + private static ObjectMapper createTreeMapper() { + return new ObjectMapper(); + } + + /** + * Creates the object mapper which will be used for deserializing Jackson + * trees into objects. + */ + private static ObjectMapper createObjectMapper() { + ObjectMapper m = new ObjectMapper(); + // Ignore properties which are not used by the object. + m.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); + + // For parameters of type Object, use the @type property to determine the + // class to instantiate. + // + // TODO: It would be ideal to do this for all non-final classes. The + // problem with using DefaultTyping.NON_FINAL is that it insists on having + // type information in the JSON for classes with useful default + // implementations, such as List. Ideally, we'd combine these defaults + // with available type information if that information's present. + m.enableDefaultTypingAsProperty( + ObjectMapper.DefaultTyping.JAVA_LANG_OBJECT, + PropertyNames.OBJECT_TYPE_NAME); + + m.registerModule(new CoderUtils.Jackson2Module()); + + return m; + } + } + + /** + * Registers a module to use during object deserialization. + */ + public static void registerModule(Module module) { + SingletonHelper.OBJECT_MAPPER.registerModule(module); + } + + /** + * Deserializes an object from a Dataflow structured encoding (represented in + * Java as a map). + *

+ * The standard Dataflow SDK object serialization protocol is based on JSON. + * Data is typically encoded as a JSON object whose fields represent the + * object's data. + *

+ * The actual deserialization is performed by Jackson, which can deserialize + * public fields, use JavaBean setters, or use injection annotations to + * indicate how to construct the object. The {@link ObjectMapper} used is + * configured to use the "@type" field as the name of the class to instantiate + * (supporting polymorphic types), and may be further configured by + * annotations or via {@link #registerModule}. + *

+ * @see + * Jackson Data-Binding + * @see + * Jackson-Annotations + * @param serialized the object in untyped decoded form (i.e. a nested {@link Map}) + * @param clazz the expected object class + */ + public static T deserialize(Map serialized, Class clazz) { + try { + return SingletonHelper.OBJECT_MAPPER.treeToValue( + SingletonHelper.TREE_MAPPER.valueToTree( + deserializeCloudKnownTypes(serialized)), + clazz); + } catch (JsonProcessingException e) { + throw new RuntimeException( + "Unable to deserialize class " + clazz, e); + } + } + + /** + * Recursively walks the supplied map, looking for well-known cloud type + * information (keyed as {@link PropertyNames#OBJECT_TYPE_NAME}, matching a + * URI value from the {@link CloudKnownType} enum. Upon finding this type + * information, it converts it into the correspondingly typed Java value. + */ + private static Object deserializeCloudKnownTypes(Object src) { + if (src instanceof Map) { + Map srcMap = (Map) src; + @Nullable Object value = srcMap.get(PropertyNames.SCALAR_FIELD_NAME); + @Nullable CloudKnownType type = + CloudKnownType.forUri((String) srcMap.get(PropertyNames.OBJECT_TYPE_NAME)); + if (type != null && value != null) { + // It's a value of a well-known cloud type; let the known type handler + // handle the translation. + Object result = type.parse(value, type.defaultClass()); + return result; + } + // Otherwise, it's just an ordinary map. + Map dest = new HashMap<>(srcMap.size()); + for (Map.Entry entry : srcMap.entrySet()) { + dest.put(entry.getKey(), deserializeCloudKnownTypes(entry.getValue())); + } + return dest; + } + if (src instanceof List) { + List srcList = (List) src; + List dest = new ArrayList<>(srcList.size()); + for (Object obj : srcList) { + dest.add(deserializeCloudKnownTypes(obj)); + } + return dest; + } + // Neither a Map nor a List; no translation needed. + return src; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ShardingWritableByteChannel.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ShardingWritableByteChannel.java new file mode 100644 index 0000000000000..4a3322b345355 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ShardingWritableByteChannel.java @@ -0,0 +1,118 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; + +/** + * Implements a WritableByteChannel which may contain multiple output shards. + * + *

This provides {@link #writeToShard}, which takes a shard number for + * writing to a particular shard. + * + *

The channel is considered open if all downstream channels are open, and + * closes all downstream channels when closed. + */ +public class ShardingWritableByteChannel implements WritableByteChannel { + + /** + * Special shard number which causes a write to all shards. + */ + public static final int ALL_SHARDS = -2; + + + private final ArrayList writers = new ArrayList<>(); + + /** + * Returns the number of output shards. + */ + public int getNumShards() { + return writers.size(); + } + + /** + * Adds another shard output channel. + */ + public void addChannel(WritableByteChannel writer) { + writers.add(writer); + } + + /** + * Returns the WritableByteChannel associated with the given shard number. + */ + public WritableByteChannel getChannel(int shardNum) { + return writers.get(shardNum); + } + + /** + * Writes the buffer to the given shard. + * + *

This does not change the current output shard. + * + * @return The total number of bytes written. If the shard number is + * {@link #ALL_SHARDS}, then the total is the sum of each individual shard + * write. + */ + public int writeToShard(int shardNum, ByteBuffer src) throws IOException { + if (shardNum >= 0) { + return writers.get(shardNum).write(src); + } + + switch (shardNum) { + case ALL_SHARDS: + int size = 0; + for (WritableByteChannel writer : writers) { + size += writer.write(src); + } + return size; + + default: + throw new IllegalArgumentException("Illegal shard number: " + shardNum); + } + } + + /** + * Writes a buffer to all shards. + * + *

Same as calling {@code writeToShard(ALL_SHARDS, buf)}. + */ + @Override + public int write(ByteBuffer src) throws IOException { + return writeToShard(ALL_SHARDS, src); + } + + @Override + public boolean isOpen() { + for (WritableByteChannel writer : writers) { + if (!writer.isOpen()) { + return false; + } + } + + return true; + } + + @Override + public void close() throws IOException { + for (WritableByteChannel writer : writers) { + writer.close(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java new file mode 100644 index 0000000000000..dcfd58aee92d2 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java @@ -0,0 +1,133 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.PartitioningWindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.values.KV; + +import java.io.IOException; + +/** + * DoFn that merges windows and groups elements in those windows. + * + * @param key type + * @param input value element type + * @param output value element type + * @param window type + */ +public class StreamingGroupAlsoByWindowsDoFn + extends DoFn>, KV> implements DoFn.RequiresKeyedState { + + protected WindowingFn windowingFn; + protected Coder inputCoder; + + protected StreamingGroupAlsoByWindowsDoFn( + WindowingFn windowingFn, + Coder inputCoder) { + this.windowingFn = windowingFn; + this.inputCoder = inputCoder; + } + + public static + StreamingGroupAlsoByWindowsDoFn create( + WindowingFn windowingFn, + Coder inputCoder) { + return new StreamingGroupAlsoByWindowsDoFn<>(windowingFn, inputCoder); + } + + private AbstractWindowSet createWindowSet( + K key, + DoFnProcessContext> context, + AbstractWindowSet.ActiveWindowManager activeWindowManager) throws Exception { + if (windowingFn instanceof PartitioningWindowingFn) { + return new PartitionBufferingWindowSet( + key, windowingFn, inputCoder, context, activeWindowManager); + } else { + return new BufferingWindowSet(key, windowingFn, inputCoder, context, activeWindowManager); + } + } + + @Override + public void processElement(ProcessContext processContext) throws Exception { + DoFnProcessContext>, KV> context = + (DoFnProcessContext>, KV>) processContext; + if (!context.element().isTimer()) { + KV element = context.element().element(); + K key = element.getKey(); + VI value = element.getValue(); + AbstractWindowSet windowSet = createWindowSet( + key, context, new StreamingActiveWindowManager<>(context, windowingFn.windowCoder())); + + for (BoundedWindow window : context.windows()) { + windowSet.put((W) window, value); + } + + windowSet.flush(); + } else { + TimerOrElement timer = context.element(); + AbstractWindowSet windowSet = createWindowSet( + (K) timer.key(), context, new StreamingActiveWindowManager<>( + context, windowingFn.windowCoder())); + + // Attempt to merge windows before emitting; that may remove the current window under + // consideration. + ((WindowingFn) windowingFn) + .mergeWindows(new AbstractWindowSet.WindowMergeContext(windowSet, windowingFn)); + + W window = WindowUtils.windowFromString(timer.tag(), windowingFn.windowCoder()); + boolean windowExists; + try { + windowExists = windowSet.contains(window); + } catch (UnsupportedOperationException e) { + windowExists = true; + } + if (windowExists) { + windowSet.markCompleted(window); + windowSet.flush(); + } + } + } + + private static class StreamingActiveWindowManager + implements AbstractWindowSet.ActiveWindowManager { + DoFnProcessContext context; + Coder coder; + + StreamingActiveWindowManager( + DoFnProcessContext context, + Coder coder) { + this.context = context; + this.coder = coder; + } + + @Override + public void addWindow(W window) throws IOException { + context.context.stepContext.getExecutionContext().setTimer( + WindowUtils.windowToString(window, coder), window.maxTimestamp()); + } + + @Override + public void removeWindow(W window) throws IOException { + context.context.stepContext.getExecutionContext().deleteTimer( + WindowUtils.windowToString(window, coder)); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StringUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StringUtils.java new file mode 100644 index 0000000000000..382683c2de3c3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StringUtils.java @@ -0,0 +1,146 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.common.base.Joiner; + +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Utilities for working with JSON and other human-readable string formats. + */ +public class StringUtils { + /** + * Converts the given array of bytes into a legal JSON string. + * + * Uses a simple strategy of converting each byte to a single char, + * except for non-printable chars, non-ASCII chars, and '%', '\', + * and '"', which are encoded as three chars in '%xx' format, where + * 'xx' is the hexadecimal encoding of the byte. + */ + public static String byteArrayToJsonString(byte[] bytes) { + StringBuilder sb = new StringBuilder(bytes.length * 2); + for (byte b : bytes) { + if (b >= 32 && b < 127) { + // A printable ascii character. + char c = (char) b; + if (c != '%' && c != '\\' && c != '\"') { + // Not an escape prefix or special character, either. + // Send through unchanged. + sb.append(c); + continue; + } + } + // Send through escaped. Use '%xx' format. + sb.append(String.format("%%%02x", b)); + } + return sb.toString(); + } + + /** + * Converts the given string, encoded using {@link #byteArrayToJsonString}, + * into a byte array. + * + * @throws IllegalArgumentException if the argument string is not legal + */ + public static byte[] jsonStringToByteArray(String string) { + List bytes = new ArrayList<>(); + for (int i = 0; i < string.length(); ) { + char c = string.charAt(i); + Byte b; + if (c == '%') { + // Escaped. Expect '%xx' format. + try { + b = (byte) Integer.parseInt(string.substring(i + 1, i + 3), 16); + } catch (IndexOutOfBoundsException | NumberFormatException exn) { + throw new IllegalArgumentException( + "not in legal encoded format; " + + "substring [" + i + ".." + (i + 2) + "] not in format \"%xx\"", + exn); + } + i += 3; + } else { + // Send through unchanged. + b = (byte) c; + i++; + } + bytes.add(b); + } + byte[] byteArray = new byte[bytes.size()]; + int i = 0; + for (Byte b : bytes) { + byteArray[i++] = b; + } + return byteArray; + } + + private static final String[] STANDARD_NAME_SUFFIXES = + new String[]{"DoFn", "Fn"}; + + /** + * Pattern to match a non-anonymous inner class. + * Eg, matches "Foo$Bar", or even "Foo$1$Bar", but not "Foo$1" or "Foo$1$2". + */ + private static final Pattern NAMED_INNER_CLASS = + Pattern.compile(".+\\$(?[^0-9].*)"); + + /** + * Returns a simple name for a class. + * + *

Note: this is non-invertible - the name may be simplified to an + * extent that it cannot be mapped back to the original class. + * + *

This can be used to generate human-readable transform names. It + * removes the package from the name, and removes common suffixes. + * + *

Examples: + *

    + *
  • {@code some.package.WordSummaryDoFn} -> "WordSummary" + *
  • {@code another.package.PairingFn} -> "Pairing" + *
+ */ + public static String approximateSimpleName(Class clazz) { + String fullName = clazz.getName(); + String shortName = fullName.substring(fullName.lastIndexOf('.') + 1); + + // Simplify inner class name by dropping outer class prefixes. + Matcher m = NAMED_INNER_CLASS.matcher(shortName); + if (m.matches()) { + shortName = m.group("INNER"); + } + + // Drop common suffixes for each named component. + String[] names = shortName.split("\\$"); + for (int i = 0; i < names.length; i++) { + names[i] = simplifyNameComponent(names[i]); + } + + return Joiner.on('$').join(names); + } + + private static String simplifyNameComponent(String name) { + for (String suffix : STANDARD_NAME_SUFFIXES) { + if (name.endsWith(suffix) && name.length() > suffix.length()) { + return name.substring(0, name.length() - suffix.length()); + } + } + return name; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Structs.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Structs.java new file mode 100644 index 0000000000000..8fb2e834f19e5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Structs.java @@ -0,0 +1,345 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.util.Data; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import javax.annotation.Nullable; + +/** + * A collection of static methods for manipulating datastructure representations + * transferred via the Dataflow API. + */ +public final class Structs { + private Structs() {} // Non-instantiable + + public static String getString(Map map, String name) throws Exception { + return getValue(map, name, String.class, "a string"); + } + + public static String getString( + Map map, String name, @Nullable String defaultValue) + throws Exception { + return getValue(map, name, String.class, "a string", defaultValue); + } + + public static byte[] getBytes(Map map, String name) throws Exception { + @Nullable byte[] result = getBytes(map, name, null); + if (result == null) { + throw new ParameterNotFoundException(name, map); + } + return result; + } + + @Nullable + public static byte[] getBytes(Map map, String name, @Nullable byte[] defaultValue) + throws Exception { + @Nullable String jsonString = getString(map, name, null); + if (jsonString == null) { + return defaultValue; + } + // TODO: Need to agree on a format for encoding bytes in + // a string that can be sent over the Apiary wire, over the cloud + // map task work API. base64 encoding seems pretty common. Switch to it? + return StringUtils.jsonStringToByteArray(jsonString); + } + + public static Boolean getBoolean(Map map, String name) throws Exception { + return getValue(map, name, Boolean.class, "a boolean"); + } + + @Nullable + public static Boolean getBoolean( + Map map, String name, @Nullable Boolean defaultValue) + throws Exception { + return getValue(map, name, Boolean.class, "a boolean", defaultValue); + } + + public static Long getLong(Map map, String name) throws Exception { + return getValue(map, name, Long.class, "an int"); + } + + @Nullable + public static Long getLong(Map map, String name, @Nullable Long defaultValue) + throws Exception { + return getValue(map, name, Long.class, "an int", defaultValue); + } + + @Nullable + public static List getStrings( + Map map, String name, @Nullable List defaultValue) + throws Exception { + @Nullable Object value = map.get(name); + if (value == null) { + if (map.containsKey(name)) { + throw new IncorrectTypeException(name, map, "a string or a list"); + } + return defaultValue; + } + if (Data.isNull(value)) { + // This is a JSON literal null. When represented as a list of strings, + // this is an empty list. + return Collections.emptyList(); + } + @Nullable String singletonString = decodeValue(value, String.class); + if (singletonString != null) { + return Collections.singletonList(singletonString); + } + if (!(value instanceof List)) { + throw new IncorrectTypeException(name, map, "a string or a list"); + } + @SuppressWarnings("unchecked") + List elements = (List) value; + List result = new ArrayList<>(elements.size()); + for (Object o : elements) { + @Nullable String s = decodeValue(o, String.class); + if (s == null) { + throw new IncorrectTypeException(name, map, "a list of strings"); + } + result.add(s); + } + return result; + } + + public static Map getObject(Map map, String name) + throws Exception { + @Nullable Map result = getObject(map, name, null); + if (result == null) { + throw new ParameterNotFoundException(name, map); + } + return result; + } + + @Nullable + public static Map getObject( + Map map, String name, @Nullable Map defaultValue) + throws Exception { + @Nullable Object value = map.get(name); + if (value == null) { + if (map.containsKey(name)) { + throw new IncorrectTypeException(name, map, "an object"); + } + return defaultValue; + } + return checkObject(value, map, name); + } + + private static Map checkObject( + Object value, Map map, String name) throws Exception { + if (Data.isNull(value)) { + // This is a JSON literal null. When represented as an object, this is an + // empty map. + return Collections.emptyMap(); + } + if (!(value instanceof Map)) { + throw new IncorrectTypeException(name, map, "an object (not a map)"); + } + @SuppressWarnings("unchecked") + Map mapValue = (Map) value; + if (!mapValue.containsKey(PropertyNames.OBJECT_TYPE_NAME)) { + throw new IncorrectTypeException(name, map, + "an object (no \"" + PropertyNames.OBJECT_TYPE_NAME + "\" field)"); + } + return mapValue; + } + + public static Map getDictionary( + Map map, String name) throws Exception { + @Nullable Object value = map.get(name); + if (value == null) { + throw new ParameterNotFoundException(name, map); + } + if (Data.isNull(value)) { + // This is a JSON literal null. When represented as a dictionary, this is + // an empty map. + return Collections.emptyMap(); + } + if (!(value instanceof Map)) { + throw new IncorrectTypeException(name, map, "a dictionary"); + } + @SuppressWarnings("unchecked") + Map result = (Map) value; + return result; + } + + @Nullable + public static Map getDictionary( + Map map, String name, @Nullable Map defaultValue) + throws Exception { + @Nullable Object value = map.get(name); + if (value == null) { + if (map.containsKey(name)) { + throw new IncorrectTypeException(name, map, "a dictionary"); + } + return defaultValue; + } + if (Data.isNull(value)) { + // This is a JSON literal null. When represented as a dictionary, this is + // an empty map. + return Collections.emptyMap(); + } + if (!(value instanceof Map)) { + throw new IncorrectTypeException(name, map, "a dictionary"); + } + @SuppressWarnings("unchecked") + Map result = (Map) value; + return result; + } + + // Builder operations. + + public static void addString(Map map, String name, String value) { + addObject(map, name, CloudObject.forString(value)); + } + + public static void addBoolean(Map map, String name, boolean value) { + addObject(map, name, CloudObject.forBoolean(value)); + } + + public static void addLong(Map map, String name, long value) { + addObject(map, name, CloudObject.forInteger(value)); + } + + public static void addObject( + Map map, String name, Map value) { + map.put(name, value); + } + + public static void addNull(Map map, String name) { + map.put(name, Data.nullOf(Object.class)); + } + + public static void addLongs(Map map, String name, long... longs) { + List> elements = new ArrayList<>(longs.length); + for (Long value : longs) { + elements.add(CloudObject.forInteger(value)); + } + map.put(name, elements); + } + + public static void addList( + Map map, String name, List> elements) { + map.put(name, elements); + } + + public static void addStringList(Map map, String name, List elements) { + ArrayList objects = new ArrayList<>(elements.size()); + for (String element : elements) { + objects.add(CloudObject.forString(element)); + } + addList(map, name, objects); + } + + public static > void addList( + Map map, String name, T[] elements) { + map.put(name, Arrays.asList(elements)); + } + + public static void addDictionary( + Map map, String name, Map value) { + map.put(name, value); + } + + public static void addDouble(Map map, String name, Double value) { + addObject(map, name, CloudObject.forFloat(value)); + } + + // Helper methods for a few of the accessor methods. + + private static T getValue(Map map, String name, Class clazz, String type) + throws Exception { + @Nullable T result = getValue(map, name, clazz, type, null); + if (result == null) { + throw new ParameterNotFoundException(name, map); + } + return result; + } + + @Nullable + private static T getValue( + Map map, String name, Class clazz, String type, @Nullable T defaultValue) + throws Exception { + @Nullable Object value = map.get(name); + if (value == null) { + if (map.containsKey(name)) { + throw new IncorrectTypeException(name, map, type); + } + return defaultValue; + } + T result = decodeValue(value, clazz); + if (result == null) { + // The value exists, but can't be decoded. + throw new IncorrectTypeException(name, map, type); + } + return result; + } + + @Nullable + private static T decodeValue(Object value, Class clazz) { + try { + if (value.getClass() == clazz) { + // decodeValue() is only called for final classes; if the class matches, + // it's safe to just return the value, and if it doesn't match, decoding + // is needed. + return clazz.cast(value); + } + if (!(value instanceof Map)) { + return null; + } + @SuppressWarnings("unchecked") + Map map = (Map) value; + @Nullable String typeName = (String) map.get(PropertyNames.OBJECT_TYPE_NAME); + if (typeName == null) { + return null; + } + @Nullable CloudKnownType knownType = CloudKnownType.forUri(typeName); + if (knownType == null) { + return null; + } + @Nullable Object scalar = map.get(PropertyNames.SCALAR_FIELD_NAME); + if (scalar == null) { + return null; + } + return knownType.parse(scalar, clazz); + } catch (ClassCastException e) { + // If any class cast fails during decoding, the value's not decodable. + return null; + } + } + + private static final class ParameterNotFoundException extends Exception { + private static final long serialVersionUID = 0; + + public ParameterNotFoundException(String name, Map map) { + super("didn't find required parameter " + name + " in " + map); + } + } + + private static final class IncorrectTypeException extends Exception { + private static final long serialVersionUID = 0; + + public IncorrectTypeException(String name, Map map, String type) { + super("required parameter " + name + " in " + map + " not " + type); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TestCredential.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TestCredential.java new file mode 100644 index 0000000000000..fa02a6bf3185b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TestCredential.java @@ -0,0 +1,49 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.auth.oauth2.BearerToken; +import com.google.api.client.auth.oauth2.Credential; +import com.google.api.client.auth.oauth2.TokenResponse; + +import java.io.IOException; + +/** + * Fake credential, for use in testing. + */ +public class TestCredential extends Credential { + + private final String token; + + public TestCredential() { + this("NULL"); + } + + public TestCredential(String token) { + super(new Builder( + BearerToken.authorizationHeaderAccessMethod())); + this.token = token; + } + + @Override + protected TokenResponse executeRefreshToken() throws IOException { + TokenResponse response = new TokenResponse(); + response.setExpiresInSeconds(5L * 60); + response.setAccessToken(token); + return response; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimeUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimeUtil.java new file mode 100644 index 0000000000000..48324818ca63b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimeUtil.java @@ -0,0 +1,164 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.joda.time.ReadableDuration; +import org.joda.time.ReadableInstant; +import org.joda.time.chrono.ISOChronology; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import javax.annotation.Nullable; + +/** + * A helper class for converting between Dataflow API and SDK time + * representations. + *

+ * Dataflow API times are strings of the form + * {@code YYYY-MM-dd'T'HH:mm:ss[.nnnn]'Z'}: that is, RFC 3339 + * strings with optional fractional seconds and a 'Z' offset. + *

+ * Dataflow API durations are strings of the form {@code ['-']sssss[.nnnn]'s'}: + * that is, seconds with optional fractional seconds and a literal 's' at the end. + *

+ * In both formats, fractional seconds are either three digits (millisecond + * resolution), six digits (microsecond resolution), or nine digits (nanosecond + * resolution). + */ +public final class TimeUtil { + private TimeUtil() {} // Non-instantiable. + + private static final Pattern DURATION_PATTERN = Pattern.compile("(\\d+)(?:\\.(\\d+))?s"); + private static final Pattern TIME_PATTERN = + Pattern.compile("(\\d{4})-(\\d{2})-(\\d{2})T(\\d{2}):(\\d{2}):(\\d{2})(?:\\.(\\d+))?Z"); + + /** + * Converts a {@link ReadableInstant} into a Dateflow API time value. + */ + public static String toCloudTime(ReadableInstant instant) { + // Note that since Joda objects use millisecond resolution, we always + // produce either no fractional seconds or fractional seconds with + // millisecond resolution. + + // Translate the ReadableInstant to a DateTime with ISOChronology. + DateTime time = new DateTime(instant); + + int millis = time.getMillisOfSecond(); + if (millis == 0) { + return String.format("%04d-%02d-%02dT%02d:%02d:%02dZ", + time.getYear(), + time.getMonthOfYear(), + time.getDayOfMonth(), + time.getHourOfDay(), + time.getMinuteOfHour(), + time.getSecondOfMinute()); + } else { + return String.format("%04d-%02d-%02dT%02d:%02d:%02d.%03dZ", + time.getYear(), + time.getMonthOfYear(), + time.getDayOfMonth(), + time.getHourOfDay(), + time.getMinuteOfHour(), + time.getSecondOfMinute(), + millis); + } + } + + /** + * Converts a time value received via the Dataflow API into the corresponding + * {@link Instant}. + * @return the parsed time, or null if a parse error occurs + */ + @Nullable + public static Instant fromCloudTime(String time) { + Matcher matcher = TIME_PATTERN.matcher(time); + if (!matcher.matches()) { + return null; + } + int year = Integer.valueOf(matcher.group(1)); + int month = Integer.valueOf(matcher.group(2)); + int day = Integer.valueOf(matcher.group(3)); + int hour = Integer.valueOf(matcher.group(4)); + int minute = Integer.valueOf(matcher.group(5)); + int second = Integer.valueOf(matcher.group(6)); + int millis = 0; + + String frac = matcher.group(7); + if (frac != null) { + int fracs = Integer.valueOf(frac); + if (frac.length() == 3) { // millisecond resolution + millis = fracs; + } else if (frac.length() == 6) { // microsecond resolution + millis = fracs / 1000; + } else if (frac.length() == 9) { // nanosecond resolution + millis = fracs / 1000000; + } else { + return null; + } + } + + return new DateTime(year, month, day, hour, minute, second, millis, + ISOChronology.getInstanceUTC()).toInstant(); + } + + /** + * Converts a {@link ReadableDuration} into a Dataflow API duration string. + */ + public static String toCloudDuration(ReadableDuration duration) { + // Note that since Joda objects use millisecond resolution, we always + // produce either no fractional seconds or fractional seconds with + // millisecond resolution. + long millis = duration.getMillis(); + long seconds = millis / 1000; + millis = millis % 1000; + if (millis == 0) { + return String.format("%ds", seconds); + } else { + return String.format("%d.%03ds", seconds, millis); + } + } + + /** + * Converts a Dataflow API duration string into a {@link Duration}. + * @return the parsed duration, or null if a parse error occurs + */ + @Nullable + public static Duration fromCloudDuration(String duration) { + Matcher matcher = DURATION_PATTERN.matcher(duration); + if (!matcher.matches()) { + return null; + } + long millis = Long.valueOf(matcher.group(1)) * 1000; + String frac = matcher.group(2); + if (frac != null) { + long fracs = Long.valueOf(frac); + if (frac.length() == 3) { // millisecond resolution + millis += fracs; + } else if (frac.length() == 6) { // microsecond resolution + millis += fracs / 1000; + } else if (frac.length() == 9) { // nanosecond resolution + millis += fracs / 1000000; + } else { + return null; + } + } + return Duration.millis(millis); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerOrElement.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerOrElement.java new file mode 100644 index 0000000000000..4859f8ae5f39b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerOrElement.java @@ -0,0 +1,195 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StandardCoder; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.joda.time.Instant; + +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.List; + +/** + * Class representing either a timer, or arbitrary element. + * Used as the input type of {@link StreamingGroupAlsoByWindowsDoFn}. + * + * @param the element type + */ +public class TimerOrElement { + + /** + * Creates a new {@code TimerOrElement} representing a timer. + * + * @param the element type + */ + public static TimerOrElement timer( + String tag, Instant timestamp, Object key) { + return new TimerOrElement<>(tag, timestamp, key); + } + + /** + * Creates a new {@code TimerOrElement} representing an element. + * + * @param the element type + */ + public static TimerOrElement element(E element) { + return new TimerOrElement<>(element); + } + + /** + * Returns whether this is a timer or an element. + */ + public boolean isTimer() { + return isTimer; + } + + /** + * If this is a timer, returns its tag, otherwise throws an exception. + */ + public String tag() { + if (!isTimer) { + throw new IllegalStateException("tag() called, but this is an element"); + } + return tag; + } + + /** + * If this is a timer, returns its timestamp, otherwise throws an exception. + */ + public Instant timestamp() { + if (!isTimer) { + throw new IllegalStateException("timestamp() called, but this is an element"); + } + return timestamp; + } + + /** + * If this is a timer, returns its key, otherwise throws an exception. + */ + public Object key() { + if (!isTimer) { + throw new IllegalStateException("key() called, but this is an element"); + } + return key; + } + + /** + * If this is an element, returns it, otherwise throws an exception. + */ + public E element() { + if (isTimer) { + throw new IllegalStateException("element() called, but this is a timer"); + } + return element; + } + + /** + * Coder that forwards {@code ByteSizeObserver} calls to an underlying element coder. + * {@code TimerOrElement} objects never need to be encoded, so this class does not + * support the {@code encode} and {@code decode} methods. + */ + public static class TimerOrElementCoder extends StandardCoder> { + final Coder elemCoder; + + /** + * Creates a new {@code TimerOrElement.Coder} that wraps the given {@link Coder}. + */ + public static TimerOrElementCoder of(Coder elemCoder) { + return new TimerOrElementCoder<>(elemCoder); + } + + @JsonCreator + public static TimerOrElementCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List components) { + return of((Coder) components.get(0)); + } + + @Override + public void encode(TimerOrElement value, OutputStream outStream, Context context) { + throw new UnsupportedOperationException(); + } + + @Override + public TimerOrElement decode(InputStream inStream, Context context) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isRegisterByteSizeObserverCheap(TimerOrElement value, Context context) { + if (value.isTimer()) { + return true; + } else { + return elemCoder.isRegisterByteSizeObserverCheap(value.element(), context); + } + } + + @Override + public void registerByteSizeObserver( + TimerOrElement value, ElementByteSizeObserver observer, Context context) + throws Exception{ + if (!value.isTimer()) { + elemCoder.registerByteSizeObserver(value.element(), observer, context); + } + } + + @Override + public boolean isDeterministic() { + return elemCoder.isDeterministic(); + } + + @Override + public List> getCoderArguments() { + return Arrays.asList(elemCoder); + } + + public Coder getElementCoder() { + return elemCoder; + } + + private TimerOrElementCoder(Coder elemCoder) { + this.elemCoder = elemCoder; + } + } + + ////////////////////////////////////////////////////////////////////////////// + + private boolean isTimer; + private String tag; + private Instant timestamp; + private Object key; + private E element; + + TimerOrElement(String tag, Instant timestamp, Object key) { + this.isTimer = true; + this.tag = tag; + this.timestamp = timestamp; + this.key = key; + } + + TimerOrElement(E element) { + this.isTimer = false; + this.element = element; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java new file mode 100644 index 0000000000000..e27f7fcc4f885 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java @@ -0,0 +1,141 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport; +import com.google.api.client.http.HttpTransport; +import com.google.api.client.json.JsonFactory; +import com.google.api.client.json.jackson2.JacksonFactory; +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.storage.Storage; +import com.google.cloud.dataflow.sdk.options.BigQueryOptions; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.GcsOptions; +import com.google.cloud.dataflow.sdk.options.StreamingOptions; + +import java.io.IOException; +import java.net.MalformedURLException; +import java.net.URL; +import java.security.GeneralSecurityException; + +/** + * Helpers for cloud communication. + */ +public class Transport { + + private static class SingletonHelper { + /** Global instance of the JSON factory. */ + private static final JsonFactory JSON_FACTORY; + + /** Global instance of the HTTP transport. */ + private static final HttpTransport HTTP_TRANSPORT; + + static { + try { + JSON_FACTORY = JacksonFactory.getDefaultInstance(); + HTTP_TRANSPORT = GoogleNetHttpTransport.newTrustedTransport(); + } catch (GeneralSecurityException | IOException e) { + throw new RuntimeException(e); + } + } + } + + public static HttpTransport getTransport() { + return SingletonHelper.HTTP_TRANSPORT; + } + + public static JsonFactory getJsonFactory() { + return SingletonHelper.JSON_FACTORY; + } + + /** + * Returns a BigQuery client builder. + *

+ * Note: this client's endpoint is not modified by the + * {@link DataflowPipelineDebugOptions#getApiRootUrl()} option. + */ + public static Bigquery.Builder + newBigQueryClient(BigQueryOptions options) { + return new Bigquery.Builder(getTransport(), getJsonFactory(), + new RetryHttpRequestInitializer(options.getGcpCredential())) + .setApplicationName(options.getAppName()); + } + +/** + * Returns a Pubsub client builder. + *

+ * Note: this client's endpoint is not modified by the + * {@link DataflowPipelineDebugOptions#getApiRootUrl()} option. + */ + public static Pubsub.Builder + newPubsubClient(StreamingOptions options) { + return new Pubsub.Builder(getTransport(), getJsonFactory(), + new RetryHttpRequestInitializer(options.getGcpCredential())) + .setApplicationName(options.getAppName()); + } + + /** + * Returns a Google Cloud Dataflow client builder. + */ + public static Dataflow.Builder newDataflowClient(DataflowPipelineOptions options) { + String rootUrl = options.getApiRootUrl(); + String servicePath = options.getDataflowEndpoint(); + if (servicePath.contains("://")) { + try { + URL url = new URL(servicePath); + rootUrl = url.getProtocol() + "://" + url.getHost() + + (url.getPort() > 0 ? ":" + url.getPort() : ""); + servicePath = url.getPath(); + } catch (MalformedURLException e) { + throw new RuntimeException("Invalid URL: " + servicePath); + } + } + + return new Dataflow.Builder(getTransport(), + getJsonFactory(), + new RetryHttpRequestInitializer(options.getGcpCredential())) + .setApplicationName(options.getAppName()) + .setRootUrl(rootUrl) + .setServicePath(servicePath); + } + + /** + * Returns a Dataflow client which does not automatically retry failed + * requests. + */ + public static Dataflow.Builder + newRawDataflowClient(DataflowPipelineOptions options) { + return newDataflowClient(options) + .setHttpRequestInitializer(options.getGcpCredential()); + } + + /** + * Returns a Cloud Storage client builder. + *

+ * Note: this client's endpoint is not modified by the + * {@link DataflowPipelineDebugOptions#getApiRootUrl()} option. + */ + public static Storage.Builder + newStorageClient(GcsOptions options) { + return new Storage.Builder(getTransport(), getJsonFactory(), + new RetryHttpRequestInitializer(options.getGcpCredential())) + .setApplicationName(options.getAppName()); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UserCodeException.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UserCodeException.java new file mode 100644 index 0000000000000..a0bfed1626f92 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UserCodeException.java @@ -0,0 +1,132 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Objects; + +/** + * An exception that was thrown in user-code. Sets the stack trace + * from the first time execution enters user code down through the + * rest of the user's stack frames until the exception is + * reached. + */ +public class UserCodeException extends RuntimeException { + private static final Logger LOG = LoggerFactory.getLogger(UserCodeException.class); + + public UserCodeException(Throwable t) { + super(t); + + StackTraceElement[] currentFrames = + Thread.currentThread().getStackTrace(); + + // We're interested in getting the third stack frame here, since + // the exception stack trace includes the getStackTrace frame from + // Thread and the frame from where the UserCodeException is + // actually thrown. If there aren't more than two frames, + // something is odd about where the exception was thrown, so leave + // the stack trace alone and allow it to propagate. + // + // For example, if an exception in user code has a stack trace like this: + // + // java.lang.NullPointerException + // at com.google.cloud.dataflow.sdk.examples. + // SimpleWordCount$ExtractWordsFn.dieHere(SimpleWordCount.java:23) + // at com.google.cloud.dataflow.sdk.examples. + // SimpleWordCount$ExtractWordsFn. + // processElement(SimpleWordCount.java:27) + // at com.google.cloud.dataflow.sdk. + // DoFnRunner.processElement(DoFnRunner.java:95) <-- caught here + // at com.google.cloud.dataflow.sdk. + // worker.NormalParDoFn.processElement(NormalParDoFn.java:119) + // at com.google.cloud.dataflow.sdk. + // worker.executor.ParDoOperation.process(ParDoOperation.java:65) + // at com.google.cloud.dataflow.sdk. + // worker.executor.ReadOperation.start(ReadOperation.java:65) + // at com.google.cloud.dataflow.sdk. + // worker.executor.MapTaskExecutor.execute(MapTaskExecutor.java:79) + // at com.google.cloud.dataflow.sdk. + // worker.DataflowWorkerHarness.main(DataflowWorkerHarness.java:95) + // + // It would be truncated to: + // + // java.lang.NullPointerException + // at com.google.cloud.dataflow.sdk.examples. + // SimpleWordCount$ExtractWordsFn.dieHere(SimpleWordCount.java:23) + // at com.google.cloud.dataflow.sdk.examples. + // SimpleWordCount$ExtractWordsFn. + // processElement(SimpleWordCount.java:27) + // + // However, we need to get the third stack frame from the + // getStackTrace, since after catching the error in DoFnRunner, + // the trace is two frames deeper by the time we get it: + // + // [0] java.lang.Thread.getStackTrace(Thread.java:1568) + // [1] com.google.cloud.dataflow.sdk. + // UserCodeException.(UserCodeException.java:16) + // [2] com.google.cloud.dataflow.sdk. + // DoFnRunner.processElement(DoFnRunner.java:95) <-- common frame + // + // We then proceed to truncate the original exception at the + // common frame, setting the UserCodeException's cause to the + // truncated stack trace. + + // Check to make sure the stack is > 2 deep. + if (currentFrames.length <= 2) { + LOG.error("Expecting stack trace to be > 2 frames long."); + return; + } + + // Perform some checks to make sure javac doesn't change from below us. + if (!Objects.equals(currentFrames[1].getClassName(), getClass().getName())) { + LOG.error("Expected second frame coming from Thread.currentThread.getStackTrace() " + + "to be {}, was: {}", getClass().getName(), currentFrames[1].getClassName()); + return; + } + if (Objects.equals(currentFrames[2].getClassName(), currentFrames[1].getClassName())) { + LOG.error("Javac's Thread.CurrentThread.getStackTrace() changed unexpectedly."); + return; + } + + // Now that all checks have passed, select the common frame. + StackTraceElement callingFrame = currentFrames[2]; + // Truncate the user-level stack trace below where the + // UserCodeException was thrown. + truncateStackTrace(callingFrame, t); + } + + /** + * Truncates this Throwable's stack frame at the given frame, + * removing all frames below. + */ + private void truncateStackTrace( + StackTraceElement currentFrame, Throwable t) { + int index = 0; + StackTraceElement[] stackTrace = t.getStackTrace(); + for (StackTraceElement element : stackTrace) { + if (Objects.equals(element.getClassName(), currentFrame.getClassName()) && + Objects.equals(element.getMethodName(), currentFrame.getMethodName())) { + t.setStackTrace(Arrays.copyOfRange(stackTrace, 0, index)); + break; + } + index++; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Values.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Values.java new file mode 100644 index 0000000000000..f5ce4540d931a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Values.java @@ -0,0 +1,88 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import java.util.Map; + +import javax.annotation.Nullable; + +/** + * A collection of static methods for manipulating value representations + * transfered via the Dataflow API. + */ +public final class Values { + private Values() {} // Non-instantiable + + public static Boolean asBoolean(Object value) throws ClassCastException { + @Nullable Boolean knownResult = checkKnownValue(CloudKnownType.BOOLEAN, value, Boolean.class); + if (knownResult != null) { + return knownResult; + } + return Boolean.class.cast(value); + } + + public static Double asDouble(Object value) throws ClassCastException { + @Nullable Double knownResult = checkKnownValue(CloudKnownType.FLOAT, value, Double.class); + if (knownResult != null) { + return knownResult; + } + if (value instanceof Double) { + return (Double) value; + } + return ((Float) value).doubleValue(); + } + + public static Long asLong(Object value) throws ClassCastException { + @Nullable Long knownResult = checkKnownValue(CloudKnownType.INTEGER, value, Long.class); + if (knownResult != null) { + return knownResult; + } + if (value instanceof Long) { + return (Long) value; + } + return ((Integer) value).longValue(); + } + + public static String asString(Object value) throws ClassCastException { + @Nullable String knownResult = checkKnownValue(CloudKnownType.TEXT, value, String.class); + if (knownResult != null) { + return knownResult; + } + return String.class.cast(value); + } + + @Nullable + private static T checkKnownValue(CloudKnownType type, Object value, Class clazz) { + if (!(value instanceof Map)) { + return null; + } + Map map = (Map) value; + @Nullable String typeName = (String) map.get(PropertyNames.OBJECT_TYPE_NAME); + if (typeName == null) { + return null; + } + @Nullable CloudKnownType knownType = CloudKnownType.forUri(typeName); + if (knownType == null || knownType != type) { + return null; + } + @Nullable Object scalar = map.get(PropertyNames.SCALAR_FIELD_NAME); + if (scalar == null) { + return null; + } + return knownType.parse(scalar, clazz); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/VarInt.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/VarInt.java new file mode 100644 index 0000000000000..a7399473d4b4a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/VarInt.java @@ -0,0 +1,115 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** + * Variable-length encoding for integers. + * + * Handles, in a common encoding format, signed bytes, shorts, ints, and longs. + * Takes between 1 and 10 bytes. + * Less efficient than BigEndian{Int,Long} coder for negative or large numbers. + * All negative ints are encoded using 5 bytes, longs take 10 bytes. + */ +public class VarInt { + + private static long convertIntToLongNoSignExtend(int v) { + return ((long) v) & 0xFFFFFFFFL; + } + + /** + * Encodes the given value onto the stream. + */ + public static void encode(int v, OutputStream stream) throws IOException { + encode(convertIntToLongNoSignExtend(v), stream); + } + + /** + * Encodes the given value onto the stream. + */ + public static void encode(long v, OutputStream stream) throws IOException { + do { + // Encode next 7 bits + terminator bit + long bits = v & 0x7F; + v >>>= 7; + byte b = (byte) (bits | ((v != 0) ? 0x80 : 0)); + stream.write(b); + } while (v != 0); + } + + /** + * Decodes an integer value from the given stream. + */ + public static int decodeInt(InputStream stream) throws IOException { + long r = decodeLong(stream); + if (r < 0 || r >= 1L << 32) { + throw new IOException("varint overflow " + r); + } + return (int) r; + } + + /** + * Decodes a long value from the given stream. + */ + public static long decodeLong(InputStream stream) throws IOException { + long result = 0; + int shift = 0; + int b; + do { + // Get 7 bits from next byte + b = stream.read(); + if (b < 0) { + if (shift == 0) { + throw new EOFException(); + } else { + throw new IOException("varint not terminated"); + } + } + long bits = b & 0x7F; + if (shift >= 64 || (shift == 63 && bits > 1)) { + // Out of range + throw new IOException("varint too long"); + } + result |= bits << shift; + shift += 7; + } while ((b & 0x80) != 0); + return result; + } + + /** + * Returns the length of the encoding of the given value (in bytes). + */ + public static int getLength(int v) { + return getLength(convertIntToLongNoSignExtend(v)); + } + + /** + * Returns the length of the encoding of the given value (in bytes). + */ + public static int getLength(long v) { + int result = 0; + do { + result++; + v >>>= 7; + } while (v != 0); + return result; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowUtils.java new file mode 100644 index 0000000000000..de0a8f24ba645 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowUtils.java @@ -0,0 +1,62 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.util.Base64; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +/** + * Utility functions related to serializing windows. + */ +class WindowUtils { + private static final String BUFFER_TAG_PREFIX = "buffer:"; + + /** + * Converts the given window to a base64-encoded String using the given coder. + */ + public static String windowToString(W window, Coder coder) throws IOException { + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + coder.encode(window, stream, Coder.Context.OUTER); + byte[] rawWindow = stream.toByteArray(); + return Base64.encodeBase64String(rawWindow); + } + + /** + * Parses a window from a base64-encoded String using the given coder. + */ + public static W windowFromString(String serializedWindow, Coder coder) throws IOException { + return coder.decode( + new ByteArrayInputStream(Base64.decodeBase64(serializedWindow)), + Coder.Context.OUTER); + } + + /** + * Returns a tag for storing buffered data in per-key state. + */ + public static CodedTupleTag bufferTag( + W window, Coder windowCoder, Coder elemCoder) + throws IOException { + return CodedTupleTag.of( + BUFFER_TAG_PREFIX + windowToString(window, windowCoder), elemCoder); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java new file mode 100644 index 0000000000000..de310b8271149 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java @@ -0,0 +1,368 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.CollectionCoder; +import com.google.cloud.dataflow.sdk.coders.InstantCoder; +import com.google.cloud.dataflow.sdk.coders.StandardCoder; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; + +/** + * An immutable triple of value, timestamp, and windows. + * + * @param the type of the value + */ +public class WindowedValue { + + private final V value; + private final Instant timestamp; + private final Collection windows; + + /** + * Returns a {@code WindowedValue} with the given value, timestamp, and windows. + */ + public static WindowedValue of( + V value, + Instant timestamp, + Collection windows) { + return new WindowedValue<>(value, timestamp, windows); + } + + /** + * Returns a {@code WindowedValue} with the given value, default timestamp, + * and {@code GlobalWindow}. + */ + public static WindowedValue valueInGlobalWindow(V value) { + return new WindowedValue<>(value, + new Instant(Long.MIN_VALUE), + Arrays.asList(GlobalWindow.Window.INSTANCE)); + } + + /** + * Returns a {@code WindowedValue} with the given value and default timestamp and empty windows. + */ + public static WindowedValue valueInEmptyWindows(V value) { + return new WindowedValue<>(value, + new Instant(Long.MIN_VALUE), + new ArrayList()); + } + + private WindowedValue(V value, + Instant timestamp, + Collection windows) { + this.value = value; + this.timestamp = timestamp; + this.windows = windows; + } + + /** + * Returns a new {@code WindowedValue} that is a copy of this one, but with a different value. + */ + public WindowedValue withValue(V value) { + return new WindowedValue<>(value, this.timestamp, this.windows); + } + + /** + * Returns the value of this {@code WindowedValue}. + */ + public V getValue() { + return value; + } + + /** + * Returns the timestamp of this {@code WindowedValue}. + */ + public Instant getTimestamp() { + return timestamp; + } + + /** + * Returns the windows of this {@code WindowedValue}. + */ + public Collection getWindows() { + return windows; + } + + /** + * Returns the {@code Coder} to use for a {@code WindowedValue}, + * using the given valueCoder and windowCoder. + */ + public static WindowedValueCoder getFullCoder( + Coder valueCoder, + Coder windowCoder) { + return FullWindowedValueCoder.of(valueCoder, windowCoder); + } + + /** + * Returns the {@code ValueOnlyCoder} from the given valueCoder. + */ + public static WindowedValueCoder getValueOnlyCoder(Coder valueCoder) { + return ValueOnlyWindowedValueCoder.of(valueCoder); + } + + @Override + public boolean equals(Object o) { + if (o instanceof WindowedValue) { + WindowedValue that = (WindowedValue) o; + if (that.timestamp.isEqual(timestamp) && that.windows.size() == windows.size()) { + for (Iterator thatIterator = that.windows.iterator(), thisIterator = windows.iterator(); + thatIterator.hasNext() && thisIterator.hasNext(); + /* do nothng */) { + if (!thatIterator.next().equals(thisIterator.next())) { + return false; + } + } + return true; + } + } + return false; + } + + @Override + public int hashCode() { + return Objects.hash(value, timestamp, Arrays.hashCode(windows.toArray())); + } + + @Override + public String toString() { + return "[WindowedValue: " + value + ", timestamp: " + timestamp.getMillis() + + ", windows: " + windows + "]"; + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Abstract class for {@code WindowedValue} coder. + */ + public abstract static class WindowedValueCoder + extends StandardCoder> { + final Coder valueCoder; + + WindowedValueCoder(Coder valueCoder) { + this.valueCoder = checkNotNull(valueCoder); + } + + /** + * Returns the value coder. + */ + public Coder getValueCoder() { + return valueCoder; + } + + /** + * Returns a new {@code WindowedValueCoder} that is a copy of this one, + * but with a different value coder. + */ + public abstract WindowedValueCoder withValueCoder(Coder valueCoder); + } + + /** + * Coder for {@code WindowedValue}. + */ + public static class FullWindowedValueCoder extends WindowedValueCoder { + private final Coder windowCoder; + // Precompute and cache the coder for a list of windows. + private final Coder> windowsCoder; + + public static FullWindowedValueCoder of( + Coder valueCoder, + Coder windowCoder) { + return new FullWindowedValueCoder<>(valueCoder, windowCoder); + } + + @JsonCreator + public static FullWindowedValueCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> components) { + checkArgument(components.size() == 2, + "Expecting 2 components, got " + components.size()); + return of(components.get(0), + (Coder) components.get(1)); + } + + @SuppressWarnings("unchecked") + FullWindowedValueCoder(Coder valueCoder, + Coder windowCoder) { + super(valueCoder); + this.windowCoder = checkNotNull(windowCoder); + // It's not possible to statically type-check correct use of the + // windowCoder (we have to ensure externally that we only get + // windows of the class handled by windowCoder), so type + // windowsCoder in a way that makes encode() and decode() work + // right, and cast the window type away here. + this.windowsCoder = (Coder) CollectionCoder.of(this.windowCoder); + } + + public Coder getWindowCoder() { + return windowCoder; + } + + public Coder> getWindowsCoder() { + return windowsCoder; + } + + @Override + public WindowedValueCoder withValueCoder(Coder valueCoder) { + return new FullWindowedValueCoder<>(valueCoder, windowCoder); + } + + @Override + public void encode(WindowedValue windowedElem, + OutputStream outStream, + Context context) + throws CoderException, IOException { + Context nestedContext = context.nested(); + valueCoder.encode(windowedElem.getValue(), outStream, nestedContext); + InstantCoder.of().encode( + windowedElem.getTimestamp(), outStream, nestedContext); + windowsCoder.encode(windowedElem.getWindows(), outStream, nestedContext); + } + + @Override + public WindowedValue decode(InputStream inStream, Context context) + throws CoderException, IOException { + Context nestedContext = context.nested(); + T value = valueCoder.decode(inStream, nestedContext); + Instant timestamp = InstantCoder.of().decode(inStream, nestedContext); + Collection windows = + windowsCoder.decode(inStream, nestedContext); + return WindowedValue.of(value, timestamp, windows); + } + + @Override + public boolean isDeterministic() { + return valueCoder.isDeterministic() && windowCoder.isDeterministic(); + } + + @Override + public void registerByteSizeObserver(WindowedValue value, + ElementByteSizeObserver observer, + Context context) throws Exception { + valueCoder.registerByteSizeObserver(value.getValue(), observer, context); + InstantCoder.of().registerByteSizeObserver(value.getTimestamp(), observer, context); + windowsCoder.registerByteSizeObserver(value.getWindows(), observer, context); + } + + @Override + public CloudObject asCloudObject() { + CloudObject result = super.asCloudObject(); + addBoolean(result, PropertyNames.IS_WRAPPER, true); + return result; + } + + @Override + public List> getCoderArguments() { + return null; + } + + @Override + public List> getComponents() { + return Arrays.>asList(valueCoder, windowCoder); + } + } + + /** + * Coder for {@code WindowedValue}. + * + *

A {@code ValueOnlyWindowedValueCoder} only encodes and decodes the value. It drops + * timestamp and windows for encoding, and uses defaults timestamp, and windows for decoding. + */ + public static class ValueOnlyWindowedValueCoder extends WindowedValueCoder { + + public static ValueOnlyWindowedValueCoder of( + Coder valueCoder) { + return new ValueOnlyWindowedValueCoder<>(valueCoder); + } + + @JsonCreator + public static ValueOnlyWindowedValueCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> components) { + checkArgument(components.size() == 1, "Expecting 1 component, got " + components.size()); + return of(components.get(0)); + } + + ValueOnlyWindowedValueCoder(Coder valueCoder) { + super(valueCoder); + } + + @Override + public WindowedValueCoder withValueCoder(Coder valueCoder) { + return new ValueOnlyWindowedValueCoder<>(valueCoder); + } + + @Override + public void encode(WindowedValue windowedElem, OutputStream outStream, Context context) + throws CoderException, IOException { + valueCoder.encode(windowedElem.getValue(), outStream, context); + } + + @Override + public WindowedValue decode(InputStream inStream, Context context) + throws CoderException, IOException { + T value = valueCoder.decode(inStream, context); + return WindowedValue.valueInGlobalWindow(value); + } + + @Override + public boolean isDeterministic() { + return valueCoder.isDeterministic(); + } + + @Override + public void registerByteSizeObserver( + WindowedValue value, ElementByteSizeObserver observer, Context context) + throws Exception { + valueCoder.registerByteSizeObserver(value.getValue(), observer, context); + } + + @Override + public CloudObject asCloudObject() { + CloudObject result = super.asCloudObject(); + addBoolean(result, PropertyNames.IS_WRAPPER, true); + return result; + } + + @Override + public List> getCoderArguments() { + return Arrays.>asList(valueCoder); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Counter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Counter.java new file mode 100644 index 0000000000000..8b5f636ac5da3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Counter.java @@ -0,0 +1,730 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.AND; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.OR; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SET; + +import com.google.common.reflect.TypeToken; + +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; +import java.util.logging.Logger; + +/** + * A Counter enables the aggregation of a stream of values over time. The + * cumulative aggregate value is updated as new values are added, or it can be + * reset to a new value. Multiple kinds of aggregation are supported depending + * on the type of the counter. + * + *

Counters compare using value equality of their name, kind, and + * cumulative value. Equal counters should have equal toString()s. + * + * @param the type of values aggregated by this counter + */ +public abstract class Counter { + private static final Logger LOG = Logger.getLogger(Counter.class.getName()); + + /** + * Possible kinds of counter aggregation. + */ + public static enum AggregationKind { + + /** + * Computes the sum of all added values. + * Applicable to {@link Integer}, {@link Long}, and {@link Double} values. + */ + SUM, + + /** + * Computes the maximum value of all added values. + * Applicable to {@link Integer}, {@link Long}, and {@link Double} values. + */ + MAX, + + /** + * Computes the minimum value of all added values. + * Applicable to {@link Integer}, {@link Long}, and {@link Double} values. + */ + MIN, + + /** + * Computes the arithmetic mean of all added values. Applicable to + * {@link Integer}, {@link Long}, and {@link Double} values. + */ + MEAN, + + /** + * Computes the set of all added values. Applicable to {@link Integer}, + * {@link Long}, {@link Double}, and {@link String} values. + */ + SET, + + /** + * Computes boolean AND over all added values. + * Applicable only to {@link Boolean} values. + */ + AND, + + /** + * Computes boolean OR over all added values. Applicable only to + * {@link Boolean} values. + */ + OR + // TODO: consider adding VECTOR_SUM, HISTOGRAM, KV_SET, PRODUCT, TOP. + } + + /** + * Constructs a new {@link Counter} that aggregates {@link Integer}, values + * according to the desired aggregation kind. The supported aggregation kinds + * are {@link AggregationKind#SUM}, {@link AggregationKind#MIN}, + * {@link AggregationKind#MAX}, {@link AggregationKind#MEAN}, and + * {@link AggregationKind#SET}. This is a convenience wrapper over a + * {@link Counter} implementation that aggregates {@link Long} values. This is + * useful when the application handles (boxed) {@link Integer} values which + * are not readily convertible to the (boxed) {@link Long} values otherwise + * expected by the {@link Counter} implementation aggregating {@link Long} + * values. + * + * @param name the name of the new counter + * @param kind the new counter's aggregation kind + * @return the newly constructed Counter + * @throws IllegalArgumentException if the aggregation kind is not supported + */ + public static Counter ints(String name, AggregationKind kind) { + return new IntegerCounter(name, kind); + } + + /** + * Constructs a new {@link Counter} that aggregates {@link Long} values + * according to the desired aggregation kind. The supported aggregation kinds + * are {@link AggregationKind#SUM}, {@link AggregationKind#MIN}, + * {@link AggregationKind#MAX}, {@link AggregationKind#MEAN}, and + * {@link AggregationKind#SET}. + * + * @param name the name of the new counter + * @param kind the new counter's aggregation kind + * @return the newly constructed Counter + * @throws IllegalArgumentException if the aggregation kind is not supported + */ + public static Counter longs(String name, AggregationKind kind) { + return new LongCounter(name, kind); + } + + /** + * Constructs a new {@link Counter} that aggregates {@link Double} values + * according to the desired aggregation kind. The supported aggregation kinds + * are {@link AggregationKind#SUM}, {@link AggregationKind#MIN}, + * {@link AggregationKind#MAX}, {@link AggregationKind#MEAN}, and + * {@link AggregationKind#SET}. + * + * @param name the name of the new counter + * @param kind the new counter's aggregation kind + * @return the newly constructed Counter + * @throws IllegalArgumentException if the aggregation kind is not supported + */ + public static Counter doubles(String name, AggregationKind kind) { + return new DoubleCounter(name, kind); + } + + /** + * Constructs a new {@link Counter} that aggregates {@link Boolean} values + * according to the desired aggregation kind. The only supported aggregation + * kinds are {@link AggregationKind#AND} and {@link AggregationKind#OR}. + * + * @param name the name of the new counter + * @param kind the new counter's aggregation kind + * @return the newly constructed Counter + * @throws IllegalArgumentException if the aggregation kind is not supported + */ + public static Counter booleans(String name, AggregationKind kind) { + return new BooleanCounter(name, kind); + } + + /** + * Constructs a new {@link Counter} that aggregates {@link String} values + * according to the desired aggregation kind. The only supported aggregation + * kind is {@link AggregationKind#SET}. + * + * @param name the name of the new counter + * @param kind the new counter's aggregation kind + * @return the newly constructed Counter + * @throws IllegalArgumentException if the aggregation kind is not supported + */ + public static Counter strings(String name, AggregationKind kind) { + return new StringCounter(name, kind); + } + + + ////////////////////////////////////////////////////////////////////////////// + + /** + * Adds a new value to the aggregation stream. Returns this (to allow method + * chaining). + */ + public abstract Counter addValue(T value); + + /** + * Resets the aggregation stream to this new value. Returns this (to allow + * method chaining). + */ + public Counter resetToValue(T value) { + return resetToValue(-1, value); + } + + /** + * Resets the aggregation stream to this new value. Returns this (to allow + * method chaining). The value of elementCount must be -1 for non-MEAN + * aggregations. The value of elementCount must be non-negative for MEAN + * aggregation. + */ + public synchronized Counter resetToValue(long elementCount, T value) { + aggregate = value; + deltaAggregate = value; + + if (kind.equals(MEAN)) { + if (elementCount < 0) { + throw new AssertionError( + "elementCount must be non-negative for MEAN aggregation"); + } + count = elementCount; + deltaCount = elementCount; + } else { + if (elementCount != -1) { + throw new AssertionError( + "elementCount must be -1 for non-MEAN aggregations"); + } + count = 0; + deltaCount = 0; + } + + if (kind.equals(SET)) { + set.clear(); + set.add(value); + deltaSet = new HashSet<>(); + deltaSet.add(value); + } + return this; + } + + /** Resets the counter's delta value to have no values accumulated. */ + public abstract void resetDelta(); + + /** + * Returns the counter's name. + */ + public String getName() { + return name; + } + + /** + * Returns the counter's aggregation kind. + */ + public AggregationKind getKind() { + return kind; + } + + /** + * Returns the counter's type. + */ + public Class getType() { + return new TypeToken(getClass()) {}.getRawType(); + } + + /** + * Returns the aggregated value, or the sum for MEAN aggregation, either + * total or, if delta, since the last update extraction or resetDelta, + * if not a SET aggregation. + */ + public T getAggregate(boolean delta) { + return delta ? deltaAggregate : aggregate; + } + + /** + * Returns the number of aggregated values, either total or, if + * delta, since the last update extraction or resetDelta, if a MEAN + * aggregation. + */ + public long getCount(boolean delta) { + return delta ? deltaCount : count; + } + + /** + * Returns the set of all aggregated values, either total or, if + * delta, since the last update extraction or resetDelta, if a SET + * aggregation. + */ + public Set getSet(boolean delta) { + return delta ? deltaSet : set; + } + + /** + * Returns a string representation of the Counter. Useful for debugging logs. + * Example return value: "ElementCount:SUM(15)". + */ + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getName()); + sb.append(":"); + sb.append(getKind()); + sb.append("("); + switch (kind) { + case SUM: + case MAX: + case MIN: + case AND: + case OR: + sb.append(aggregate); + break; + case MEAN: + sb.append(aggregate); + sb.append("/"); + sb.append(count); + break; + case SET: + sb.append(set); + break; + default: + throw illegalArgumentException(); + } + sb.append(")"); + + return sb.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (o instanceof Counter) { + Counter that = (Counter) o; + return this.name.equals(that.name) + && this.kind == that.kind + && this.getClass().equals(that.getClass()) + && this.count == that.count + && Objects.equals(this.aggregate, that.aggregate) + && Objects.equals(this.set, that.set); + } else { + return false; + } + } + + @Override + public int hashCode() { + return Objects.hash(getClass(), name, kind, aggregate, count, set); + } + + /** + * Returns whether this Counter is compatible with that Counter. If + * so, they can be merged into a single Counter. + */ + public boolean isCompatibleWith(Counter that) { + return this.name.equals(that.name) + && this.kind == that.kind + && this.getClass().equals(that.getClass()); + } + + + ////////////////////////////////////////////////////////////////////////////// + + /** The name of this counter. */ + protected final String name; + + /** The kind of aggregation function to apply to this counter. */ + protected final AggregationKind kind; + + /** The total cumulative aggregation value. Holds sum for MEAN aggregation. */ + protected T aggregate; + + /** The cumulative aggregation value since the last update extraction. */ + protected T deltaAggregate; + + /** The total number of aggregated values. Useful for MEAN aggregation. */ + protected long count; + + /** The number of aggregated values since the last update extraction. */ + protected long deltaCount; + + /** Holds the set of all aggregated values. Used only for SET aggregation. */ + protected Set set; + + /** Holds the set of aggregated values since the last update extraction. */ + protected Set deltaSet; + + protected Counter(String name, AggregationKind kind) { + this.name = name; + this.kind = kind; + this.count = 0; + this.deltaCount = 0; + if (kind.equals(SET)) { + set = new HashSet<>(); + deltaSet = new HashSet<>(); + } + } + + + ////////////////////////////////////////////////////////////////////////////// + + /** + * Implements a {@link Counter} for {@link Long} values. + */ + private static class LongCounter extends Counter { + + /** Initializes a new {@link Counter} for {@link Long} values. */ + private LongCounter(String name, AggregationKind kind) { + super(name, kind); + switch (kind) { + case SUM: + case MEAN: + aggregate = deltaAggregate = 0L; + break; + case MAX: + aggregate = deltaAggregate = Long.MIN_VALUE; + break; + case MIN: + aggregate = deltaAggregate = Long.MAX_VALUE; + break; + case SET: + break; + default: + throw illegalArgumentException(); + } + } + + @Override + public synchronized LongCounter addValue(Long value) { + switch (kind) { + case SUM: + aggregate += value; + deltaAggregate += value; + break; + case MEAN: + aggregate += value; + deltaAggregate += value; + count++; + deltaCount++; + break; + case MAX: + aggregate = Math.max(aggregate, value); + deltaAggregate = Math.max(deltaAggregate, value); + break; + case MIN: + aggregate = Math.min(aggregate, value); + deltaAggregate = Math.min(deltaAggregate, value); + break; + case SET: + set.add(value); + deltaSet.add(value); + break; + default: + throw illegalArgumentException(); + } + return this; + } + + @Override + public synchronized void resetDelta() { + switch (kind) { + case SUM: + deltaAggregate = 0L; + break; + case MEAN: + deltaAggregate = 0L; + deltaCount = 0; + break; + case MAX: + deltaAggregate = Long.MIN_VALUE; + break; + case MIN: + deltaAggregate = Long.MAX_VALUE; + break; + case SET: + deltaSet = new HashSet<>(); + break; + default: + throw illegalArgumentException(); + } + } + } + + /** + * Implements a {@link Counter} for {@link Double} values. + */ + private static class DoubleCounter extends Counter { + + /** Initializes a new {@link Counter} for {@link Double} values. */ + private DoubleCounter(String name, AggregationKind kind) { + super(name, kind); + switch (kind) { + case SUM: + case MEAN: + aggregate = deltaAggregate = 0.0; + break; + case MAX: + aggregate = deltaAggregate = Double.MIN_VALUE; + break; + case MIN: + aggregate = deltaAggregate = Double.MAX_VALUE; + break; + case SET: + break; + default: + throw illegalArgumentException(); + } + } + + @Override + public synchronized DoubleCounter addValue(Double value) { + switch (kind) { + case SUM: + aggregate += value; + deltaAggregate += value; + break; + case MEAN: + aggregate += value; + deltaAggregate += value; + count++; + deltaCount++; + break; + case MAX: + aggregate = Math.max(aggregate, value); + deltaAggregate = Math.max(deltaAggregate, value); + break; + case MIN: + aggregate = Math.min(aggregate, value); + deltaAggregate = Math.min(deltaAggregate, value); + break; + case SET: + set.add(value); + deltaSet.add(value); + break; + default: + throw illegalArgumentException(); + } + return this; + } + + @Override + public synchronized void resetDelta() { + switch (kind) { + case SUM: + deltaAggregate = 0.0; + break; + case MEAN: + deltaAggregate = 0.0; + deltaCount = 0; + break; + case MAX: + deltaAggregate = Double.MIN_VALUE; + break; + case MIN: + deltaAggregate = Double.MAX_VALUE; + break; + case SET: + deltaSet = new HashSet<>(); + break; + default: + throw illegalArgumentException(); + } + } + } + + /** + * Implements a {@link Counter} for {@link Boolean} values. + */ + private static class BooleanCounter extends Counter { + + /** Initializes a new {@link Counter} for {@link Boolean} values. */ + private BooleanCounter(String name, AggregationKind kind) { + super(name, kind); + if (kind.equals(AND)) { + aggregate = deltaAggregate = true; + } else if (kind.equals(OR)) { + aggregate = deltaAggregate = false; + } else { + throw illegalArgumentException(); + } + } + + @Override + public synchronized BooleanCounter addValue(Boolean value) { + if (kind.equals(AND)) { + aggregate &= value; + deltaAggregate &= value; + } else { // kind.equals(OR)) + aggregate |= value; + deltaAggregate |= value; + } + return this; + } + + @Override + public synchronized void resetDelta() { + switch (kind) { + case AND: + deltaAggregate = true; + break; + case OR: + deltaAggregate = false; + break; + default: + throw illegalArgumentException(); + } + } + } + + /** + * Implements a {@link Counter} for {@link String} values. + */ + private static class StringCounter extends Counter { + + /** Initializes a new {@link Counter} for {@link String} values. */ + private StringCounter(String name, AggregationKind kind) { + super(name, kind); + if (!kind.equals(SET)) { + throw illegalArgumentException(); + } + } + + @Override + public synchronized StringCounter addValue(String value) { + set.add(value); + deltaSet.add(value); + return this; + } + + @Override + public synchronized void resetDelta() { + switch (kind) { + case SET: + deltaSet = new HashSet<>(); + break; + default: + throw illegalArgumentException(); + } + } + } + + /** + * Implements a {@link Counter} for {@link Integer} values. + */ + private static class IntegerCounter extends Counter { + + /** Initializes a new {@link Counter} for {@link Integer} values. */ + private IntegerCounter(String name, AggregationKind kind) { + super(name, kind); + switch (kind) { + case SUM: + case MEAN: + aggregate = deltaAggregate = 0; + break; + case MAX: + aggregate = deltaAggregate = Integer.MIN_VALUE; + break; + case MIN: + aggregate = deltaAggregate = Integer.MAX_VALUE; + break; + case SET: + break; + default: + throw illegalArgumentException(); + } + } + + @Override + public synchronized IntegerCounter addValue(Integer value) { + switch (kind) { + case SUM: + aggregate += value; + deltaAggregate += value; + break; + case MEAN: + aggregate += value; + deltaAggregate += value; + count++; + deltaCount++; + break; + case MAX: + aggregate = Math.max(aggregate, value); + deltaAggregate = Math.max(deltaAggregate, value); + break; + case MIN: + aggregate = Math.min(aggregate, value); + deltaAggregate = Math.min(deltaAggregate, value); + break; + case SET: + set.add(value); + deltaSet.add(value); + break; + default: + throw illegalArgumentException(); + } + return this; + } + + @Override + public synchronized void resetDelta() { + switch (kind) { + case SUM: + deltaAggregate = 0; + break; + case MEAN: + deltaAggregate = 0; + deltaCount = 0; + break; + case MAX: + deltaAggregate = Integer.MIN_VALUE; + break; + case MIN: + deltaAggregate = Integer.MAX_VALUE; + break; + case SET: + deltaSet = new HashSet<>(); + break; + default: + throw illegalArgumentException(); + } + } + } + + + ////////////////////////////////////////////////////////////////////////////// + + /** + * Constructs an {@link IllegalArgumentException} explaining that this + * {@link Counter}'s aggregation kind is not supported by its value type. + */ + protected IllegalArgumentException illegalArgumentException() { + return new IllegalArgumentException("Cannot compute " + kind + + " aggregation over " + getType().getSimpleName() + " values."); + } + + + ////////////////////////////////////////////////////////////////////////////// + + // For testing. + synchronized T getTotalAggregate() { return aggregate; } + synchronized T getDeltaAggregate() { return deltaAggregate; } + synchronized long getTotalCount() { return count; } + synchronized long getDeltaCount() { return deltaCount; } + synchronized Set getTotalSet() { return set; } + synchronized Set getDeltaSet() { return deltaSet; } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/CounterSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/CounterSet.java new file mode 100644 index 0000000000000..a9e83f3237919 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/CounterSet.java @@ -0,0 +1,152 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import java.util.AbstractSet; +import java.util.HashMap; +import java.util.Iterator; + +/** + * A CounterSet maintains a set of {@link Counter}s. + * + *

Thread-safe. + */ +public class CounterSet extends AbstractSet> { + + /** Registered counters. */ + private final HashMap> counters = new HashMap<>(); + + private final AddCounterMutator addCounterMutator = new AddCounterMutator(); + + /** + * Constructs a CounterSet containing the given Counters. + */ + public CounterSet(Counter... counters) { + for (Counter counter : counters) { + addNewCounter(counter); + } + } + + /** + * Returns an object that supports adding additional counters into + * this CounterSet. + */ + public AddCounterMutator getAddCounterMutator() { + return addCounterMutator; + } + + /** + * Adds a new counter, throwing an exception if a counter of the + * same name already exists. + */ + public void addNewCounter(Counter counter) { + if (!addCounter(counter)) { + throw new IllegalArgumentException( + "Counter " + counter + " duplicates an existing counter in " + this); + } + } + + /** + * Adds the given Counter to this CounterSet. + * + *

If a counter with the same name already exists, it will be + * reused, as long as it is compatible. + * + * @return the Counter that was reused, or added + * @throws IllegalArgumentException if the a counter with the same + * name but an incompatible kind had already been added + */ + public synchronized Counter addOrReuseCounter(Counter counter) { + Counter oldCounter = counters.get(counter.getName()); + if (oldCounter == null) { + // A new counter. + counters.put(counter.getName(), counter); + return counter; + } + if (counter.isCompatibleWith(oldCounter)) { + // Return the counter to reuse. + @SuppressWarnings("unchecked") + Counter compatibleCounter = (Counter) oldCounter; + return compatibleCounter; + } + throw new IllegalArgumentException( + "Counter " + counter + " duplicates incompatible counter " + + oldCounter + " in " + this); + } + + /** + * Adds a counter. Returns {@code true} if the counter was added to the set + * and false if the given counter was {@code null} or it already existed in + * the set. + * + * @param counter to register + */ + public boolean addCounter(Counter counter) { + return add(counter); + } + + /** + * Returns the Counter with the given name in this CounterSet; + * returns null if no such Counter exists. + */ + public synchronized Counter getExistingCounter(String name) { + return counters.get(name); + } + + @Override + public synchronized Iterator> iterator() { + return counters.values().iterator(); + } + + @Override + public synchronized int size() { + return counters.size(); + } + + @Override + public synchronized boolean add(Counter e) { + if (null == e) { + return false; + } + if (counters.containsKey(e.getName())) { + return false; + } + counters.put(e.getName(), e); + return true; + } + + /** + * A nested class that supports adding additional counters into the + * enclosing CounterSet. This is useful as a mutator; hiding other + * public methods of the CounterSet. + */ + public class AddCounterMutator { + /** + * Adds the given Counter into the enclosing CounterSet. + * + *

If a counter with the same name already exists, it will be + * reused, as long as it has the same type. + * + * @return the Counter that was reused, or added + * @throws IllegalArgumentException if the a counter with the same + * name but an incompatible kind had already been added + */ + public Counter addCounter(Counter counter) { + return addOrReuseCounter(counter); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObservable.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObservable.java new file mode 100644 index 0000000000000..447dadcb8ef75 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObservable.java @@ -0,0 +1,41 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +/** + * An interface for things that allow observing the size in bytes of + * encoded values of type {@code T}. + * + * @param the type of the values being observed + */ +public interface ElementByteSizeObservable { + /** + * Returns whether {@link #registerByteSizeObserver} is cheap enough + * to call for every element, that is, if this + * {@code ElementByteSizeObservable} can calculate the byte size of + * the element to be coded in roughly constant time (or lazily). + */ + public boolean isRegisterByteSizeObserverCheap(T value); + + /** + * Notifies the {@code ElementByteSizeObserver} about the byte size + * of the encoded value using this {@code ElementByteSizeObservable}. + */ + public void registerByteSizeObserver(T value, + ElementByteSizeObserver observer) + throws Exception; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObservableIterable.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObservableIterable.java new file mode 100644 index 0000000000000..f8f727090237a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObservableIterable.java @@ -0,0 +1,63 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import java.util.ArrayList; +import java.util.List; +import java.util.Observer; + +/** + * An abstract class used for iterables that notify observers about size in + * bytes of their elements, as they are being iterated over. + * + * @param the type of elements returned by this iterable + * @param type type of iterator returned by this iterable + */ +public abstract class ElementByteSizeObservableIterable< + V, VI extends ElementByteSizeObservableIterator> + implements Iterable { + private List observers = new ArrayList<>(); + + /** + * Derived classes override this method to return an iterator for this + * iterable. + */ + protected abstract VI createIterator(); + + /** + * Sets the observer, which will observe the iterator returned in + * the next call to iterator() method. Future calls to iterator() + * won't be observed, unless an observer is set again. + */ + public void addObserver(Observer observer) { + observers.add(observer); + } + + /** + * Returns a new iterator for this iterable. If an observer was set in + * a previous call to setObserver(), it will observe the iterator returned. + */ + @Override + public VI iterator() { + VI iterator = createIterator(); + for (Observer observer : observers) { + iterator.addObserver(observer); + } + observers.clear(); + return iterator; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObservableIterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObservableIterator.java new file mode 100644 index 0000000000000..50c9add0edaab --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObservableIterator.java @@ -0,0 +1,36 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import java.util.Iterator; +import java.util.Observable; + +/** + * An abstract class used for iterators that notify observers about size in + * bytes of their elements, as they are being iterated over. The subclasses + * need to implement the standard Iterator interface and call method + * notifyValueReturned() for each element read and/or iterated over. + * + * @param value type + */ +public abstract class ElementByteSizeObservableIterator + extends Observable implements Iterator { + protected final void notifyValueReturned(long byteSize) { + setChanged(); + notifyObservers(byteSize); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObserver.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObserver.java new file mode 100644 index 0000000000000..9cccb4365c6f3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ElementByteSizeObserver.java @@ -0,0 +1,84 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import java.util.Observable; +import java.util.Observer; + +/** + * An observer that gets notified when additional bytes are read + * and/or used. It adds all bytes into a local counter. When the + * observer gets advanced via the next() call, it adds the total byte + * count to the specified counter, and prepares for the next element. + */ +public class ElementByteSizeObserver implements Observer { + private final Counter counter; + private boolean isLazy = false; + private long totalSize = 0; + + public ElementByteSizeObserver(Counter counter) { + this.counter = counter; + } + + /** + * Sets byte counting for the current element as lazy. That is, the + * observer will get notified of the element's byte count only as + * element's pieces are being processed or iterated over. + */ + public void setLazy() { + isLazy = true; + } + + /** + * Returns whether byte counting for the current element is lazy, that is, + * whether the observer gets notified of the element's byte count only as + * element's pieces are being processed or iterated over. + */ + public boolean getIsLazy() { + return isLazy; + } + + /** + * Updates the observer with a context specified, but without an instance of + * the Observable. + */ + public void update(Object obj) { + update(null, obj); + } + + @Override + public void update(Observable obs, Object obj) { + if (obj instanceof Long) { + totalSize += (Long) obj; + } else if (obj instanceof Integer) { + totalSize += (Integer) obj; + } else { + throw new AssertionError("unexpected parameter object"); + } + } + + /** + * Advances the observer to the next element. Adds the current total byte + * size to the counter, and prepares the observer for the next element. + */ + public void advance() { + counter.addValue(totalSize); + + totalSize = 0; + isLazy = false; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ForwardingReiterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ForwardingReiterator.java new file mode 100644 index 0000000000000..f3008232a1074 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ForwardingReiterator.java @@ -0,0 +1,83 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * A {@link Reiterator} which forwards to another {@code Reiterator}, useful for + * implementing {@code Reiterator} wrappers. + * + * @param the type of elements returned by this iterator + */ +public abstract class ForwardingReiterator + implements Reiterator, Cloneable { + private Reiterator base; + + /** + * Constructs a {@link ForwardingReiterator}. + * @param base supplies a base reiterator to forward requests to. This + * reiterator will be used directly; it will not be copied by the constructor. + */ + public ForwardingReiterator(Reiterator base) { + this.base = checkNotNull(base); + } + + @Override + protected ForwardingReiterator clone() { + ForwardingReiterator result; + try { + result = (ForwardingReiterator) super.clone(); + } catch (CloneNotSupportedException e) { + throw new AssertionError( + "Object.clone() for a ForwardingReiterator threw " + + "CloneNotSupportedException; this should not happen, " + + "since ForwardingReiterator implements Cloneable.", + e); + } + result.base = base.copy(); + return result; + } + + @Override + public boolean hasNext() { + return base.hasNext(); + } + + @Override + public T next() { + return base.next(); + } + + @Override + public void remove() { + base.remove(); + } + + /** + * {@inheritDoc} + * + *

This implementation uses {@link #clone} to construct a duplicate of the + * {@link Reiterator}. Derived classes must either implement + * {@link Cloneable} semantics, or must provide an alternative implementation + * of this method. + */ + @Override + public ForwardingReiterator copy() { + return clone(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Metric.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Metric.java new file mode 100644 index 0000000000000..23a590743b21f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Metric.java @@ -0,0 +1,45 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +/** + * A metric (e.g., CPU usage) that can be reported by a worker. + * + * @param the type of the metric's value + */ +public abstract class Metric { + String name; + T value; + + public Metric(String name, T value) { + this.name = name; + this.value = value; + } + + public String getName() { return name; } + + public T getValue() { return value; } + + /** + * A double-valued Metric. + */ + public static class DoubleMetric extends Metric { + public DoubleMetric(String name, double value) { + super(name, value); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/PeekingReiterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/PeekingReiterator.java new file mode 100644 index 0000000000000..d139380c65c12 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/PeekingReiterator.java @@ -0,0 +1,98 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import java.util.NoSuchElementException; + +/** + * A {@link Reiterator} that supports one-element lookahead during iteration. + * + * @param the type of elements returned by this iterator + */ +public final class PeekingReiterator implements Reiterator { + private T nextElement; + private boolean nextElementComputed; + private final Reiterator iterator; + + public PeekingReiterator(Reiterator iterator) { + this.iterator = checkNotNull(iterator); + } + + PeekingReiterator(PeekingReiterator it) { + this.iterator = checkNotNull(it).iterator.copy(); + this.nextElement = it.nextElement; + this.nextElementComputed = it.nextElementComputed; + } + + @Override + public boolean hasNext() { + computeNext(); + return nextElementComputed; + } + + @Override + public T next() { + T result = peek(); + nextElementComputed = false; + return result; + } + + /** + * {@inheritDoc} + * + *

If {@link #peek} is called, {@code remove} is disallowed until + * {@link #next} has been subsequently called. + */ + @Override + public void remove() { + checkState(!nextElementComputed, + "After peek(), remove() is disallowed until next() is called"); + iterator.remove(); + } + + @Override + public PeekingReiterator copy() { + return new PeekingReiterator(this); + } + + /** + * Returns the element that would be returned by {@link #next}, without + * actually consuming the element. + * @throws NoSuchElementException if there is no next element + */ + public T peek() { + computeNext(); + if (!nextElementComputed) { + throw new NoSuchElementException(); + } + return nextElement; + } + + private void computeNext() { + if (nextElementComputed) { + return; + } + if (!iterator.hasNext()) { + return; + } + nextElement = iterator.next(); + nextElementComputed = true; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Reiterable.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Reiterable.java new file mode 100644 index 0000000000000..ebf30459e2778 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Reiterable.java @@ -0,0 +1,27 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +/** + * An {@link Iterable} that returns {@link Reiterator} iterators. + * + * @param the type of elements returned by the iterator + */ +public interface Reiterable extends Iterable { + @Override + public Reiterator iterator(); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Reiterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Reiterator.java new file mode 100644 index 0000000000000..7613a3a37bd37 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Reiterator.java @@ -0,0 +1,39 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import java.util.Iterator; + +/** + * An {@link Iterator} with the ability to copy its iteration state. + * + * @param the type of elements returned by this iterator + */ +public interface Reiterator extends Iterator { + /** + * Returns a copy of the current {@link Reiterator}. The copy's iteration + * state is logically independent of the current iterator; each may be + * advanced without affecting the other. + * + *

The returned {@code Reiterator} is not guaranteed to return + * referentially identical iteration results as the original + * {@link Reiterator}, although {@link Object#equals} will typically return + * true for the corresponding elements of each if the original source is + * logically immutable. + */ + public Reiterator copy(); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/package-info.java new file mode 100644 index 0000000000000..0dd2af486ba0f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** Defines utilities shared by multiple PipelineRunner implementations. **/ +package com.google.cloud.dataflow.sdk.util.common; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/BatchingShuffleEntryReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/BatchingShuffleEntryReader.java new file mode 100644 index 0000000000000..2a596c0d86f86 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/BatchingShuffleEntryReader.java @@ -0,0 +1,148 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import com.google.cloud.dataflow.sdk.util.common.Reiterator; + +import java.util.ListIterator; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; + +/** + * BatchingShuffleEntryReader provides a mechanism for reading entries from + * a shuffle dataset. + */ +@NotThreadSafe +public final class BatchingShuffleEntryReader implements ShuffleEntryReader { + private final ShuffleBatchReader batchReader; + + /** + * Constructs a {@link BatchingShuffleEntryReader} + * + * @param batchReader supplies the underlying + * {@link ShuffleBatchReader} to read batches of entries from + */ + public BatchingShuffleEntryReader( + ShuffleBatchReader batchReader) { + this.batchReader = checkNotNull(batchReader); + } + + @Override + public Reiterator read( + @Nullable ShufflePosition startPosition, + @Nullable ShufflePosition endPosition) { + return new ShuffleReadIterator(startPosition, endPosition); + } + + /** + * ShuffleReadIterator iterates over a (potentially huge) sequence of shuffle + * entries. + */ + private final class ShuffleReadIterator implements Reiterator { + // Shuffle service returns entries in pages. If the response contains a + // non-null nextStartPosition, we have to ask for more pages. The response + // with null nextStartPosition signifies the end of stream. + @Nullable private final ShufflePosition endPosition; + @Nullable private ShufflePosition nextStartPosition; + + /** The most recently read batch. */ + @Nullable ShuffleBatchReader.Batch currentBatch; + /** An iterator over the most recently read batch. */ + @Nullable private ListIterator entries; + + ShuffleReadIterator(@Nullable ShufflePosition startPosition, + @Nullable ShufflePosition endPosition) { + this.nextStartPosition = startPosition; + this.endPosition = endPosition; + } + + private ShuffleReadIterator(ShuffleReadIterator it) { + this.endPosition = it.endPosition; + this.nextStartPosition = it.nextStartPosition; + this.currentBatch = it.currentBatch; + // The idea here: if the iterator being copied was in the middle of a + // batch (the typical case), create a new iteration state at the same + // point in the same batch. + this.entries = (it.entries == null + ? null + : it.currentBatch.entries.listIterator(it.entries.nextIndex())); + } + + @Override + public boolean hasNext() { + fillEntriesIfNeeded(); + // TODO: Report API errors to the caller using checked + // exceptions. + return entries.hasNext(); + } + + @Override + public ShuffleEntry next() throws NoSuchElementException { + fillEntriesIfNeeded(); + ShuffleEntry entry = entries.next(); + return entry; + } + + @Override + public void remove() throws UnsupportedOperationException { + throw new UnsupportedOperationException(); + } + + @Override + public ShuffleReadIterator copy() { + return new ShuffleReadIterator(this); + } + + private void fillEntriesIfNeeded() { + if (entries != null && entries.hasNext()) { + // Has more records in the current page, or error. + return; + } + + if (entries != null && nextStartPosition == null) { + // End of stream. + checkState(!entries.hasNext()); + return; + } + + do { + fillEntries(); + } while (!entries.hasNext() && nextStartPosition != null); + } + + private void fillEntries() { + try { + ShuffleBatchReader.Batch batch = + batchReader.read(nextStartPosition, endPosition); + nextStartPosition = batch.nextStartPosition; + entries = batch.entries.listIterator(); + currentBatch = batch; + } catch (RuntimeException e) { + throw e; + } catch (Throwable t) { + throw new RuntimeException(t); + } + + checkState(entries != null); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CachingShuffleBatchReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CachingShuffleBatchReader.java new file mode 100644 index 0000000000000..87abf21d4229b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CachingShuffleBatchReader.java @@ -0,0 +1,228 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.base.Objects; +import com.google.common.base.Throwables; + +import java.io.IOException; +import java.lang.ref.Reference; +import java.lang.ref.ReferenceQueue; +import java.lang.ref.SoftReference; +import java.util.HashMap; + +import javax.annotation.Nullable; + +/** A {@link ShuffleBatchReader} that caches batches as they're read. */ +public final class CachingShuffleBatchReader implements ShuffleBatchReader { + private final ShuffleBatchReader reader; + + // The cache itself is implemented as a HashMap of RangeReadReference values, + // keyed by the start and end positions describing the range of a particular + // request (represented by BatchRange). + // + // The first reader for a particular range builds an AsyncReadResult for the + // result, inserts it into the cache, drops the lock, and then completes the + // read; subsequent readers simply wait for the AsyncReadResult to complete. + // + // Note that overlapping ranges are considered distinct; cached entries for + // one range are not used for any other range, even if doing so would avoid a + // fetch. + // + // So this is not a particularly sophisticated algorithm: a smarter cache + // would be able to use subranges of previous requests to satisfy new + // requests. But in this particular case, we expect that the simple algorithm + // will work well. For a given shuffle source, the splits read by various + // iterators over that source starting from a particular position (which is + // how this class is used in practice) should turn out to be constant, if the + // result returned by the service for a particular [start, end) range are + // consistent. So we're not expecting to see overlapping ranges of entries + // within a cache. + // + // It's also been shown -- by implementing it -- that the more thorough + // algorithm is relatively complex, with numerous edge cases requiring very + // careful thought to get right. It's doable, but non-trivial and hard to + // understand and maintain; without a compelling justification, it's better to + // stick with the simpler implementation. + // + // @VisibleForTesting + final HashMap cache = new HashMap<>(); + + // The queue of references which have been collected by the garbage collector. + // This queue should only be used with references of class RangeReadReference. + private final ReferenceQueue refQueue = new ReferenceQueue<>(); + + /** + * Constructs a new {@link CachingShuffleBatchReader}. + * + * @param reader supplies the downstream {@link ShuffleBatchReader} + * this {@code CachingShuffleBatchReader} will use to issue reads + */ + public CachingShuffleBatchReader(ShuffleBatchReader reader) { + this.reader = checkNotNull(reader); + } + + @Override + public Batch read( + @Nullable ShufflePosition startPosition, + @Nullable ShufflePosition endPosition) throws IOException { + + @Nullable AsyncReadResult waitResult = null; + @Nullable AsyncReadResult runResult = null; + final BatchRange batchRange = new BatchRange(startPosition, endPosition); + + synchronized (cache) { + // Remove any GCd entries. + for (Reference ref = refQueue.poll(); + ref != null; + ref = refQueue.poll()) { + RangeReadReference rangeReadRef = (RangeReadReference) ref; + cache.remove(rangeReadRef.getBatchRange()); + } + + // Find the range reference; note that one might not be in the map, or it + // might contain a null if its target has been GCd. + @Nullable RangeReadReference rangeReadRef = cache.get(batchRange); + + // Get a strong reference to the existing AsyncReadResult for the range, if possible. + if (rangeReadRef != null) { + waitResult = rangeReadRef.get(); + } + + // Create a new AsyncReadResult if one is needed. + if (waitResult == null) { + runResult = new AsyncReadResult(); + waitResult = runResult; + rangeReadRef = null; // Replace the previous RangeReadReference. + } + + // Insert a new RangeReadReference into the map if we don't have a usable + // one (either we weren't able to find one in the map, or we did but it + // was already cleared by the GC). + if (rangeReadRef == null) { + cache.put(batchRange, + new RangeReadReference(batchRange, runResult, refQueue)); + } + } // Drop the cache lock. + + if (runResult != null) { + // This thread created the AsyncReadResult, and is responsible for + // actually performing the read. + try { + Batch result = reader.read(startPosition, endPosition); + runResult.setResult(result); + } catch (RuntimeException | IOException e) { + runResult.setException(e); + synchronized (cache) { + // No reason to continue to cache the fact that there was a problem. + // Note that since this thread holds a strong reference to the + // AsyncReadResult, it won't be GCd, so the soft reference held by the + // cache is guaranteed to still be present. + cache.remove(batchRange); + } + } + } + + return waitResult.getResult(); + } + + /** The key for the entries stored in the batch cache. */ + // @VisibleForTesting + static final class BatchRange { + @Nullable private final ShufflePosition startPosition; + @Nullable private final ShufflePosition endPosition; + + public BatchRange(@Nullable ShufflePosition startPosition, + @Nullable ShufflePosition endPosition) { + this.startPosition = startPosition; + this.endPosition = endPosition; + } + + @Override + public boolean equals(Object o) { + return o == this + || (o instanceof BatchRange + && Objects.equal(((BatchRange) o).startPosition, startPosition) + && Objects.equal(((BatchRange) o).endPosition, endPosition)); + } + + @Override + public int hashCode() { + return Objects.hashCode(startPosition, endPosition); + } + } + + /** Holds an asynchronously batch read result. */ + private static final class AsyncReadResult { + @Nullable private Batch batch = null; + @Nullable private Throwable thrown = null; + + public synchronized void setResult(Batch b) { + batch = b; + notifyAll(); + } + + public synchronized void setException(Throwable t) { + thrown = t; + notifyAll(); + } + + public synchronized Batch getResult() throws IOException { + while (batch == null && thrown == null) { + try { + wait(); + } catch (InterruptedException e) { + throw new RuntimeException("interrupted", e); + } + } + if (thrown != null) { + // N.B. setException can only be called with a RuntimeException or an + // IOException, so propagateIfPossible should always do the throw. + Throwables.propagateIfPossible(thrown, IOException.class); + throw new RuntimeException("unexpected", thrown); + } + return batch; + } + } + + /** + * Maintains a soft reference to an AsyncReadResult. + * + *

This class extends {@link SoftReference} so that when the garbage + * collector collects a batch and adds its reference to the cache's reference + * queue, that reference can be cast back to {@code RangeReadReference}, + * allowing us to identify the reference's position in the cache (and to + * therefore remove it). + */ + // @VisibleForTesting + static final class RangeReadReference extends SoftReference { + private final BatchRange range; + + public RangeReadReference( + BatchRange range, AsyncReadResult result, + ReferenceQueue refQueue) { + super(result, refQueue); + this.range = checkNotNull(range); + } + + public BatchRange getBatchRange() { + return range; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CustomSourceFormat.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CustomSourceFormat.java new file mode 100644 index 0000000000000..4fc67d60f3c3f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CustomSourceFormat.java @@ -0,0 +1,61 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +/** + * An interface for sources which can perform operations on source specifications, such as + * splitting the source and computing its metadata. See {@code SourceOperationRequest} for details. + */ +public interface CustomSourceFormat { + /** + * Performs an operation on the specification of a source. + * See {@code SourceOperationRequest} for details. + */ + public SourceOperationResponse performSourceOperation(SourceOperationRequest operation) + throws Exception; + + /** + * A representation of an operation on the specification of a source, + * e.g. splitting a source into shards, getting the metadata of a source, + * etc. + * + *

The common worker framework does not interpret instances of + * this interface. But a tool-specific framework can make assumptions + * about the implementation, and so the concrete Source subclasses used + * by a tool-specific framework should match. + */ + public interface SourceOperationRequest { + } + + /** + * A representation of the result of a SourceOperationRequest. + * + *

See the comment on {@link SourceOperationRequest} for how instances of this + * interface are used by the rest of the framework. + */ + public interface SourceOperationResponse { + } + + /** + * A representation of a specification of a source. + * + *

See the comment on {@link SourceOperationRequest} for how instances of this + * interface are used by the rest of the framework. + */ + public interface SourceSpec { + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperation.java new file mode 100644 index 0000000000000..6325d1ac5cdb8 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperation.java @@ -0,0 +1,54 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +/** + * A flatten operation. + */ +public class FlattenOperation extends ReceivingOperation { + public FlattenOperation(String operationName, + OutputReceiver[] receivers, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) { + super(operationName, receivers, + counterPrefix, addCounterMutator, stateSampler); + } + + /** Invoked by tests. */ + public FlattenOperation(OutputReceiver outputReceiver, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) { + this("FlattenOperation", new OutputReceiver[]{ outputReceiver }, + counterPrefix, addCounterMutator, stateSampler); + } + + @Override + public void process(Object elem) throws Exception { + try (StateSampler.ScopedState process = + stateSampler.scopedState(processState)) { + checkStarted(); + Receiver receiver = receivers[0]; + if (receiver != null) { + receiver.process(elem); + } + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/GroupingShuffleEntryIterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/GroupingShuffleEntryIterator.java new file mode 100644 index 0000000000000..19428201f0395 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/GroupingShuffleEntryIterator.java @@ -0,0 +1,216 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObservableIterable; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObservableIterator; +import com.google.cloud.dataflow.sdk.util.common.PeekingReiterator; +import com.google.cloud.dataflow.sdk.util.common.Reiterable; +import com.google.cloud.dataflow.sdk.util.common.Reiterator; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +/** + * An iterator through KeyGroupedShuffleEntries. + */ +public abstract class GroupingShuffleEntryIterator + implements Iterator { + /** The iterator through the underlying shuffle records. */ + private PeekingReiterator shuffleIterator; + + /** + * The key of the most recent KeyGroupedShuffleEntries returned by + * {@link #next}, if any. + * + *

If currentKeyBytes is non-null, then it's the key for the last entry + * returned by {@link #next}, and all incoming entries with that key should + * be skipped over by this iterator (since this iterator is iterating over + * keys, not the individual values associated with a given key). + * + *

If currentKeyBytes is null, and shuffleIterator.hasNext(), then the + * key of shuffleIterator.next() is the key of the next + * KeyGroupedShuffleEntries to return from {@link #next}. + */ + @Nullable private byte[] currentKeyBytes = null; + + /** + * Constructs a GroupingShuffleEntryIterator, given a Reiterator + * over ungrouped ShuffleEntries, assuming the ungrouped + * ShuffleEntries for a given key are consecutive. + */ + public GroupingShuffleEntryIterator( + Reiterator shuffleIterator) { + this.shuffleIterator = + new PeekingReiterator( + new ProgressTrackingReiterator<>( + shuffleIterator, + new ProgressTrackerGroup() { + @Override + protected void report(ShuffleEntry entry) { + notifyElementRead(entry.length()); + } + }.start())); + } + + /** Notifies observers about a new element read. */ + protected abstract void notifyElementRead(long byteSize); + + @Override + public boolean hasNext() { + advanceIteratorToNextKey(); + return shuffleIterator.hasNext(); + } + + @Override + public KeyGroupedShuffleEntries next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + ShuffleEntry entry = shuffleIterator.peek(); + currentKeyBytes = entry.getKey(); + return new KeyGroupedShuffleEntries( + entry.getPosition(), + currentKeyBytes, + new ValuesIterable(new ValuesIterator(currentKeyBytes))); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + private void advanceIteratorToNextKey() { + if (currentKeyBytes == null) { + return; + } + while (shuffleIterator.hasNext()) { + ShuffleEntry entry = shuffleIterator.peek(); + if (!Arrays.equals(entry.getKey(), currentKeyBytes)) { + break; + } + shuffleIterator.next(); + } + currentKeyBytes = null; + } + + private static class ValuesIterable + extends ElementByteSizeObservableIterable + implements Reiterable { + private final ValuesIterator base; + + public ValuesIterable(ValuesIterator base) { + this.base = checkNotNull(base); + } + + @Override + public ValuesIterator createIterator() { + return base.copy(); + } + } + + /** + * Provides the {@link Reiterator} used to iterate through the + * shuffle entries of a KeyGroupedShuffleEntries. + */ + private class ValuesIterator + extends ElementByteSizeObservableIterator + implements Reiterator { + // N.B. This class is *not* static; it maintains a reference to its + // enclosing KeyGroupedShuffleEntriesIterator instance so that it can update + // that instance's shuffleIterator as an optimization. + + private final byte[] valueKeyBytes; + private final PeekingReiterator valueShuffleIterator; + private final ProgressTracker tracker; + private boolean nextKnownValid = false; + + public ValuesIterator(byte[] valueKeyBytes) { + this.valueKeyBytes = checkNotNull(valueKeyBytes); + this.valueShuffleIterator = shuffleIterator.copy(); + // N.B. The ProgressTrackerGroup captures the reference to the original + // ValuesIterator for a given values iteration. Which happens to be + // exactly what we want, since this is also the ValuesIterator whose + // base Observable has the references to all of the Observers watching + // the iteration. Copied ValuesIterator instances do *not* have these + // Observers, but that's fine, since the derived ProgressTracker + // instances reference the ProgressTrackerGroup which references the + // original ValuesIterator which does have them. + this.tracker = new ProgressTrackerGroup() { + @Override + protected void report(ShuffleEntry entry) { + notifyValueReturned(entry.length()); + } + }.start(); + } + + private ValuesIterator(ValuesIterator it) { + this.valueKeyBytes = it.valueKeyBytes; + this.valueShuffleIterator = it.valueShuffleIterator.copy(); + this.tracker = it.tracker.copy(); + this.nextKnownValid = it.nextKnownValid; + } + + @Override + public boolean hasNext() { + if (nextKnownValid) { + return true; + } + if (!valueShuffleIterator.hasNext()) { + return false; + } + ShuffleEntry entry = valueShuffleIterator.peek(); + nextKnownValid = Arrays.equals(entry.getKey(), valueKeyBytes); + + // Opportunistically update the parent KeyGroupedShuffleEntriesIterator, + // potentially allowing it to skip a large number of key/value pairs + // with this key. + if (!nextKnownValid && valueKeyBytes == currentKeyBytes) { + shuffleIterator = valueShuffleIterator.copy(); + currentKeyBytes = null; + } + + return nextKnownValid; + } + + @Override + public ShuffleEntry next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + ShuffleEntry entry = valueShuffleIterator.next(); + nextKnownValid = false; + tracker.saw(entry); + return entry; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public ValuesIterator copy() { + return new ValuesIterator(this); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/KeyGroupedShuffleEntries.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/KeyGroupedShuffleEntries.java new file mode 100644 index 0000000000000..1b8b552b521e8 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/KeyGroupedShuffleEntries.java @@ -0,0 +1,35 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import com.google.cloud.dataflow.sdk.util.common.Reiterable; + +/** + * A collection of ShuffleEntries, all with the same key. + */ +public class KeyGroupedShuffleEntries { + public final byte[] position; + public final byte[] key; + public final Reiterable values; + + public KeyGroupedShuffleEntries(byte[] position, byte[] key, + Reiterable values) { + this.position = position; + this.key = key; + this.values = values; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java new file mode 100644 index 0000000000000..45d5e8c6715e3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java @@ -0,0 +1,116 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.ListIterator; + +/** + * An executor for a map task, defined by a list of Operations. + */ +public class MapTaskExecutor extends WorkExecutor { + private static final Logger LOG = + LoggerFactory.getLogger(MapTaskExecutor.class); + + /** The operations in the map task, in execution order. */ + public final List operations; + + /** The StateSampler for tracking where time is being spent, or null. */ + protected final StateSampler stateSampler; + + /** + * Creates a new MapTaskExecutor. + * + * @param operations the operations of the map task, in order of execution + * @param counters a set of system counters associated with + * operations, which may get extended during execution + * @param stateSampler a state sampler for tracking where time is being spent + */ + public MapTaskExecutor(List operations, + CounterSet counters, + StateSampler stateSampler) { + super(counters); + this.operations = operations; + this.stateSampler = stateSampler; + } + + @Override + public void execute() throws Exception { + LOG.debug("executing map task"); + + // Start operations, in reverse-execution-order, so that a + // consumer is started before a producer might output to it. + // Starting a root operation such as a ReadOperation does the work + // of processing the input dataset. + LOG.debug("starting operations"); + ListIterator iterator = + operations.listIterator(operations.size()); + while (iterator.hasPrevious()) { + Operation op = iterator.previous(); + op.start(); + } + + // Finish operations, in forward-execution-order, so that a + // producer finishes outputting to its consumers before those + // consumers are themselves finished. + LOG.debug("finishing operations"); + for (Operation op : operations) { + op.finish(); + } + + LOG.debug("map task execution complete"); + + // TODO: support for success / failure ports? + } + + @Override + public Source.Progress getWorkerProgress() throws Exception { + return getReadOperation().getProgress(); + } + + @Override + public Source.Position proposeStopPosition( + Source.Progress proposedStopPosition) throws Exception { + return getReadOperation().proposeStopPosition(proposedStopPosition); + } + + ReadOperation getReadOperation() throws Exception { + if (operations == null || operations.isEmpty()) { + throw new IllegalStateException( + "Map task has no operation."); + } + + Operation readOperation = operations.get(0); + if (!(readOperation instanceof ReadOperation)) { + throw new IllegalStateException( + "First operation in the map task is not a ReadOperation."); + } + + return (ReadOperation) readOperation; + } + + @Override + public void close() throws Exception { + stateSampler.close(); + super.close(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Operation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Operation.java new file mode 100644 index 0000000000000..bedc081cec99d --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Operation.java @@ -0,0 +1,132 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +/** + * The abstract base class for Operations, which correspond to + * Instructions in the original MapTask InstructionGraph. + * + * Call start() to start the operation. + * + * A read operation's start() method actually reads the data, and in + * effect runs the pipeline. + * + * Call finish() to finish the operation. + * + * Since both start() and finish() may call process() on + * this operation's consumers, start an operation after + * starting its consumers, and finish an operation before + * finishing its consumers. + */ +public abstract class Operation { + /** + * The array of consuming receivers, one per operation output + * "port" (e.g., DoFn main or side output). A receiver might be + * null if that output isn't being consumed. + */ + public final OutputReceiver[] receivers; + + /** + * The possible initialization states of an Operation. + * For internal self-checking purposes. + */ + public enum InitializationState { + // start() hasn't yet been called. + UNSTARTED, + + // start() has been called, but finish() hasn't yet been called. + STARTED, + + // finish() has been called. + FINISHED + } + + /** The initialization state of this Operation. */ + public InitializationState initializationState = + InitializationState.UNSTARTED; + + protected final StateSampler stateSampler; + + protected final int startState; + protected final int processState; + protected final int finishState; + + public Operation(String operationName, + OutputReceiver[] receivers, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) { + this.receivers = receivers; + this.stateSampler = stateSampler; + startState = stateSampler.stateForName(operationName + "-start"); + processState = stateSampler.stateForName(operationName + "-process"); + finishState = stateSampler.stateForName(operationName + "-finish"); + } + + /** + * Checks that this oepration is not yet started, throwing an + * exception otherwise. + */ + void checkUnstarted() { + if (initializationState != InitializationState.UNSTARTED) { + throw new AssertionError( + "expecting this instruction to not yet be started"); + } + } + + /** + * Checks that this oepration has been started but not yet finished, + * throwing an exception otherwise. + */ + void checkStarted() { + if (initializationState != InitializationState.STARTED) { + throw new AssertionError( + "expecting this instruction to be started"); + } + } + + /** + * Checks that this oepration has been finished, throwing an + * exception otherwise. + */ + void checkFinished() { + if (initializationState != InitializationState.FINISHED) { + throw new AssertionError( + "expecting this instruction to be finished"); + } + } + + /** + * Starts this Operation's execution. Called after all successsor + * consuming operations have been started. + */ + public void start() throws Exception { + checkUnstarted(); + initializationState = InitializationState.STARTED; + } + + /** + * Finishes this Operation's execution. Called after all + * predecessor producing operations have been finished. + */ + public void finish() throws Exception { + checkStarted(); + initializationState = InitializationState.FINISHED; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiver.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiver.java new file mode 100644 index 0000000000000..a13b74afbf8b3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiver.java @@ -0,0 +1,207 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; + +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObservable; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +/** + * Receiver that forwards each input it receives to each of a list of + * output Receivers. Additionally, it tracks output counters, that is, size + * information for elements passing through. + */ +public class OutputReceiver implements Receiver { + private final String outputName; + // Might be null, e.g., undeclared outputs will not have an + // elementByteSizeObservable. + private final ElementByteSizeObservable elementByteSizeObservable; + private final Counter elementCount; + private Counter byteCount = null; + private Counter meanByteCount = null; + private ElementByteSizeObserver byteCountObserver = null; + private ElementByteSizeObserver meanByteCountObserver = null; + private final List outputs = new ArrayList<>(); + private final Random randomGenerator = new Random(); + private int samplingToken = 0; + private final int samplingTokenUpperBound = 1000000; // Lowest sampling probability: 0.001%. + private final int samplingCutoff = 10; + + public OutputReceiver(String outputName, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator) { + this(outputName, (ElementByteSizeObservable) null, + counterPrefix, addCounterMutator); + } + + public OutputReceiver(String outputName, + ElementByteSizeObservable elementByteSizeObservable, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator) { + this.outputName = outputName; + this.elementByteSizeObservable = elementByteSizeObservable; + + elementCount = addCounterMutator.addCounter( + Counter.longs(elementsCounterName(counterPrefix, outputName), SUM)); + + if (elementByteSizeObservable != null) { + String bytesCounterName = bytesCounterName(counterPrefix, outputName); + if (bytesCounterName != null) { + byteCount = addCounterMutator.addCounter( + Counter.longs(bytesCounterName, SUM)); + byteCountObserver = new ElementByteSizeObserver(byteCount); + } + String meanBytesCounterName = + meanBytesCounterName(counterPrefix, outputName); + if (meanBytesCounterName != null) { + meanByteCount = addCounterMutator.addCounter( + Counter.longs(meanBytesCounterName, MEAN)); + meanByteCountObserver = new ElementByteSizeObserver(meanByteCount); + } + } + } + + protected String elementsCounterName(String counterPrefix, + String outputName) { + return outputName + "-ElementCount"; + } + protected String bytesCounterName(String counterPrefix, + String outputName) { + return null; + } + protected String meanBytesCounterName(String counterPrefix, + String outputName) { + return outputName + "-MeanByteCount"; + } + + /** + * Adds a new receiver that this OutputReceiver forwards to. + */ + public void addOutput(Receiver receiver) { + outputs.add(receiver); + } + + @Override + public void process(Object elem) throws Exception { + // Increment element counter. + elementCount.addValue(1L); + + // Increment byte counter. + boolean advanceByteCountObserver = false; + boolean advanceMeanByteCountObserver = false; + if ((byteCountObserver != null || meanByteCountObserver != null) + && (sampleElement() + || elementByteSizeObservable.isRegisterByteSizeObserverCheap( + elem))) { + + if (byteCountObserver != null) { + elementByteSizeObservable.registerByteSizeObserver( + elem, byteCountObserver); + } + if (meanByteCountObserver != null) { + elementByteSizeObservable.registerByteSizeObserver( + elem, meanByteCountObserver); + } + + if (byteCountObserver != null) { + if (!byteCountObserver.getIsLazy()) { + byteCountObserver.advance(); + } else { + advanceByteCountObserver = true; + } + } + if (meanByteCountObserver != null) { + if (!meanByteCountObserver.getIsLazy()) { + meanByteCountObserver.advance(); + } else { + advanceMeanByteCountObserver = true; + } + } + } + + // Fan-out. + for (Receiver out : outputs) { + if (out != null) { + out.process(elem); + } + } + + // Advance lazy ElementByteSizeObservers, if any. + // Note that user's code is allowed to store the element of one + // DoFn.processElement() call and access it later on. We are still + // calling next() here, causing an update to byteCount. If user's + // code really accesses more element's pieces later on, their byte + // count would accrue against a future element. This is not ideal, + // but still approximately correct. + if (advanceByteCountObserver) { + byteCountObserver.advance(); + } + if (advanceMeanByteCountObserver) { + meanByteCountObserver.advance(); + } + } + + public String getName() { + return outputName; + } + + public Counter getElementCount() { + return elementCount; + } + + public Counter getByteCount() { + return byteCount; + } + + public Counter getMeanByteCount() { + return meanByteCount; + } + + protected boolean sampleElement() { + // Sampling probability decreases as the element count is increasing. + // We unconditionally sample the first samplingCutoff elements. For the + // next samplingCutoff elements, the sampling probability drops from 100% + // to 50%. The probability of sampling the Nth element is: + // min(1, samplingCutoff / N), with an additional lower bound of + // samplingCutoff / samplingTokenUpperBound. This algorithm may be refined + // later. + samplingToken = Math.min(samplingToken + 1, samplingTokenUpperBound); + return randomGenerator.nextInt(samplingToken) < samplingCutoff; + } + + /** Invoked by tests only. */ + public int getReceiverCount() { + return outputs.size(); + } + + /** Invoked by tests only. */ + public Receiver getOnlyReceiver() { + if (outputs.size() != 1) { + throw new AssertionError("only one receiver expected"); + } + + return outputs.get(0); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoFn.java new file mode 100644 index 0000000000000..b922acc412d4c --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoFn.java @@ -0,0 +1,28 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +/** + * Abstract base class for ParDoFns, invocable by ParDoOperations. + */ +public abstract class ParDoFn { + public abstract void startBundle(Receiver... receivers) throws Exception; + + public abstract void processElement(Object elem) throws Exception; + + public abstract void finishBundle() throws Exception; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperation.java new file mode 100644 index 0000000000000..7a620983476f0 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperation.java @@ -0,0 +1,65 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +/** + * A ParDo mapping function. + */ +public class ParDoOperation extends ReceivingOperation { + public final ParDoFn fn; + + public ParDoOperation(String operationName, + ParDoFn fn, + OutputReceiver[] outputReceivers, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) { + super(operationName, outputReceivers, + counterPrefix, addCounterMutator, stateSampler); + this.fn = fn; + } + + @Override + public void start() throws Exception { + try (StateSampler.ScopedState start = + stateSampler.scopedState(startState)) { + super.start(); + fn.startBundle(receivers); + } + } + + @Override + public void process(Object elem) throws Exception { + try (StateSampler.ScopedState process = + stateSampler.scopedState(processState)) { + checkStarted(); + fn.processElement(elem); + } + } + + @Override + public void finish() throws Exception { + try (StateSampler.ScopedState finish = + stateSampler.scopedState(finishState)) { + checkStarted(); + fn.finishBundle(); + super.finish(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperation.java new file mode 100644 index 0000000000000..a4afa5b2820d2 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperation.java @@ -0,0 +1,521 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; + +/** + * A partial group-by-key operation. + */ +public class PartialGroupByKeyOperation extends ReceivingOperation { + /** + * Provides client-specific operations for grouping keys. + */ + public static interface GroupingKeyCreator { + public Object createGroupingKey(K key) throws Exception; + } + + /** + * Provides client-specific operations for size estimates. + */ + public static interface SizeEstimator { + public long estimateSize(E element) throws Exception; + } + + /** + * Provides client-specific operations for working with elements + * that are key/value or key/values pairs. + */ + public interface PairInfo { + public Object getKeyFromInputPair(Object pair); + public Object getValueFromInputPair(Object pair); + public Object makeOutputPair(Object key, Object value); + } + + /** + * Provides client-specific operations for combining values. + */ + public interface Combiner { + public VA createAccumulator(K key); + public VA add(K key, VA accumulator, VI value); + public VA merge(K key, Iterable accumulators); + public VO extract(K key, VA accumulator); + } + + /** + * A wrapper around a byte[] that uses structural, value-based + * equality rather than byte[]'s normal object identity. + */ + public static class StructuralByteArray { + byte[] value; + + public StructuralByteArray(byte[] value) { + this.value = value; + } + + public byte[] getValue() { return value; } + + @Override + public boolean equals(Object o) { + if (o instanceof StructuralByteArray) { + StructuralByteArray that = (StructuralByteArray) o; + return Arrays.equals(this.value, that.value); + } else { + return false; + } + } + + @Override + public int hashCode() { + return Arrays.hashCode(value); + } + + @Override + public String toString() { + return "Val" + Arrays.toString(value); + } + } + + // By default, how many bytes we allow the grouping table to consume before + // it has to be flushed. + static final long DEFAULT_MAX_GROUPING_TABLE_BYTES = 100_000_000L; + + // How many bytes a word in the JVM has. + static final int BYTES_PER_JVM_WORD = getBytesPerJvmWord(); + + /** + * The number of bytes of overhead to store an entry in the + * grouping table (a {@code HashMap}), + * ignoring the actual number of bytes in the keys and values: + * + * - an array element (1 word), + * - a HashMap.Entry (4 words), + * - a StructuralByteArray (1 words), + * - a backing array (guessed at 1 word for the length), + * - a KeyAndValues (2 words), + * - an ArrayList (2 words), + * - a backing array (1 word), + * - per-object overhead (JVM-specific, guessed at 2 words * 6 objects). + */ + static final int PER_KEY_OVERHEAD = 24 * BYTES_PER_JVM_WORD; + + final GroupingTable groupingTable; + + @SuppressWarnings("unchecked") + public PartialGroupByKeyOperation( + String operationName, + GroupingKeyCreator groupingKeyCreator, + SizeEstimator keySizeEstimator, SizeEstimator valueSizeEstimator, + PairInfo pairInfo, + OutputReceiver[] receivers, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) { + super(operationName, receivers, counterPrefix, addCounterMutator, stateSampler); + groupingTable = new BufferingGroupingTable( + DEFAULT_MAX_GROUPING_TABLE_BYTES, groupingKeyCreator, + pairInfo, keySizeEstimator, valueSizeEstimator); + } + + @SuppressWarnings("unchecked") + public PartialGroupByKeyOperation( + String operationName, + GroupingKeyCreator groupingKeyCreator, + SizeEstimator keySizeEstimator, SizeEstimator valueSizeEstimator, + double sizeEstimatorSampleRate, + PairInfo pairInfo, + OutputReceiver[] receivers, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) { + this(operationName, groupingKeyCreator, + new SamplingSizeEstimator(keySizeEstimator, sizeEstimatorSampleRate, 1.0), + new SamplingSizeEstimator(valueSizeEstimator, sizeEstimatorSampleRate, 1.0), + pairInfo, receivers, counterPrefix, addCounterMutator, stateSampler); + } + + /** Invoked by tests. */ + public PartialGroupByKeyOperation( + GroupingKeyCreator groupingKeyCreator, + SizeEstimator keySizeEstimator, SizeEstimator valueSizeEstimator, + PairInfo pairInfo, + OutputReceiver outputReceiver, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) { + this("PartialGroupByKeyOperation", groupingKeyCreator, + keySizeEstimator, valueSizeEstimator, pairInfo, + new OutputReceiver[]{ outputReceiver }, + counterPrefix, + addCounterMutator, + stateSampler); + } + + @Override + public void process(Object elem) throws Exception { + try (StateSampler.ScopedState process = + stateSampler.scopedState(processState)) { + if (receivers[0] != null) { + groupingTable.put(elem, receivers[0]); + } + } + } + + @Override + public void finish() throws Exception { + try (StateSampler.ScopedState finish = + stateSampler.scopedState(finishState)) { + checkStarted(); + if (receivers[0] != null) { + groupingTable.flush(receivers[0]); + } + super.finish(); + } + } + + /** + * Sets the maximum amount of memory the grouping table is allowed to + * consume before it has to be flushed. + */ + // @VisibleForTesting + public void setMaxGroupingTableBytes(long maxSize) { + groupingTable.maxSize = maxSize; + } + + /** + * Returns the amount of memory the grouping table currently consumes. + */ + // @VisibleForTesting + public long getGroupingTableBytes() { + return groupingTable.size; + } + + /** + * Returns the number of bytes in a JVM word. In case we failed to + * find the answer, returns 8. + */ + static int getBytesPerJvmWord() { + String wordSizeInBits = System.getProperty("sun.arch.data.model"); + try { + return Integer.parseInt(wordSizeInBits) / 8; + } catch (NumberFormatException e) { + // The JVM word size is unknown. Assume 64-bit. + return 8; + } + } + + private abstract static class GroupingTable { + + // Keep the table relatively full to increase the chance of collisions. + private static final double TARGET_LOAD = 0.9; + + private long maxSize; + private final GroupingKeyCreator groupingKeyCreator; + private final PairInfo pairInfo; + + private long size = 0; + private Map> table; + + public GroupingTable(long maxSize, + GroupingKeyCreator groupingKeyCreator, + PairInfo pairInfo) { + this.maxSize = maxSize; + this.groupingKeyCreator = groupingKeyCreator; + this.pairInfo = pairInfo; + this.table = new HashMap<>(); + } + + interface GroupingTableEntry { + public K getKey(); + public VA getValue(); + public void add(VI value) throws Exception; + public long getSize(); + } + + public abstract GroupingTableEntry createTableEntry(K key) throws Exception; + + /** + * Adds a pair to this table, possibly flushing some entries to output + * if the table is full. + */ + @SuppressWarnings("unchecked") + public void put(Object pair, Receiver receiver) throws Exception { + put((K) pairInfo.getKeyFromInputPair(pair), + (VI) pairInfo.getValueFromInputPair(pair), + receiver); + } + + /** + * Adds the key and value to this table, possibly flushing some entries + * to output if the table is full. + */ + public void put(K key, VI value, Receiver receiver) throws Exception { + Object groupingKey = groupingKeyCreator.createGroupingKey(key); + GroupingTableEntry entry = table.get(groupingKey); + if (entry == null) { + entry = createTableEntry(key); + table.put(groupingKey, entry); + size += PER_KEY_OVERHEAD; + } else { + size -= entry.getSize(); + } + entry.add(value); + size += entry.getSize(); + + if (size >= maxSize) { + long targetSize = (long) (TARGET_LOAD * maxSize); + Iterator> entries = + table.values().iterator(); + while (size >= targetSize) { + if (!entries.hasNext()) { + // Should never happen, but sizes may be estimates... + size = 0; + break; + } + GroupingTableEntry toFlush = entries.next(); + entries.remove(); + size -= toFlush.getSize() + PER_KEY_OVERHEAD; + output(toFlush, receiver); + } + } + } + + /** + * Output the given entry. Does not actually remove it from the table or + * update this table's size. + */ + private void output(GroupingTableEntry entry, Receiver receiver) throws Exception { + receiver.process(pairInfo.makeOutputPair(entry.getKey(), entry.getValue())); + } + + /** + * Flushes all entries in this table to output. + */ + public void flush(Receiver output) throws Exception { + for (GroupingTableEntry entry : table.values()) { + output(entry, output); + } + table.clear(); + size = 0; + } + + } + + /** + * A grouping table that simply buffers all inserted values in a list. + */ + public static class BufferingGroupingTable extends GroupingTable> { + + public final SizeEstimator keySizer; + public final SizeEstimator valueSizer; + + public BufferingGroupingTable(long maxSize, + GroupingKeyCreator groupingKeyCreator, + PairInfo pairInfo, + SizeEstimator keySizer, + SizeEstimator valueSizer) { + super(maxSize, groupingKeyCreator, pairInfo); + this.keySizer = keySizer; + this.valueSizer = valueSizer; + } + + @Override + public GroupingTableEntry> createTableEntry(final K key) throws Exception { + return new GroupingTableEntry>() { + long size = keySizer.estimateSize(key); + final List values = new ArrayList<>(); + public K getKey() { return key; } + public List getValue() { return values; } + public long getSize() { return size; } + public void add(V value) throws Exception { + values.add(value); + size += BYTES_PER_JVM_WORD + valueSizer.estimateSize(value); + } + }; + } + } + + /** + * A grouping table that uses the given combiner to combine values in place. + */ + public static class CombiningGroupingTable extends GroupingTable { + + private final Combiner combiner; + private final SizeEstimator keySizer; + private final SizeEstimator valueSizer; + + public CombiningGroupingTable(long maxSize, + GroupingKeyCreator groupingKeyCreator, + PairInfo pairInfo, + Combiner combineFn, + SizeEstimator keySizer, + SizeEstimator valueSizer) { + super(maxSize, groupingKeyCreator, pairInfo); + this.combiner = combineFn; + this.keySizer = keySizer; + this.valueSizer = valueSizer; + } + + @Override + public GroupingTableEntry createTableEntry(final K key) throws Exception { + return new GroupingTableEntry() { + final long keySize = keySizer.estimateSize(key); + VA accumulator = combiner.createAccumulator(key); + long accumulatorSize = 0; // never used before a value is added... + public K getKey() { return key; } + public VA getValue() { return accumulator; } + public long getSize() { return keySize + accumulatorSize; } + public void add(VI value) throws Exception { + accumulator = combiner.add(key, accumulator, value); + accumulatorSize = valueSizer.estimateSize(accumulator); + } + }; + } + } + + + //////////////////////////////////////////////////////////////////////////// + // Size sampling. + + /** + * Implements size estimation by adaptively delegating to an underlying + * (potentially more expensive) estimator for some elements and returning + * the average value for others. + */ + public static class SamplingSizeEstimator implements SizeEstimator { + + /** + * The degree of confidence required in our expected value predictions + * before we allow under-sampling. + * + *

The value of 3.0 is a confidence interval of about 99.7% for a + * a high-degree-of-freedom t-distribution. + */ + public static final double CONFIDENCE_INTERVAL_SIGMA = 3; + + /** + * The desired size of our confidence interval (relative to the measured + * expected value). + * + *

The value of 0.25 is plus or minus 25%. + */ + public static final double CONFIDENCE_INTERVAL_SIZE = 0.25; + + /** + * Default number of elements that must be measured before elements are skipped. + */ + public static final long DEFAULT_MIN_SAMPLED = 20; + + private final SizeEstimator underlying; + private final double minSampleRate; + private final double maxSampleRate; + private final long minSampled; + private final Random random; + + private long totalElements = 0; + private long sampledElements = 0; + private long sampledSum = 0; + private double sampledSumSquares = 0; + private long estimate; + + private long nextSample = 0; + + public SamplingSizeEstimator( + SizeEstimator underlying, + double minSampleRate, + double maxSampleRate) { + this(underlying, minSampleRate, maxSampleRate, DEFAULT_MIN_SAMPLED, new Random()); + } + + public SamplingSizeEstimator(SizeEstimator underlying, + double minSampleRate, + double maxSampleRate, + long minSampled, + Random random) { + this.underlying = underlying; + this.minSampleRate = minSampleRate; + this.maxSampleRate = maxSampleRate; + this.minSampled = minSampled; + this.random = random; + } + + @Override + public long estimateSize(E element) throws Exception { + if (sampleNow()) { + return recordSample(underlying.estimateSize(element)); + } else { + return estimate; + } + } + + private boolean sampleNow() { + totalElements++; + return --nextSample < 0; + } + + private long recordSample(long value) { + sampledElements += 1; + sampledSum += value; + sampledSumSquares += value * value; + estimate = (long) Math.ceil(sampledSum / sampledElements); + long target = desiredSampleSize(); + if (sampledElements < minSampled || sampledElements < target) { + // Sample immediately. + nextSample = 0; + } else { + double rate = cap( + minSampleRate, + maxSampleRate, + Math.max(1.0 / (totalElements - minSampled + 1), // slowly ramp down + target / (double) totalElements)); // "future" target + // Uses the geometric distribution to return the likely distance between + // successive independent trials of a fixed probability p. This gives the + // same uniform distribution of branching on Math.random() < p, but with + // one random number generation per success rather than one per test, + // which can be a significant savings if p is small. + nextSample = rate == 1.0 + ? 0 + : (long) Math.floor(Math.log(random.nextDouble()) / Math.log(1 - rate)); + } + return value; + } + + private static final double cap(double min, double max, double value) { + return Math.min(max, Math.max(min, value)); + } + + private long desiredSampleSize() { + // We have no a-priori information on the actual distribution of data + // sizes, so compute our desired sample as if it were normal. + // Yes this formula is unstable for small stddev, but we only care about large stddev. + double mean = sampledSum / (double) sampledElements; + double sumSquareDiff = + (sampledSumSquares - (2 * mean * sampledSum) + (sampledElements * mean * mean)); + double stddev = Math.sqrt(sumSquareDiff / (sampledElements - 1)); + double sqrtDesiredSamples = + (CONFIDENCE_INTERVAL_SIGMA * stddev) / (CONFIDENCE_INTERVAL_SIZE * mean); + return (long) Math.ceil(sqrtDesiredSamples * sqrtDesiredSamples); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ProgressTracker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ProgressTracker.java new file mode 100644 index 0000000000000..fd26caa31e69b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ProgressTracker.java @@ -0,0 +1,38 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +/** + * Provides an interface to an object capable of tracking progress through a + * collection of elements to be processed. + * + * @param the type of elements being tracked + */ +public interface ProgressTracker { + /** + * Copies this {@link ProgressTracker}. The copied tracker will maintain its + * own independent notion of the caller's progress through the collection of + * elements being processed. + */ + public ProgressTracker copy(); + + /** + * Reports an element to this {@link ProgressTracker}, as the element is about + * to be processed. + */ + public void saw(T element); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ProgressTrackerGroup.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ProgressTrackerGroup.java new file mode 100644 index 0000000000000..7ed370f16770d --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ProgressTrackerGroup.java @@ -0,0 +1,71 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +/** + * Implements a group of linked + * {@link ProgressTracker ProgressTrackers} which + * collectively track how far a processing loop has gotten through the elements + * it's processing. Individual {@code ProgressTracker} instances may be copied, + * capturing an independent view of the progress of the system; this turns out + * to be useful for some non-trivial processing loops. The furthest point + * reached by any {@code ProgressTracker} is the one reported. + * + *

This class is abstract. Its single extension point is {@link #report}, + * which should be overriden to provide a function which handles the reporting + * of the supplied element, as appropriate. + * + * @param the type of elements being tracked + */ +public abstract class ProgressTrackerGroup { + // TODO: Instead of an abstract class, strongly consider adding an + // interface like Receiver to the SDK, so that this class can be final and all + // that good stuff. + private long nextIndexToReport = 0; + + public ProgressTrackerGroup() {} + + public final ProgressTracker start() { + return new Tracker(0); + } + + /** Reports the indicated element. */ + protected abstract void report(T element); + + private final class Tracker implements ProgressTracker { + private long nextElementIndex; + + private Tracker(long nextElementIndex) { + this.nextElementIndex = nextElementIndex; + } + + @Override + public ProgressTracker copy() { + return new Tracker(nextElementIndex); + } + + @Override + public void saw(T element) { + long thisElementIndex = nextElementIndex; + nextElementIndex++; + if (thisElementIndex == nextIndexToReport) { + nextIndexToReport = nextElementIndex; + report(element); + } + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ProgressTrackingReiterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ProgressTrackingReiterator.java new file mode 100644 index 0000000000000..8d5d43fa74889 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ProgressTrackingReiterator.java @@ -0,0 +1,57 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.cloud.dataflow.sdk.util.common.ForwardingReiterator; +import com.google.cloud.dataflow.sdk.util.common.Reiterator; + +/** + * Implements a {@link Reiterator} which uses a + * {@link ProgressTrackerGroup.Tracker ProgressTracker} to track how far + * it's gotten through some base {@code Reiterator}. + * {@link ProgressTrackingReiterator#copy} copies the {@code ProgressTracker}, + * allowing for an independent progress state. + * + * @param the type of the elements of this iterator + */ +public final class ProgressTrackingReiterator + extends ForwardingReiterator { + private ProgressTracker tracker; + + public ProgressTrackingReiterator(Reiterator base, + ProgressTracker tracker) { + super(base); + this.tracker = checkNotNull(tracker); + } + + @Override + public T next() { + T result = super.next(); + tracker.saw(result); + return result; + } + + @Override + protected ProgressTrackingReiterator clone() { + ProgressTrackingReiterator result = + (ProgressTrackingReiterator) super.clone(); + result.tracker = tracker.copy(); + return result; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java new file mode 100644 index 0000000000000..1930e0e61aaad --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java @@ -0,0 +1,233 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; + +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.common.base.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Observable; +import java.util.Observer; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +/** + * A read operation. + * + * Its start() method iterates through all elements of the source + * and emits them on its output. + */ +public class ReadOperation extends Operation { + private static final Logger LOG = LoggerFactory.getLogger(ReadOperation.class); + private static final long DEFAULT_PROGRESS_UPDATE_PERIOD_MS = TimeUnit.SECONDS.toMillis(1); + + /** The Source this operation reads from. */ + public final Source source; + + /** The total byte counter for all data read by this operation. */ + final Counter byteCount; + + /** StateSampler state for advancing the SourceIterator. */ + private final int readState; + + /** + * The Source's reader this operation reads from, created by start(). + * Guarded by sourceIteratorLock. + */ + volatile Source.SourceIterator sourceIterator = null; + private final Object sourceIteratorLock = new Object(); + + /** + * A cache of sourceIterator.getProgress() updated inside the read loop at a bounded rate. + *

+ * Necessary so that ReadOperation.getProgress() can return immediately, rather than potentially + * wait for a read to complete (which can take an unbounded time, delay a worker progress update, + * and cause lease expiration and all sorts of trouble). + */ + private AtomicReference progress = new AtomicReference<>(); + + /** + * On every iteration of the read loop, "progress" is fetched from sourceIterator if requested. + */ + private long progressUpdatePeriodMs = DEFAULT_PROGRESS_UPDATE_PERIOD_MS; + + /** + * Signals whether the next iteration of the read loop should update the progress. + * Set to true every progressUpdatePeriodMs. + */ + private AtomicBoolean isProgressUpdateRequested = new AtomicBoolean(true); + + + public ReadOperation(String operationName, Source source, OutputReceiver[] receivers, + String counterPrefix, CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) { + super(operationName, receivers, counterPrefix, addCounterMutator, stateSampler); + this.source = source; + this.byteCount = addCounterMutator.addCounter( + Counter.longs(bytesCounterName(counterPrefix, operationName), SUM)); + readState = stateSampler.stateForName(operationName + "-read"); + } + + /** Invoked by tests. */ + ReadOperation(Source source, OutputReceiver outputReceiver, String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) { + this("ReadOperation", source, new OutputReceiver[] {outputReceiver}, counterPrefix, + addCounterMutator, stateSampler); + } + + /** + * Invoked by tests. A value of 0 means "update progress on each iteration". + */ + void setProgressUpdatePeriodMs(long millis) { + Preconditions.checkArgument(millis >= 0, "Progress update period must be non-negative"); + progressUpdatePeriodMs = millis; + } + + protected String bytesCounterName(String counterPrefix, String operationName) { + return operationName + "-ByteCount"; + } + + public Source getSource() { + return source; + } + + @Override + public void start() throws Exception { + try (StateSampler.ScopedState start = stateSampler.scopedState(startState)) { + super.start(); + runReadLoop(); + } + } + + protected void runReadLoop() throws Exception { + Receiver receiver = receivers[0]; + if (receiver == null) { + // No consumer of this data; don't do anything. + return; + } + + source.addObserver(new SourceObserver()); + + try (StateSampler.ScopedState process = stateSampler.scopedState(processState)) { + synchronized (sourceIteratorLock) { + sourceIterator = source.iterator(); + } + + // TODO: Consider using the ExecutorService from PipelineOptions instead. + Thread updateRequester = new Thread() { + @Override + public void run() { + while (true) { + isProgressUpdateRequested.set(true); + try { + Thread.sleep(progressUpdatePeriodMs); + } catch (InterruptedException e) { + break; + } + } + } + }; + if (progressUpdatePeriodMs != 0) { + updateRequester.start(); + } + + try { + // Force a progress update at the beginning and at the end. + synchronized (sourceIteratorLock) { + progress.set(sourceIterator.getProgress()); + } + while (true) { + Object value; + // Stop position update request comes concurrently. + // Accesses to iterator need to be synchronized. + try (StateSampler.ScopedState read = stateSampler.scopedState(readState)) { + synchronized (sourceIteratorLock) { + if (!sourceIterator.hasNext()) { + break; + } + value = sourceIterator.next(); + + if (isProgressUpdateRequested.getAndSet(false) || progressUpdatePeriodMs == 0) { + progress.set(sourceIterator.getProgress()); + } + } + } + receiver.process(value); + } + synchronized (sourceIteratorLock) { + progress.set(sourceIterator.getProgress()); + } + } finally { + synchronized (sourceIteratorLock) { + sourceIterator.close(); + } + if (progressUpdatePeriodMs != 0) { + updateRequester.interrupt(); + updateRequester.join(); + } + } + } + } + + /** + * Returns a (possibly slightly stale) value of the progress of the task. + * Guaranteed to not block indefinitely. + * + * @return the task progress, or {@code null} if the source iterator has not + * been initialized + */ + public Source.Progress getProgress() { + return progress.get(); + } + + /** + * Relays the request to update the stop position to {@code SourceIterator}. + * + * @param proposedStopPosition the proposed stop position + * @return the new stop position updated in {@code SourceIterator}, or + * {@code null} if the source iterator has not been initialized + */ + public Source.Position proposeStopPosition(Source.Progress proposedStopPosition) { + synchronized (sourceIteratorLock) { + if (sourceIterator == null) { + LOG.warn("Iterator has not been initialized, returning null stop position."); + return null; + } + return sourceIterator.updateStopPosition(proposedStopPosition); + } + } + + /** + * This is an observer on the instance of the source. Whenever source reads + * an element, update() gets called with the byte size of the element, which + * gets added up into the ReadOperation's byte counter. + */ + private class SourceObserver implements Observer { + @Override + public void update(Observable obs, Object obj) { + Preconditions.checkArgument(obs == source, "unexpected observable" + obs); + Preconditions.checkArgument(obj instanceof Long, "unexpected parameter object: " + obj); + byteCount.addValue((long) obj); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Receiver.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Receiver.java new file mode 100644 index 0000000000000..f772ee4c24453 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Receiver.java @@ -0,0 +1,27 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +/** + * Abstract interface of things that accept inputs one at a time via process(). + */ +public interface Receiver { + /** + * Processes the element. + */ + void process(Object outputElem) throws Exception; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReceivingOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReceivingOperation.java new file mode 100644 index 0000000000000..60deea53fa9c5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReceivingOperation.java @@ -0,0 +1,45 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +/** + * The abstract base class for Operations that have inputs and + * implement process(). + */ +public abstract class ReceivingOperation extends Operation implements Receiver { + + public ReceivingOperation(String operationName, + OutputReceiver[] receivers, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) { + super(operationName, receivers, + counterPrefix, addCounterMutator, stateSampler); + } + + /** + * Adds an input to this Operation, coming from the given + * output of the given source Operation. + */ + public void attachInput(Operation source, int outputNum) { + checkUnstarted(); + OutputReceiver fanOut = source.receivers[outputNum]; + fanOut.addOutput(this); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleBatchReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleBatchReader.java new file mode 100644 index 0000000000000..f5102dd14e059 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleBatchReader.java @@ -0,0 +1,61 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import java.io.IOException; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * ShuffleBatchReader provides an interface for reading a batch of + * key/value entries from a shuffle dataset. + */ +public interface ShuffleBatchReader { + /** The result returned by #read. */ + public static class Batch { + public final List entries; + @Nullable public final ShufflePosition nextStartPosition; + + public Batch(List entries, + @Nullable ShufflePosition nextStartPosition) { + this.entries = entries; + this.nextStartPosition = nextStartPosition; + } + } + + /** + * Reads a batch of data from a shuffle dataset. + * + * @param startPosition encodes the initial key from where to read. + * This parameter may be null, indicating that the read should start + * with the first key in the dataset. + * + * @param endPosition encodes the key "just past" the end of the + * range to be read; keys up to endPosition will be returned, but + * keys equal to or greater than endPosition will not. This + * parameter may be null, indicating that the read should end just + * past the last key in the dataset (that is, the last key in the + * dataset will be included in the read, as long as that key is + * greater than or equal to startPosition). + * + * @return the first {@link Batch} of entries + */ + public Batch read(@Nullable ShufflePosition startPosition, + @Nullable ShufflePosition endPosition) + throws IOException; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleEntry.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleEntry.java new file mode 100644 index 0000000000000..750c3ac5c71c3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleEntry.java @@ -0,0 +1,110 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import java.util.Arrays; + +/** + * Entry written to/read from a shuffle dataset. + */ +public class ShuffleEntry { + final byte[] position; + final byte[] key; + final byte[] secondaryKey; + final byte[] value; + + public ShuffleEntry(byte[] key, byte[] secondaryKey, byte[] value) { + this.position = null; + this.key = key; + this.secondaryKey = secondaryKey; + this.value = value; + } + + public ShuffleEntry(byte[] position, byte[] key, byte[] secondaryKey, + byte[] value) { + this.position = position; + this.key = key; + this.secondaryKey = secondaryKey; + this.value = value; + } + + public byte[] getPosition() { + return position; + } + + public byte[] getKey() { + return key; + } + + public byte[] getSecondaryKey() { + return secondaryKey; + } + + public byte[] getValue() { + return value; + } + + public int length() { + return (position == null ? 0 : position.length) + + (key == null ? 0 : key.length) + + (secondaryKey == null ? 0 : secondaryKey.length) + + (value == null ? 0 : value.length); + } + + @Override + public String toString() { + return "ShuffleEntry(" + + byteArrayToString(position) + "," + + byteArrayToString(key) + "," + + byteArrayToString(secondaryKey) + "," + + byteArrayToString(value) + ")"; + } + + public static String byteArrayToString(byte[] bytes) { + // TODO: Use a more compact and readable representation, + // particularly for (nearly-)ascii keys and values. + return Arrays.toString(bytes); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o instanceof ShuffleEntry) { + ShuffleEntry that = (ShuffleEntry) o; + return (this.position == null ? that.position == null + : Arrays.equals(this.position, that.position)) + && (this.key == null ? that.key == null + : Arrays.equals(this.key, that.key)) + && (this.secondaryKey == null ? that.secondaryKey == null + : Arrays.equals(this.secondaryKey, that.secondaryKey)) + && (this.value == null ? that.value == null + : Arrays.equals(this.value, that.value)); + } + return false; + } + + @Override + public int hashCode() { + return getClass().hashCode() + + (position == null ? 0 : Arrays.hashCode(position)) + + (key == null ? 0 : Arrays.hashCode(key)) + + (secondaryKey == null ? 0 : Arrays.hashCode(secondaryKey)) + + (value == null ? 0 : Arrays.hashCode(value)); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleEntryReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleEntryReader.java new file mode 100644 index 0000000000000..bbc5f47a4b8ce --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleEntryReader.java @@ -0,0 +1,50 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import com.google.cloud.dataflow.sdk.util.common.Reiterator; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; + +/** + * ShuffleEntryReader provides an interface for reading key/value + * entries from a shuffle dataset. + */ +@NotThreadSafe +public interface ShuffleEntryReader { + /** + * Returns an iterator which reads a range of entries from a shuffle dataset. + * + * @param startPosition encodes the initial key from where to read. + * This parameter may be null, indicating that the read should start + * with the first key in the dataset. + * + * @param endPosition encodes the key "just past" the end of the + * range to be read; keys up to endPosition will be returned, but + * keys equal to or greater than endPosition will not. This + * parameter may be null, indicating that the read should end just + * past the last key in the dataset (that is, the last key in the + * dataset will be included in the read, as long as that key is + * greater than or equal to startPosition). + * + * @return a {@link Reiterator} over the requested range of entries. + */ + public Reiterator read( + @Nullable ShufflePosition startPosition, + @Nullable ShufflePosition endPosition); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShufflePosition.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShufflePosition.java new file mode 100644 index 0000000000000..c512269a49506 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ShufflePosition.java @@ -0,0 +1,23 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +/** + * Represents a position in a stream of ShuffleEntries. + */ +public interface ShufflePosition { +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Sink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Sink.java new file mode 100644 index 0000000000000..829fd1a391535 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Sink.java @@ -0,0 +1,47 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import java.io.IOException; + +/** + * Abstract base class for Sinks. + * + *

A Sink is written to by getting a SinkWriter and adding values to + * it. + * + * @param the type of the elements written to the sink + */ +public abstract class Sink { + /** + * Returns a Writer that allows writing to this Sink. + */ + public abstract SinkWriter writer() throws IOException; + + /** + * Writes to a Sink. + */ + public interface SinkWriter extends AutoCloseable { + /** + * Adds a value to the sink. Returns the size in bytes of the data written. + */ + public long add(E value) throws IOException; + + @Override + public void close() throws IOException; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Source.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Source.java new file mode 100644 index 0000000000000..d50b93dc54193 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Source.java @@ -0,0 +1,157 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import java.io.IOException; +import java.util.NoSuchElementException; +import java.util.Observable; + +/** + * Abstract base class for Sources. + * + *

A Source is read from by getting an Iterator-like value and + * iterating through it. + * + * @param the type of the elements read from the source + */ +public abstract class Source extends Observable { + /** + * Returns a SourceIterator that allows reading from this source. + */ + public abstract SourceIterator iterator() throws IOException; + + /** + * A stateful iterator over the data in a Source. + */ + public interface SourceIterator extends AutoCloseable { + /** + * Returns whether the source has any more elements. Some sources, + * such as GroupingShuffleSource, invalidate the return value of + * the previous next() call during the call to hasNext(). + */ + public boolean hasNext() throws IOException; + + /** + * Returns the next element. + * + * @throws NoSuchElementException if there are no more elements + */ + public T next() throws IOException; + + /** + * Copies the current SourceIterator. + * + * @throws UnsupportedOperationException if the particular implementation + * does not support copy + * @throws IOException if copying the iterator involves IO that fails + */ + public SourceIterator copy() throws IOException; + + @Override + public void close() throws IOException; + + /** + * Returns a representation of how far this iterator is through the source. + * + *

This method is not required to be thread-safe, and it will not be + * called concurrently to any other methods. + * + * @return the progress, or {@code null} if no progress measure + * can be provided + */ + public Progress getProgress(); + + /** + * Attempts to update the stop position of the task with the proposed stop + * position and returns the actual new stop position. + * + *

If the source finds the proposed one is not a convenient position to + * stop, it can pick a different stop position. The {@code SourceIterator} + * should start returning {@code false} from {@code hasNext()} once it has + * passed its stop position. Subsequent stop position updates must be in + * non-increasing order within a task. + * + *

This method is not required to be thread-safe, and it will not be + * called concurrently to any other methods. + * + * @param proposedStopPosition a proposed position to stop + * iterating through the source + * @return the new stop position, or {@code null} on failure if the + * implementation does not support position updates. + */ + public Position updateStopPosition(Progress proposedStopPosition); + } + + /** An abstract base class for SourceIterator implementations. */ + public abstract static class AbstractSourceIterator + implements SourceIterator { + @Override + public SourceIterator copy() throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException { + // By default, nothing is needed for close. + } + + @Override + public Progress getProgress() { + return null; + } + + @Override + public Position updateStopPosition(Progress proposedStopPosition) { + return null; + } + } + + /** + * A representation of how far a {@code SourceIterator} is through a + * {@code Source}. + * + *

The common worker framework does not interpret instances of + * this interface. But a tool-specific framework can make assumptions + * about the implementation, and so the concrete Source subclasses used + * by a tool-specific framework should match. + */ + public interface Progress { + } + + /** + * A representation of a position in an iteration through a + * {@code Source}. + * + *

See the comment on {@link Progress} for how instances of this + * interface are used by the rest of the framework. + */ + public interface Position { + } + + /** + * Utility method to notify observers about a new element, which has + * been read by this Source, and its size in bytes. Normally, there + * is only one observer, which is a ReadOperation that encapsules + * this Source. Derived classes must call this method whenever they + * read additional data, even if that element may never be returned + * from the corresponding source iterator. + */ + protected void notifyElementRead(long byteSize) { + setChanged(); + notifyObservers(byteSize); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSampler.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSampler.java new file mode 100644 index 0000000000000..91d90e9d2a05f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSampler.java @@ -0,0 +1,279 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +import java.util.AbstractMap.SimpleEntry; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; +import java.util.Timer; +import java.util.TimerTask; + +/** + * A StateSampler object may be used to obtain an approximate + * breakdown of the time spent by an execution context in various + * states, as a fraction of the total time. The sampling is taken at + * regular intervals, with adjustment for scheduling delay. + * + *

Thread-safe. + */ +public class StateSampler extends TimerTask implements AutoCloseable { + private final String prefix; + private CounterSet.AddCounterMutator counterSetMutator; + // Sampling period of internal Timer (thread). + public final long samplingPeriodMs; + public static final int DO_NOT_SAMPLE = -1; + public static final long DEFAULT_SAMPLING_PERIOD_MS = 200; + // Array of counters indexed by their state. + private ArrayList> countersByState = new ArrayList<>(); + // Map of state name to state. + private HashMap statesByName = new HashMap<>(); + // The current state. + private int currentState; + // The timestamp corresponding to the last state change or the last + // time the current state was sampled (and recorded). + private long stateTimestamp = 0; + + // When sampling this state, a stack trace is also logged. + private int stateToSampleThreadStacks = DO_NOT_SAMPLE; + // The thread that performed the last state transition. + private Thread sampledThread = null; + // The frequency with which the stack traces are logged, with respect + // to the sampling period. + private static final int SAMPLE_THREAD_STACK_FREQ = 10; + private int sampleThreadStackFreq = 0; + + // Using a fixed number of timers for all StateSampler objects. + private static final int NUM_TIMER_THREADS = 16; + // The timers is used for periodically sampling the states. + private static Timer[] timers = new Timer[NUM_TIMER_THREADS]; + static { + for (int i = 0; i < timers.length; ++i) { + timers[i] = new Timer("StateSampler_" + i, true /* is daemon */); + } + } + + /** + * Constructs a new {@link StateSampler} that can be used to obtain + * an approximate breakdown of the time spent by an execution + * context in various states, as a fraction of the total time. + * + * @param prefix the prefix of the counter names for the states + * @param counterSetMutator the {@link CounterSet.AddCounterMutator} + * used to create a counter for each distinct state + * @param samplingPeriodMs the sampling period in milliseconds + */ + public StateSampler(String prefix, + CounterSet.AddCounterMutator counterSetMutator, + long samplingPeriodMs) { + this.prefix = prefix; + this.counterSetMutator = counterSetMutator; + this.samplingPeriodMs = samplingPeriodMs; + currentState = DO_NOT_SAMPLE; + Random rand = new Random(); + int initialDelay = rand.nextInt((int) samplingPeriodMs); + timers[rand.nextInt(NUM_TIMER_THREADS)].scheduleAtFixedRate( + this, initialDelay, samplingPeriodMs); + stateTimestamp = System.currentTimeMillis(); + } + + /** + * Constructs a new {@link StateSampler} that can be used to obtain + * an approximate breakdown of the time spent by an execution + * context in various states, as a fraction of the total time. + * + * @param prefix the prefix of the counter names for the states + * @param counterSetMutator the {@link CounterSet.AddCounterMutator} + * used to create a counter for each distinct state + */ + public StateSampler(String prefix, + CounterSet.AddCounterMutator counterSetMutator) { + this(prefix, counterSetMutator, DEFAULT_SAMPLING_PERIOD_MS); + } + + private void printStackTrace(Thread thread) { + System.out.println("Sampled stack trace:"); + StackTraceElement[] stack = thread.getStackTrace(); + for (StackTraceElement elem : stack) { + System.out.println("\t" + elem.toString()); + } + } + + /** + * Selects a state for which the thread stacks will also be logged + * during the sampling. Useful for debugging. + * + * @param state name of the selected state + */ + public synchronized void setStateToSampleThreadStacks(int state) { + stateToSampleThreadStacks = state; + } + + @Override + public synchronized void run() { + long now = System.currentTimeMillis(); + if (currentState != DO_NOT_SAMPLE) { + countersByState.get(currentState).addValue(now - stateTimestamp); + if (sampledThread != null + && currentState == stateToSampleThreadStacks + && ++sampleThreadStackFreq >= SAMPLE_THREAD_STACK_FREQ) { + printStackTrace(sampledThread); + sampleThreadStackFreq = 0; + } + } + stateTimestamp = now; + } + + @Override + public void close() { + this.cancel(); // cancel the TimerTask + } + + /** + * Returns the state associated with a name; creating a new state if + * necessary. Using states instead of state names during state + * transitions is done for efficiency. + * + * @name the name for the state + * @return the state associated with the state name + */ + public int stateForName(String name) { + if (name.isEmpty()) { + return DO_NOT_SAMPLE; + } + + String counterName = prefix + name + "-msecs"; + synchronized (this) { + Integer state = statesByName.get(counterName); + if (state == null) { + Counter counter = counterSetMutator.addCounter( + Counter.longs(counterName, Counter.AggregationKind.SUM)); + state = countersByState.size(); + statesByName.put(name, state); + countersByState.add(counter); + } + return state; + } + } + + /** + * Sets the current thread state. + * + * @param state the new state to transition to + * @return the previous state + */ + public synchronized int setState(int state) { + // TODO: investigate whether this can be made cheaper, (e.g., + // using atomic operations). + int previousState = currentState; + currentState = state; + if (stateToSampleThreadStacks != DO_NOT_SAMPLE) { + sampledThread = Thread.currentThread(); + } + return previousState; + } + + /** + * Sets the current thread state. + * + * @param name the name of the new state to transition to + * @return the previous state + */ + public synchronized int setState(String name) { + return setState(stateForName(name)); + } + + /** + * Returns a tuple consisting of the current state and duration. + * + * @return a {@link Map.Entry} entry with current state and duration + */ + public synchronized Map.Entry getCurrentStateAndDuration() { + if (currentState == DO_NOT_SAMPLE) { + return new SimpleEntry<>("", 0L); + } + + Counter counter = countersByState.get(currentState); + return new SimpleEntry<>(counter.getName(), + counter.getAggregate(false) + + System.currentTimeMillis() - stateTimestamp); + } + + /** + * Get the duration for a given state. + * + * @param state the state whose duration is returned + * @return the duration of a given state + */ + public synchronized long getStateDuration(int state) { + Counter counter = countersByState.get(state); + return counter.getAggregate(false) + + (state == currentState + ? System.currentTimeMillis() - stateTimestamp : 0); + } + + /** + * Returns an AutoCloseable {@link ScopedState} that will perform a + * state transition to the given state, and will automatically reset + * the state to the prior state upon closing. + * + * @param state the new state to transition to + * @return a {@link ScopedState} that automatically resets the state + * to the prior state + */ + public synchronized ScopedState scopedState(int state) { + return new ScopedState(this, setState(state)); + } + + /** + * Returns an AutoCloseable {@link ScopedState} that will perform a + * state transition to the given state, and will automatically reset + * the state to the prior state upon closing. + * + * @param stateName the name of the new state + * @return a {@link ScopedState} that automatically resets the state + * to the prior state + */ + public synchronized ScopedState scopedState(String stateName) { + return new ScopedState(this, setState(stateName)); + } + + /** + * A nested class that is used to account for states and state + * transitions based on lexical scopes. + * + *

Thread-safe. + */ + public class ScopedState implements AutoCloseable { + private StateSampler sampler; + private int previousState; + + private ScopedState(StateSampler sampler, int previousState) { + this.sampler = sampler; + this.previousState = previousState; + } + + @Override + public void close() { + sampler.setState(previousState); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java new file mode 100644 index 0000000000000..63270b682ebce --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java @@ -0,0 +1,99 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.Metric; +import com.google.cloud.dataflow.sdk.util.common.Metric.DoubleMetric; + +import com.sun.management.OperatingSystemMXBean; + +import java.lang.management.ManagementFactory; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +/** + * Abstract executor for WorkItem tasks. + */ +public abstract class WorkExecutor implements AutoCloseable { + /** The output counters for this task. */ + private final CounterSet outputCounters; + + /** + * OperatingSystemMXBean for reporting CPU usage. + * + * Uses com.sun.management.OperatingSystemMXBean instead of + * java.lang.management.OperatingSystemMXBean because the former supports + * getProcessCpuLoad(). + */ + private final OperatingSystemMXBean os; + + /** + * Constructs a new WorkExecutor task. + */ + public WorkExecutor(CounterSet outputCounters) { + this.outputCounters = outputCounters; + this.os = + (OperatingSystemMXBean) ManagementFactory.getOperatingSystemMXBean(); + } + + /** + * Returns the set of output counters for this task. + */ + public CounterSet getOutputCounters() { + return outputCounters; + } + + /** + * Returns a collection of output metrics for this task. + */ + public Collection> getOutputMetrics() { + List> outputMetrics = new ArrayList<>(); + outputMetrics.add(new DoubleMetric("CPU", os.getProcessCpuLoad())); + // More metrics as needed. + return outputMetrics; + } + + /** + * Executes the task. + */ + public abstract void execute() throws Exception; + + /** + * Returns the worker's current progress. + */ + public Source.Progress getWorkerProgress() throws Exception { + // By default, return null indicating worker progress not available. + return null; + } + + /** + * Proposes that the worker changes the stop position for the current work. + * Returns the new position if accepted, otherwise {@code null}. + */ + public Source.Position proposeStopPosition( + Source.Progress proposedStopPosition) throws Exception { + // By default, returns null indicating that no task splitting happens. + return null; + } + + @Override + public void close() throws Exception { + // By default, nothing to close or shut down. + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java new file mode 100644 index 0000000000000..c5222eb04a2f1 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java @@ -0,0 +1,239 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import javax.annotation.concurrent.NotThreadSafe; + +/** + * WorkProgressUpdater allows a work executor to send work progress + * updates to the worker service. The life-cycle of the + * WorkProgressUpdater is controlled externally through its + * {@link #startReportingProgress()} and + * {@link #stopReportingProgress()} methods. The updater queries the + * worker for progress updates and sends the updates to the worker + * service. The interval between two consecutive updates is + * controlled by the worker service through reporting interval hints + * sent back in the update response messages. To avoid update storms + * and monitoring staleness, the interval between two consecutive + * updates is also bound by {@link #MIN_REPORTING_INTERVAL_MILLIS} and + * {@link #MAX_REPORTING_INTERVAL_MILLIS}. + */ +@NotThreadSafe +public abstract class WorkProgressUpdater { + private static final Logger LOG = LoggerFactory.getLogger(WorkProgressUpdater.class); + + /** The default lease duration to request from the external worker service. */ + private static final long DEFAULT_LEASE_DURATION_MILLIS = 3 * 60 * 1000; + + /** The lease renewal RPC latency margin. */ + private static final long LEASE_RENEWAL_LATENCY_MARGIN = Long.valueOf( + System.getProperty("worker_lease_renewal_latency_margin", "5000")); + + /** + * The minimum period between two consecutive progress updates. Ensures the + * {@link WorkProgressUpdater} does not generate update storms. + */ + private static final long MIN_REPORTING_INTERVAL_MILLIS = Long.valueOf( + System.getProperty("minimum_worker_update_interval_millis", "5000")); + + /** + * The maximum period between two consecutive progress updates. Ensures the + * {@link WorkProgressUpdater} does not cause monitoring staleness. + */ + private static final long MAX_REPORTING_INTERVAL_MILLIS = 10 * 60 * 1000; + + /** Worker providing the work progress updates. */ + protected final WorkExecutor worker; + + /** Executor used to schedule work progress updates. */ + private final ScheduledExecutorService executor; + + /** The lease duration to request from the external worker service. */ + protected long requestedLeaseDurationMs; + + /** The time period until the next work progress update. */ + protected long progressReportIntervalMs; + + /** + * The stop position to report to the service in the next progress update, + * or {@code null} if there is nothing to report. + * In cases that there is no split request from service, or worker failed to + * split in response to the last received split request, the task stop + * position implicitly stays the same as it was before that last request + * (as a result of a prior split request), and on the next reportProgress + * we'll send the {@code null} as a stop position update, which is a no-op + * for the service. + */ + protected Source.Position stopPositionToService; + + public WorkProgressUpdater(WorkExecutor worker) { + this.worker = worker; + this.executor = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("WorkProgressUpdater-%d") + .build()); + } + + /** + * Starts sending work progress updates to the worker service. + */ + public void startReportingProgress() { + // Send the initial work progress report half-way through the lease + // expiration. Subsequent intervals adapt to hints from the service. + long leaseRemainingTime = + leaseRemainingTime(getWorkUnitLeaseExpirationTimestamp()); + progressReportIntervalMs = nextProgressReportInterval( + leaseRemainingTime / 2, leaseRemainingTime); + requestedLeaseDurationMs = DEFAULT_LEASE_DURATION_MILLIS; + + LOG.info("Started reporting progress for work item: {}", workString()); + scheduleNextUpdate(); + } + + /** + * Stops sending work progress updates to the worker service. + * It may throw an exception if the final progress report fails to be sent for some reason. + */ + public void stopReportingProgress() throws Exception { + // TODO: Redesign to get rid of the executor and use a dedicated + // thread with a sleeper. Also unify with success/failure reporting. + + // Wait until there are no more progress updates in progress, then + // shut down. + synchronized (executor) { + executor.shutdownNow(); + } + + // We send a final progress report in case there was an unreported stop position update. + if (stopPositionToService != null) { + LOG.info("Sending final progress update with unreported stop position."); + reportProgressHelper(); // This call can fail with an exception + } + + LOG.info("Stopped reporting progress for work item: {}", workString()); + } + + /** + * Computes the time before sending the next work progress update making sure + * that it falls between the [{@link #MIN_REPORTING_INTERVAL_MILLIS}, + * {@link #MAX_REPORTING_INTERVAL_MILLIS}) interval. Makes an attempt to bound + * the result by the remaining lease time, with an RPC latency margin of + * {@link #LEASE_RENEWAL_LATENCY_MARGIN}. + * + * @param suggestedInterval the suggested progress report interval + * @param leaseRemainingTime milliseconds left before the work lease expires + * @return the time in milliseconds before sending the next progress update + */ + protected static long nextProgressReportInterval(long suggestedInterval, + long leaseRemainingTime) { + // Sanitize input in case we get a negative suggested time interval. + suggestedInterval = Math.max(0, suggestedInterval); + + // Try to send the next progress update before the next lease expiration + // allowing some RPC latency margin. + suggestedInterval = Math.min(suggestedInterval, + leaseRemainingTime - LEASE_RENEWAL_LATENCY_MARGIN); + + // Bound reporting interval to avoid staleness and progress update storms. + return Math.min(Math.max(MIN_REPORTING_INTERVAL_MILLIS, suggestedInterval), + MAX_REPORTING_INTERVAL_MILLIS); + } + + /** + * Schedules the next work progress update. + */ + private void scheduleNextUpdate() { + if (executor.isShutdown()) { + return; + } + executor.schedule(new Runnable() { + @Override + public void run() { + // Don't shut down while reporting progress. + synchronized (executor) { + if (executor.isShutdown()) { + return; + } + reportProgress(); + } + } + }, progressReportIntervalMs, TimeUnit.MILLISECONDS); + LOG.debug("Next work progress update for work item {} scheduled to occur in {} ms.", + workString(), progressReportIntervalMs); + } + + /** + * Reports the current work progress to the worker service. + */ + private void reportProgress() { + LOG.info("Updating progress on work item {}", workString()); + try { + reportProgressHelper(); + } catch (Throwable e) { + LOG.warn("Error reporting work progress update: ", e); + } finally { + scheduleNextUpdate(); + } + } + + /** + * Computes the amount of time left, in milliseconds, before a lease + * with the specified expiration timestamp expires. Returns zero if + * the lease has already expired. + */ + protected long leaseRemainingTime(long leaseExpirationTimestamp) { + long now = System.currentTimeMillis(); + if (leaseExpirationTimestamp < now) { + LOG.debug("Lease remaining time for {} is 0 ms.", workString()); + return 0; + } + LOG.debug("Lease remaining time for {} is {} ms.", + workString(), leaseExpirationTimestamp - now); + return leaseExpirationTimestamp - now; + } + + // Visible for testing. + public Source.Position getStopPosition() { + return stopPositionToService; + } + + /** + * Reports the current work progress to the worker service. + */ + protected abstract void reportProgressHelper() throws Exception; + + /** + * Returns the current work item's lease expiration timestamp. + */ + protected abstract long getWorkUnitLeaseExpirationTimestamp(); + + /** + * Returns a string representation of the work item whose progress + * is being updated, for use in logging messages. + */ + protected abstract String workString(); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperation.java new file mode 100644 index 0000000000000..6f8b2e586548f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperation.java @@ -0,0 +1,105 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; + +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +/** + * A write operation. + */ +public class WriteOperation extends ReceivingOperation { + /** + * The Sink this operation writes to. + */ + public final Sink sink; + + /** + * The total byte counter for all data written by this operation. + */ + final Counter byteCount; + + /** + * The Sink's writer this operation writes to, created by start(). + */ + Sink.SinkWriter writer; + + public WriteOperation(String operationName, + Sink sink, + OutputReceiver[] receivers, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) { + super(operationName, receivers, + counterPrefix, addCounterMutator, stateSampler); + this.sink = sink; + this.byteCount = addCounterMutator.addCounter( + Counter.longs(bytesCounterName(counterPrefix, operationName), SUM)); + } + + /** Invoked by tests. */ + public WriteOperation(Sink sink, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) { + this("WriteOperation", sink, new OutputReceiver[]{ }, + counterPrefix, addCounterMutator, stateSampler); + } + + protected String bytesCounterName(String counterPrefix, + String operationName) { + return operationName + "-ByteCount"; + } + + public Sink getSink() { + return sink; + } + + @Override + public void start() throws Exception { + try (StateSampler.ScopedState start = + stateSampler.scopedState(startState)) { + super.start(); + writer = sink.writer(); + } + } + + @Override + public void process(Object outputElem) throws Exception { + try (StateSampler.ScopedState process = + stateSampler.scopedState(processState)) { + checkStarted(); + byteCount.addValue(writer.add(outputElem)); + } + } + + @Override + public void finish() throws Exception { + try (StateSampler.ScopedState finish = + stateSampler.scopedState(finishState)) { + checkStarted(); + writer.close(); + super.finish(); + } + } + + public Counter getByteCount() { + return byteCount; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/package-info.java new file mode 100644 index 0000000000000..1bef723c9ac75 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** Defines utilities used to implement the harness that runs user code. **/ +package com.google.cloud.dataflow.sdk.util.common.worker; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPath.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPath.java new file mode 100644 index 0000000000000..f1da8b767ef2e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPath.java @@ -0,0 +1,617 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.gcsfs; + +import com.google.api.client.util.Preconditions; +import com.google.api.client.util.Strings; +import com.google.api.services.storage.model.StorageObject; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.nio.file.FileSystem; +import java.nio.file.LinkOption; +import java.nio.file.Path; +import java.nio.file.WatchEvent; +import java.nio.file.WatchKey; +import java.nio.file.WatchService; +import java.util.Iterator; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +/** + * Implements the Java NIO {@link Path} API for Google Cloud Storage paths. + * + *

GcsPath uses a slash ('/') as a directory separator. Below is + * a summary of how slashes are treated: + *

    + *
  • A GCS bucket may not contain a slash. An object may contain zero or + * more slashes. + *
  • A trailing slash always indicates a directory, which is compliant + * with POSIX.1-2008. + *
  • Slashes separate components of a path. Empty components are allowed, + * which is represented as repeated slashes. An empty component always + * refers to a directory, and always ends in a slash. + *
  • {@link #getParent()}} always returns a path ending in a slash, as the + * parent of a GcsPath is always a directory. + *
  • Use {@link #resolve(String)} to append elements to a GcsPath -- this + * applies the rules consistently and is highly recommended over any + * custom string concatenation. + *
+ * + *

GcsPath treats all GCS objects and buckets as belonging to the same + * filesystem, so the root of a GcsPath is the GcsPath bucket="", object="". + * + *

Relative paths are not associated with any bucket. This matches common + * treatment of Path in which relative paths can be constructed from one + * filesystem and appended to another filesystem. + * + * @see Java Tutorials: Path Operations + */ +public class GcsPath implements Path { + + public static final String SCHEME = "gs"; + + /** + * Creates a GcsPath from a URI. + * + *

The URI must be in the form {@code gs://[bucket]/[path]}, and may not + * contain a port, user info, a query, or a fragment. + */ + public static GcsPath fromUri(URI uri) { + Preconditions.checkArgument(uri.getScheme().equalsIgnoreCase(SCHEME), + "URI: %s is not a GCS URI", uri); + Preconditions.checkArgument(uri.getPort() == -1, + "GCS URI may not specify port: %s (%i)", uri, uri.getPort()); + Preconditions.checkArgument( + Strings.isNullOrEmpty(uri.getUserInfo()), + "GCS URI may not specify userInfo: %s (%s)", uri, uri.getUserInfo()); + Preconditions.checkArgument( + Strings.isNullOrEmpty(uri.getQuery()), + "GCS URI may not specify query: %s (%s)", uri, uri.getQuery()); + Preconditions.checkArgument( + Strings.isNullOrEmpty(uri.getFragment()), + "GCS URI may not specify fragment: %s (%s)", uri, uri.getFragment()); + + return fromUri(uri.toString()); + } + + /** + * Pattern which is used to parse a GCS URL. + * + *

This is used to separate the components. Verification is handled + * separately. + */ + private static final Pattern GCS_URI = + Pattern.compile("(?[^:]+)://(?[^/]+)(/(?.*))?"); + + /** + * Creates a GcsPath from a URI in string form. + * + *

This does not use URI parsing, which means it may accept patterns that + * the URI parser would not accept. + */ + public static GcsPath fromUri(String uri) { + Matcher m = GCS_URI.matcher(uri); + Preconditions.checkArgument(m.matches(), "Invalid GCS URI: %s", uri); + + Preconditions.checkArgument(m.group("SCHEME").equalsIgnoreCase(SCHEME), + "URI: %s is not a GCS URI", uri); + return new GcsPath(null, m.group("BUCKET"), m.group("OBJECT")); + } + + /** + * Pattern which is used to parse a GCS resource name. + */ + private static final Pattern GCS_RESOURCE_NAME = + Pattern.compile("storage.googleapis.com/(?[^/]+)(/(?.*))?"); + + /** + * Creates a GcsPath from a OnePlatform resource name in string form. + */ + public static GcsPath fromResourceName(String name) { + Matcher m = GCS_RESOURCE_NAME.matcher(name); + Preconditions.checkArgument(m.matches(), "Invalid GCS resource name: %s", name); + + return new GcsPath(null, m.group("BUCKET"), m.group("OBJECT")); + } + + /** + * Creates a GcsPath from a {@linkplain StorageObject}. + */ + public static GcsPath fromObject(StorageObject object) { + return new GcsPath(null, object.getBucket(), object.getName()); + } + + /** + * Creates a GcsPath from bucket and object components. + * + *

A GcsPath without a bucket name is treated as a relative path, which + * is a path component with no linkage to the root element. This is similar + * to a Unix path which does not begin with the root marker (a slash). + * GCS has different naming constraints and APIs for working with buckets and + * objects, so these two concepts are kept separate to avoid accidental + * attempts to treat objects as buckets, or vice versa, as much as possible. + * + *

A GcsPath without an object name is a bucket reference. + * A bucket is always a directory, which could be used to lookup or add + * files to a bucket, but could not be opened as a file. + * + *

A GcsPath containing neither bucket or object names is treated as + * the root of the GCS filesystem. A listing on the root element would return + * the buckets available to the user. + * + *

If {@code null} is passed as either parameter, it is converted to an + * empty string internally for consistency. There is no distinction between + * an empty string and a {@code null}, as neither are allowed by GCS. + * + * @param bucket a GCS bucket name, or none ({@code null} or an empty string) + * if the object is not associated with a bucket + * (e.g. relative paths or the root node). + * @param object a GCS object path, or none ({@code null} or an empty string) + * for no object. + */ + public static GcsPath fromComponents(@Nullable String bucket, + @Nullable String object) { + return new GcsPath(null, bucket, object); + } + + @Nullable + private FileSystem fs; + @Nonnull + private final String bucket; + @Nonnull + private final String object; + + /** + * Constructs a GcsPath. + * + * @param fs the associated FileSystem, if any + * @param bucket the associated bucket, or none ({@code null} or an empty + * string) for a relative path component + * @param object the object, which is a fully-qualified object name if bucket + * was also provided, or none ({@code null} or an empty string) + * for no object + * @throws java.lang.IllegalArgumentException if the bucket of object names + * are invalid. + */ + public GcsPath(@Nullable FileSystem fs, + @Nullable String bucket, + @Nullable String object) { + if (bucket == null) { + bucket = ""; + } + Preconditions.checkArgument(!bucket.contains("/"), + "GCS bucket may not contain a slash"); + Preconditions + .checkArgument(bucket.isEmpty() + || bucket.matches("[a-z0-9][-_a-z0-9.]+[a-z0-9]"), + "GCS bucket names must contain only lowercase letters, numbers, " + + "dashes (-), underscores (_), and dots (.). Bucket names " + + "must start and end with a number or letter. " + + "See https://developers.google.com/storage/docs/bucketnaming " + + "for more details. Bucket name: " + bucket); + + if (object == null) { + object = ""; + } + Preconditions.checkArgument( + object.indexOf('\n') < 0 && object.indexOf('\r') < 0, + "GCS object names must not contain Carriage Return or " + + "Line Feed characters."); + + this.fs = fs; + this.bucket = bucket; + this.object = object; + } + + /** + * Returns the bucket name associated with this GCS path, or an empty string + * if this is a relative path component. + */ + public String getBucket() { + return bucket; + } + + /** + * Returns the object name associated with this GCS path, or an empty string + * if no object is specified. + */ + public String getObject() { + return object; + } + + public void setFileSystem(FileSystem fs) { + this.fs = fs; + } + + @Override + public FileSystem getFileSystem() { + return fs; + } + + // Absolute paths are those which have a bucket and the root path. + @Override + public boolean isAbsolute() { + return !bucket.isEmpty() || object.isEmpty(); + } + + @Override + public GcsPath getRoot() { + return new GcsPath(fs, "", ""); + } + + @Override + public GcsPath getFileName() { + throw new UnsupportedOperationException(); + } + + /** + * Returns the parent path, or {@code null} if this path does not + * have a parent. + * + *

Returns a path which ends in '/', as the parent path always refers to + * a directory. + */ + @Override + public GcsPath getParent() { + if (bucket.isEmpty() && object.isEmpty()) { + // The root path has no parent, by definition. + return null; + } + + if (object.isEmpty()) { + // A GCS bucket. All buckets come from a common root. + return getRoot(); + } + + // Skip last character, in case it is a trailing slash. + int i = object.lastIndexOf('/', object.length() - 2); + if (i <= 0) { + if (bucket.isEmpty()) { + // Relative paths are not attached to the root node. + return null; + } + return new GcsPath(fs, bucket, ""); + } + + // Retain trailing slash. + return new GcsPath(fs, bucket, object.substring(0, i + 1)); + } + + @Override + public int getNameCount() { + int count = bucket.isEmpty() ? 0 : 1; + if (object.isEmpty()) { + return count; + } + + // Add another for each separator found. + int index = -1; + while ((index = object.indexOf('/', index + 1)) != -1) { + count++; + } + + return object.endsWith("/") ? count : count + 1; + } + + @Override + public GcsPath getName(int count) { + Preconditions.checkArgument(count >= 0); + + Iterator iterator = iterator(); + for (int i = 0; i < count; ++i) { + Preconditions.checkArgument(iterator.hasNext()); + iterator.next(); + } + + Preconditions.checkArgument(iterator.hasNext()); + return (GcsPath) iterator.next(); + } + + @Override + public GcsPath subpath(int beginIndex, int endIndex) { + Preconditions.checkArgument(beginIndex >= 0); + Preconditions.checkArgument(endIndex > beginIndex); + + Iterator iterator = iterator(); + for (int i = 0; i < beginIndex; ++i) { + Preconditions.checkArgument(iterator.hasNext()); + iterator.next(); + } + + GcsPath path = null; + while (beginIndex < endIndex) { + Preconditions.checkArgument(iterator.hasNext()); + if (path == null) { + path = (GcsPath) iterator.next(); + } else { + path = path.resolve(iterator.next()); + } + ++beginIndex; + } + + return path; + } + + @Override + public boolean startsWith(Path other) { + if (other instanceof GcsPath) { + GcsPath gcsPath = (GcsPath) other; + return startsWith(gcsPath.bucketAndObject()); + } else { + return startsWith(other.toString()); + } + } + + @Override + public boolean startsWith(String prefix) { + return bucketAndObject().startsWith(prefix); + } + + @Override + public boolean endsWith(Path other) { + if (other instanceof GcsPath) { + GcsPath gcsPath = (GcsPath) other; + return endsWith(gcsPath.bucketAndObject()); + } else { + return endsWith(other.toString()); + } + } + + @Override + public boolean endsWith(String suffix) { + return bucketAndObject().endsWith(suffix); + } + + // TODO: support "." and ".." path components? + @Override + public GcsPath normalize() { return this; } + + @Override + public GcsPath resolve(Path other) { + if (other instanceof GcsPath) { + GcsPath path = (GcsPath) other; + if (path.isAbsolute()) { + return path; + } else { + return resolve(path.getObject()); + } + } else { + return resolve(other.toString()); + } + } + + @Override + public GcsPath resolve(String other) { + if (bucket.isEmpty() && object.isEmpty()) { + // Resolve on a root path is equivalent to looking up a bucket and object. + other = SCHEME + "://" + other; + } + + if (other.startsWith(SCHEME + "://")) { + GcsPath path = GcsPath.fromUri(other); + path.setFileSystem(getFileSystem()); + return path; + } + + if (other.isEmpty()) { + // An empty component MUST refer to a directory. + other = "/"; + } + + if (object.isEmpty()) { + return new GcsPath(fs, bucket, other); + } else if (object.endsWith("/")) { + return new GcsPath(fs, bucket, object + other); + } else { + return new GcsPath(fs, bucket, object + "/" + other); + } + } + + @Override + public Path resolveSibling(Path other) { + throw new UnsupportedOperationException(); + } + + @Override + public Path resolveSibling(String other) { + throw new UnsupportedOperationException(); + } + + @Override + public Path relativize(Path other) { + throw new UnsupportedOperationException(); + } + + @Override + public GcsPath toAbsolutePath() { + return this; + } + + @Override + public GcsPath toRealPath(LinkOption... options) throws IOException { + return this; + } + + @Override + public File toFile() { + throw new UnsupportedOperationException(); + } + + @Override + public WatchKey register(WatchService watcher, WatchEvent.Kind[] events, + WatchEvent.Modifier... modifiers) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public WatchKey register(WatchService watcher, WatchEvent.Kind... events) + throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public Iterator iterator() { + return new NameIterator(fs, !bucket.isEmpty(), bucketAndObject()); + } + + private static class NameIterator implements Iterator { + private final FileSystem fs; + private boolean fullPath; + private String name; + + NameIterator(FileSystem fs, boolean fullPath, String name) { + this.fs = fs; + this.fullPath = fullPath; + this.name = name; + } + + @Override + public boolean hasNext() { + return !Strings.isNullOrEmpty(name); + } + + @Override + public GcsPath next() { + int i = name.indexOf('/'); + String component; + if (i >= 0) { + component = name.substring(0, i); + name = name.substring(i + 1); + } else { + component = name; + name = null; + } + if (fullPath) { + fullPath = false; + return new GcsPath(fs, component, ""); + } else { + // Relative paths have no bucket. + return new GcsPath(fs, "", component); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + } + + @Override + public int compareTo(Path other) { + if (!(other instanceof GcsPath)) { + throw new ClassCastException(); + } + + GcsPath path = (GcsPath) other; + int b = bucket.compareTo(path.bucket); + if (b != 0) { + return b; + } + + // Compare a component at a time, so that the separator char doesn't + // get compared against component contents. Eg, "a/b" < "a-1/b". + Iterator left = iterator(); + Iterator right = path.iterator(); + + while (left.hasNext() && right.hasNext()) { + String leftStr = left.next().toString(); + String rightStr = right.next().toString(); + int c = leftStr.compareTo(rightStr); + if (c != 0) { + return c; + } + } + + if (!left.hasNext() && !right.hasNext()) { + return 0; + } else { + return left.hasNext() ? 1 : -1; + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + GcsPath paths = (GcsPath) o; + return bucket.equals(paths.bucket) && object.equals(paths.object); + } + + @Override + public int hashCode() { + int result = bucket.hashCode(); + result = 31 * result + object.hashCode(); + return result; + } + + @Override + public String toString() { + if (!isAbsolute()) { + return object; + } + StringBuilder sb = new StringBuilder(); + sb.append(SCHEME) + .append("://"); + if (!bucket.isEmpty()) { + sb.append(bucket) + .append('/'); + } + sb.append(object); + return sb.toString(); + } + + // TODO: Consider using resource names for all GCS paths used by the SDK. + public String toResourceName() { + StringBuilder sb = new StringBuilder(); + sb.append("storage.googleapis.com/"); + if (!bucket.isEmpty()) { + sb.append(bucket).append('/'); + } + sb.append(object); + return sb.toString(); + } + + @Override + public URI toUri() { + try { + return new URI(SCHEME, "//" + bucketAndObject(), null); + } catch (URISyntaxException e) { + throw new RuntimeException("Unable to create URI for GCS path " + this); + } + } + + private String bucketAndObject() { + if (bucket.isEmpty()) { + return object; + } else { + return bucket + "/" + object; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/package-info.java new file mode 100644 index 0000000000000..6784109e82af8 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** Defines utilities used to interact with Google Cloud Storage. **/ +package com.google.cloud.dataflow.sdk.util.gcsfs; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/ClientRequestHelper.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/ClientRequestHelper.java new file mode 100644 index 0000000000000..155dd79f795b3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/ClientRequestHelper.java @@ -0,0 +1,40 @@ +/** + * Copyright 2013 Google Inc. All Rights Reserved. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.gcsio; + +import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; +import com.google.api.client.http.HttpHeaders; + +/** + * ClientRequestHelper provides wrapper methods around final methods of AbstractGoogleClientRequest + * to allow overriding them if necessary. Typically should be used for testing purposes only. + */ +public class ClientRequestHelper { + /** + * Wraps AbstractGoogleClientRequest.getRequestHeaders(). + */ + public HttpHeaders getRequestHeaders(AbstractGoogleClientRequest clientRequest) { + return clientRequest.getRequestHeaders(); + } + + /** + * Wraps AbstractGoogleClientRequest.getMediaHttpUploader(). + */ + public void setChunkSize(AbstractGoogleClientRequest clientRequest, int chunkSize) { + clientRequest.getMediaHttpUploader().setChunkSize(chunkSize); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageExceptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageExceptions.java new file mode 100644 index 0000000000000..5535a90826a9a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageExceptions.java @@ -0,0 +1,82 @@ +/** + * Copyright 2013 Google Inc. All Rights Reserved. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.gcsio; + +import com.google.api.client.util.Preconditions; +import com.google.common.base.Strings; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.List; + +/** + * Miscellaneous helper methods for standardizing the types of exceptions thrown by the various + * GCS-based FileSystems. + */ +public class GoogleCloudStorageExceptions { + /** + * Creates FileNotFoundException with suitable message for a GCS bucket or object. + */ + public static FileNotFoundException getFileNotFoundException( + String bucketName, String objectName) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(bucketName), + "bucketName must not be null or empty"); + if (objectName == null) { + objectName = ""; + } + return new FileNotFoundException( + String.format("Item not found: %s/%s", bucketName, objectName)); + } + + /** + * Creates a composite IOException out of multiple IOExceptions. If there is only a single + * {@code innerException}, it will be returned as-is without wrapping into an outer exception. + * it. + */ + public static IOException createCompositeException( + List innerExceptions) { + Preconditions.checkArgument(innerExceptions != null, + "innerExceptions must not be null"); + Preconditions.checkArgument(innerExceptions.size() > 0, + "innerExceptions must contain at least one element"); + + if (innerExceptions.size() == 1) { + return innerExceptions.get(0); + } + + IOException combined = new IOException("Multiple IOExceptions."); + for (IOException inner : innerExceptions) { + combined.addSuppressed(inner); + } + return combined; + } + + /** + * Wraps the given IOException into another IOException, adding the given error message and a + * reference to the supplied bucket and object. It allows one to know which bucket and object + * were being accessed when the exception occurred for an operation. + */ + public static IOException wrapException(IOException e, String message, + String bucketName, String objectName) { + String name = "bucket: " + bucketName; + if (!Strings.isNullOrEmpty(objectName)) { + name += ", object: " + objectName; + } + String fullMessage = String.format("%s: %s", message, name); + return new IOException(fullMessage, e); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java new file mode 100644 index 0000000000000..a3d9b65347b2a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java @@ -0,0 +1,538 @@ +/** + * Copyright 2013 Google Inc. All Rights Reserved. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.gcsio; + +import com.google.api.client.http.HttpResponse; +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.ExponentialBackOff; +import com.google.api.client.util.NanoClock; +import com.google.api.client.util.Preconditions; +import com.google.api.client.util.Sleeper; +import com.google.api.services.storage.Storage; +import com.google.cloud.dataflow.sdk.util.ApiErrorExtractor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.nio.channels.ClosedChannelException; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.SeekableByteChannel; +import java.util.regex.Pattern; + +/** + * Provides seekable read access to GCS. + */ +public class GoogleCloudStorageReadChannel implements SeekableByteChannel { + // Logger. + private static final Logger LOG = LoggerFactory.getLogger(GoogleCloudStorageReadChannel.class); + + // Used to separate elements of a Content-Range + private static final Pattern SLASH = Pattern.compile("/"); + + // GCS access instance. + private Storage gcs; + + // Name of the bucket containing the object being read. + private String bucketName; + + // Name of the object being read. + private String objectName; + + // Read channel. + private ReadableByteChannel readChannel; + + // True if this channel is open, false otherwise. + private boolean channelIsOpen; + + // Current read position in the channel. + private long currentPosition = -1; + + // When a caller calls position(long) to set stream position, we record the target position + // and defer the actual seek operation until the caller tries to read from the channel. + // This allows us to avoid an unnecessary seek to position 0 that would take place on creation + // of this instance in cases where caller intends to start reading at some other offset. + // If lazySeekPending is set to true, it indicates that a target position has been set + // but the actual seek operation is still pending. + private boolean lazySeekPending; + + // Size of the object being read. + private long size = -1; + + // Maximum number of automatic retries when reading from the underlying channel without making + // progress; each time at least one byte is successfully read, the counter of attempted retries + // is reset. + // TODO: Wire this setting out to GHFS; it should correspond to adding the wiring for + // setting the equivalent value inside HttpRequest.java which determines the low-level retries + // during "execute()" calls. The default in HttpRequest.java is also 10. + private int maxRetries = 10; + + // Helper delegate for turning IOExceptions from API calls into higher-level semantics. + private final ApiErrorExtractor errorExtractor; + + // Sleeper used for waiting between retries. + private Sleeper sleeper = Sleeper.DEFAULT; + + // The clock used by ExponentialBackOff to determine when the maximum total elapsed time has + // passed doing a series of retries. + private NanoClock clock = NanoClock.SYSTEM; + + // Lazily initialized BackOff for sleeping between retries; only ever initialized if a retry is + // necessary. + private BackOff backOff = null; + + // Settings used for instantiating the default BackOff used for determining wait time between + // retries. TODO: Wire these out to be settable by the Hadoop configs. + // The number of milliseconds to wait before the very first retry in a series of retries. + public static final int DEFAULT_BACKOFF_INITIAL_INTERVAL_MILLIS = 200; + + // The amount of jitter introduced when computing the next retry sleep interval so that when + // many clients are retrying, they don't all retry at the same time. + public static final double DEFAULT_BACKOFF_RANDOMIZATION_FACTOR = 0.5; + + // The base of the exponent used for exponential backoff; each subsequent sleep interval is + // roughly this many times the previous interval. + public static final double DEFAULT_BACKOFF_MULTIPLIER = 1.5; + + // The maximum amount of sleep between retries; at this point, there will be no further + // exponential backoff. This prevents intervals from growing unreasonably large. + public static final int DEFAULT_BACKOFF_MAX_INTERVAL_MILLIS = 10 * 1000; + + // The maximum total time elapsed since the first retry over the course of a series of retries. + // This makes it easier to bound the maximum time it takes to respond to a permanent failure + // without having to calculate the summation of a series of exponentiated intervals while + // accounting for the randomization of backoff intervals. + public static final int DEFAULT_BACKOFF_MAX_ELAPSED_TIME_MILLIS = 2 * 60 * 1000; + + // ClientRequestHelper to be used instead of calling final methods in client requests. + private static ClientRequestHelper clientRequestHelper = new ClientRequestHelper(); + + /** + * Constructs an instance of GoogleCloudStorageReadChannel. + * + * @param gcs storage object instance + * @param bucketName name of the bucket containing the object to read + * @param objectName name of the object to read + * @throws java.io.FileNotFoundException if the given object does not exist + * @throws IOException on IO error + */ + public GoogleCloudStorageReadChannel( + Storage gcs, String bucketName, String objectName, ApiErrorExtractor errorExtractor) + throws IOException { + this.gcs = gcs; + this.bucketName = bucketName; + this.objectName = objectName; + this.errorExtractor = errorExtractor; + channelIsOpen = true; + position(0); + } + + /** + * Constructs an instance of GoogleCloudStorageReadChannel. + * Used for unit testing only. Do not use elsewhere. + * + * @throws IOException on IO error + */ + GoogleCloudStorageReadChannel() + throws IOException { + this.errorExtractor = null; + channelIsOpen = true; + position(0); + } + + /** + * Sets the ClientRequestHelper to be used instead of calling final methods in client requests. + */ + static void setClientRequestHelper(ClientRequestHelper helper) { + clientRequestHelper = helper; + } + + /** + * Sets the Sleeper used for sleeping between retries. + */ + void setSleeper(Sleeper sleeper) { + Preconditions.checkArgument(sleeper != null, "sleeper must not be null!"); + this.sleeper = sleeper; + } + + /** + * Sets the clock to be used for determining when max total time has elapsed doing retries. + */ + void setNanoClock(NanoClock clock) { + Preconditions.checkArgument(clock != null, "clock must not be null!"); + this.clock = clock; + } + + /** + * Sets the backoff for determining sleep duration between retries. + * + * @param backOff May be null to force the next usage to auto-initialize with default settings. + */ + void setBackOff(BackOff backOff) { + this.backOff = backOff; + } + + /** + * Gets the backoff used for determining sleep duration between retries. May be null if it was + * never lazily initialized. + */ + BackOff getBackOff() { + return backOff; + } + + /** + * Helper for initializing the BackOff used for retries. + */ + private BackOff createBackOff() { + return new ExponentialBackOff.Builder() + .setInitialIntervalMillis(DEFAULT_BACKOFF_INITIAL_INTERVAL_MILLIS) + .setRandomizationFactor(DEFAULT_BACKOFF_RANDOMIZATION_FACTOR) + .setMultiplier(DEFAULT_BACKOFF_MULTIPLIER) + .setMaxIntervalMillis(DEFAULT_BACKOFF_MAX_INTERVAL_MILLIS) + .setMaxElapsedTimeMillis(DEFAULT_BACKOFF_MAX_ELAPSED_TIME_MILLIS) + .setNanoClock(clock) + .build(); + } + + /** + * Sets the number of times to automatically retry by re-opening the underlying readChannel + * whenever an exception occurs while reading from it. The count of attempted retries is reset + * whenever at least one byte is successfully read, so this number of retries refers to retries + * made without achieving any forward progress. + */ + public void setMaxRetries(int maxRetries) { + this.maxRetries = maxRetries; + } + + /** + * Reads from this channel and stores read data in the given buffer. + * + * @param buffer buffer to read data into + * @return number of bytes read or -1 on end-of-stream + * @throws java.io.IOException on IO error + */ + @Override + public int read(ByteBuffer buffer) + throws IOException { + throwIfNotOpen(); + + // Don't try to read if the buffer has no space. + if (buffer.remaining() == 0) { + return 0; + } + + // Perform a lazy seek if not done already. + performLazySeek(); + + int totalBytesRead = 0; + int retriesAttempted = 0; + + // We read from a streaming source. We may not get all the bytes we asked for + // in the first read. Therefore, loop till we either read the required number of + // bytes or we reach end-of-stream. + do { + int remainingBeforeRead = buffer.remaining(); + try { + int numBytesRead = readChannel.read(buffer); + Preconditions.checkState(numBytesRead != 0, "Read 0 bytes without blocking!"); + if (numBytesRead < 0) { + break; + } + totalBytesRead += numBytesRead; + currentPosition += numBytesRead; + + // The count of retriesAttempted is per low-level readChannel.read call; each time we make + // progress we reset the retry counter. + retriesAttempted = 0; + } catch (IOException ioe) { + // TODO: Refactor any reusable logic for retries into a separate RetryHelper class. + if (retriesAttempted == maxRetries) { + LOG.warn("Already attempted max of {} retries while reading '{}'; throwing exception.", + maxRetries, StorageResourceId.createReadableString(bucketName, objectName)); + throw ioe; + } else { + if (retriesAttempted == 0) { + // If this is the first of a series of retries, we also want to reset the backOff + // to have fresh initial values. + if (backOff == null) { + backOff = createBackOff(); + } else { + backOff.reset(); + } + } + + ++retriesAttempted; + LOG.warn("Got exception while reading '{}'; retry # {}. Sleeping...", + StorageResourceId.createReadableString(bucketName, objectName), + retriesAttempted, ioe); + + try { + boolean backOffSuccessful = BackOffUtils.next(sleeper, backOff); + if (!backOffSuccessful) { + LOG.warn("BackOff returned false; maximum total elapsed time exhausted. Giving up " + + "after {} retries for '{}'", retriesAttempted, + StorageResourceId.createReadableString(bucketName, objectName)); + throw ioe; + } + } catch (InterruptedException ie) { + LOG.warn("Interrupted while sleeping before retry." + + "Giving up after {} retries for '{}'", retriesAttempted, + StorageResourceId.createReadableString(bucketName, objectName)); + ioe.addSuppressed(ie); + throw ioe; + } + LOG.info("Done sleeping before retry for '{}'; retry # {}.", + StorageResourceId.createReadableString(bucketName, objectName), + retriesAttempted); + + if (buffer.remaining() != remainingBeforeRead) { + int partialRead = remainingBeforeRead - buffer.remaining(); + LOG.info("Despite exception, had partial read of {} bytes; resetting retry count.", + partialRead); + retriesAttempted = 0; + totalBytesRead += partialRead; + currentPosition += partialRead; + } + + // Force the stream to be reopened by seeking to the current position. + long newPosition = currentPosition; + currentPosition = -1; + position(newPosition); + performLazySeek(); + } + } + } while (buffer.remaining() > 0); + + // If this method was called when the stream was already at EOF + // (indicated by totalBytesRead == 0) then return EOF else, + // return the number of bytes read. + return (totalBytesRead == 0) ? -1 : totalBytesRead; + } + + @Override + public int write(ByteBuffer src) throws IOException { + throw new UnsupportedOperationException("Cannot mutate read-only channel"); + } + + /** + * Tells whether this channel is open. + * + * @return a value indicating whether this channel is open + */ + @Override + public boolean isOpen() { + return channelIsOpen; + } + + /** + * Closes this channel. + * + * @throws IOException on IO error + */ + @Override + public void close() + throws IOException { + throwIfNotOpen(); + channelIsOpen = false; + if (readChannel != null) { + readChannel.close(); + } + } + + /** + * Returns this channel's current position. + * + * @return this channel's current position + */ + @Override + public long position() + throws IOException { + throwIfNotOpen(); + return currentPosition; + } + + /** + * Sets this channel's position. + * + * @param newPosition the new position, counting the number of bytes from the beginning. + * @return this channel instance + * @throws java.io.FileNotFoundException if the underlying object does not exist. + * @throws IOException on IO error + */ + @Override + public SeekableByteChannel position(long newPosition) + throws IOException { + throwIfNotOpen(); + + // If the position has not changed, avoid the expensive operation. + if (newPosition == currentPosition) { + return this; + } + + validatePosition(newPosition); + currentPosition = newPosition; + lazySeekPending = true; + return this; + } + + /** + * Returns size of the object to which this channel is connected. + * + * @return size of the object to which this channel is connected + * @throws IOException on IO error + */ + @Override + public long size() + throws IOException { + throwIfNotOpen(); + // Perform a lazy seek if not done already so that size of this channel is set correctly. + performLazySeek(); + return size; + } + + @Override + public SeekableByteChannel truncate(long size) throws IOException { + throw new UnsupportedOperationException("Cannot mutate read-only channel"); + } + + /** + * Sets size of this channel to the given value. + */ + protected void setSize(long size) { + this.size = size; + } + + /** + * Validates that the given position is valid for this channel. + */ + protected void validatePosition(long newPosition) { + // Validate: 0 <= newPosition + if (newPosition < 0) { + throw new IllegalArgumentException( + String.format("Invalid seek offset: position value (%d) must be >= 0", newPosition)); + } + + // Validate: newPosition < size + // Note that we access this.size directly rather than calling size() to avoid initiating + // lazy seek that leads to recursive error. We validate newPosition < size only when size of + // this channel has been computed by a prior call. This means that position could be + // potentially set to an invalid value (>= size) by position(long). However, that error + // gets caught during lazy seek. + if ((size >= 0) && (newPosition >= size)) { + throw new IllegalArgumentException( + String.format( + "Invalid seek offset: position value (%d) must be between 0 and %d", + newPosition, size)); + } + } + + /** + * Seeks to the given position in the underlying stream. + * + * Note: Seek is an expensive operation because a new stream is opened each time. + * + * @throws java.io.FileNotFoundException if the underlying object does not exist. + * @throws IOException on IO error + */ + private void performLazySeek() + throws IOException { + + // Return quickly if there is no pending seek operation. + if (!lazySeekPending) { + return; + } + + // Close the underlying channel if it is open. + if (readChannel != null) { + readChannel.close(); + } + + InputStream objectContentStream = openStreamAndSetSize(currentPosition); + readChannel = Channels.newChannel(objectContentStream); + lazySeekPending = false; + } + + /** + * Opens the underlying stream, sets its position to the given value and sets size based on + * stream content size. + * + * @param newPosition position to seek into the new stream. + * @throws IOException on IO error + */ + protected InputStream openStreamAndSetSize(long newPosition) + throws IOException { + validatePosition(newPosition); + Storage.Objects.Get getObject = gcs.objects().get(bucketName, objectName); + // Set the range on the existing request headers which may have been initialized with things + // like user-agent already. + clientRequestHelper.getRequestHeaders(getObject) + .setRange(String.format("bytes=%d-", newPosition)); + HttpResponse response; + try { + response = getObject.executeMedia(); + } catch (IOException e) { + if (errorExtractor.itemNotFound(e)) { + throw GoogleCloudStorageExceptions + .getFileNotFoundException(bucketName, objectName); + } else if (errorExtractor.rangeNotSatisfiable(e) + && newPosition == 0 + && size == -1) { + // We don't know the size yet (size == -1) and we're seeking to byte 0, but got 'range + // not satisfiable'; the object must be empty. + LOG.info("Got 'range not satisfiable' for reading {} at position 0; assuming empty.", + StorageResourceId.createReadableString(bucketName, objectName)); + size = 0; + return new ByteArrayInputStream(new byte[0]); + } else { + String msg = String.format("Error reading %s at position %d", + StorageResourceId.createReadableString(bucketName, objectName), newPosition); + throw new IOException(msg, e); + } + } + + String contentRange = response.getHeaders().getContentRange(); + if (response.getHeaders().getContentLength() != null) { + size = response.getHeaders().getContentLength() + newPosition; + } else if (contentRange != null) { + String sizeStr = SLASH.split(contentRange)[1]; + try { + size = Long.parseLong(sizeStr); + } catch (NumberFormatException e) { + throw new IOException( + "Could not determine size from response from Content-Range: " + contentRange, e); + } + } else { + throw new IOException("Could not determine size of response"); + } + return response.getContent(); + } + + /** + * Throws if this channel is not currently open. + */ + private void throwIfNotOpen() + throws IOException { + if (!isOpen()) { + throw new ClosedChannelException(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageWriteChannel.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageWriteChannel.java new file mode 100644 index 0000000000000..11113d0367ea5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageWriteChannel.java @@ -0,0 +1,379 @@ +/** + * Copyright 2013 Google Inc. All Rights Reserved. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.gcsio; + +import com.google.api.client.http.HttpHeaders; +import com.google.api.client.http.InputStreamContent; +import com.google.api.client.util.Preconditions; +import com.google.api.services.storage.Storage; +import com.google.api.services.storage.model.StorageObject; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.io.PipedInputStream; +import java.io.PipedOutputStream; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.nio.channels.ClosedChannelException; +import java.nio.channels.WritableByteChannel; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; + +/** + * Implements WritableByteChannel to provide write access to GCS. + */ +public class GoogleCloudStorageWriteChannel + implements WritableByteChannel { + + // The minimum logging interval for upload progress. + private static final long MIN_LOGGING_INTERVAL_MS = 60000L; + + // Logger. + private static final Logger LOG = LoggerFactory.getLogger(GoogleCloudStorageWriteChannel.class); + + // Buffering used in the upload path: + // There are a series of buffers used along the upload path. It is important to understand their + // function before tweaking their values. + // + // Note: Most values are already tweaked based on performance measurements. If you want to change + // buffer sizes, you should change only 1 buffer size at a time to make sure you understand + // the correlation between various buffers and their characteristics. + // + // Upload path: + // Uploading a file involves the following steps: + // -- caller creates a write stream. It involves creating a pipe between data writer (controlled + // by the caller) and data uploader. + // The writer and the uploader are on separate threads. That is, pipe operation is asynchronous + // between its + // two ends. + // -- caller puts data in a ByteBuffer and calls write(ByteBuffer). The write() method starts + // writing into sink end of the pipe. It blocks if pipe buffer is full till the other end + // reads data to make space. + // -- MediaHttpUploader code keeps on reading from the source end of the pipe till it has + // uploadBufferSize amount of data. + // + // The following buffers are involved along the above path: + // -- ByteBuffer passed by caller. We have no control over its size. + // + // -- Pipe buffer. + // size = UPLOAD_PIPE_BUFFER_SIZE_DEFAULT (1 MB) + // Increasing size does not have noticeable difference on performance. + // + // -- Buffer used by Java client + // code. + // size = UPLOAD_CHUNK_SIZE_DEFAULT (64 MB) + + // A pipe that connects write channel used by caller to the input stream used by GCS uploader. + // The uploader reads from input stream which blocks till a caller writes some data to the + // write channel (pipeSinkChannel below). The pipe is formed by connecting pipeSink to pipeSource. + private PipedOutputStream pipeSink; + private PipedInputStream pipeSource; + + // Size of buffer used by upload pipe. + private int pipeBufferSize = UPLOAD_PIPE_BUFFER_SIZE_DEFAULT; + + // A channel wrapper over pipeSink. + private WritableByteChannel pipeSinkChannel; + + // Upload operation that takes place on a separate thread. + private UploadOperation uploadOperation; + + // Default GCS upload granularity. + private static final int GCS_UPLOAD_GRANULARITY = 8 * 1024 * 1024; + + // Upper limit on object size. + // We use less than 250GB limit to avoid potential boundary errors + // in scotty/blobstore stack. + private static final long UPLOAD_MAX_SIZE = 249 * 1024 * 1024 * 1024L; + + // Chunk size to use. Limit the amount of memory used in low memory + // environments such as small AppEngine instances. + private static final int UPLOAD_CHUNK_SIZE_DEFAULT = + Runtime.getRuntime().totalMemory() < 512 * 1024 * 1024 + ? GCS_UPLOAD_GRANULARITY : 8 * GCS_UPLOAD_GRANULARITY; + + // If true, we get very high write throughput but writing files larger than UPLOAD_MAX_SIZE + // will not succeed. Set it to false to allow larger files at lower throughput. + private static boolean limitFileSizeTo250Gb = true; + + // Chunk size to use. + static int uploadBufferSize = UPLOAD_CHUNK_SIZE_DEFAULT; + + // Default size of upload buffer. + public static final int UPLOAD_PIPE_BUFFER_SIZE_DEFAULT = 1 * 1024 * 1024; + + // ClientRequestHelper to be used instead of calling final methods in client requests. + private static ClientRequestHelper clientRequestHelper = new ClientRequestHelper(); + + /** + * Allows running upload operation on a background thread. + */ + static class UploadOperation + implements Runnable { + + // Object to be uploaded. This object declared final for safe object publishing. + private final Storage.Objects.Insert insertObject; + + // Exception encountered during upload. + Throwable exception; + + // Allows other threads to wait for this operation to be complete. This object declared final + // for safe object publishing. + final CountDownLatch uploadDone = new CountDownLatch(1); + + // Read end of the pipe. This object declared final for safe object publishing. + private final InputStream pipeSource; + + /** + * Constructs an instance of UploadOperation. + * + * @param insertObject object to be uploaded + */ + public UploadOperation(Storage.Objects.Insert insertObject, InputStream pipeSource) { + this.insertObject = insertObject; + this.pipeSource = pipeSource; + } + + /** + * Gets exception/error encountered during upload or null. + */ + public Throwable exception() { + return exception; + } + + /** + * Runs the upload operation. + */ + @Override + public void run() { + try { + insertObject.execute(); + } catch (Throwable t) { + exception = t; + LOG.error("Upload failure", t); + } finally { + uploadDone.countDown(); + try { + // Close this end of the pipe so that the writer at the other end + // will not hang indefinitely. + pipeSource.close(); + } catch (IOException ioe) { + LOG.error("Error trying to close pipe.source()", ioe); + // Log and ignore IOException while trying to close the channel, + // as there is not much we can do about it. + } + } + } + + public void waitForCompletion() { + do { + try { + uploadDone.await(); + } catch (InterruptedException e) { + // Ignore it and continue to wait. + } + } while(uploadDone.getCount() > 0); + } + } + + /** + * Constructs an instance of GoogleCloudStorageWriteChannel. + * + * @param threadPool thread pool to use for running the upload operation + * @param gcs storage object instance + * @param bucketName name of the bucket to create object in + * @param objectName name of the object to create + * @throws IOException on IO error + */ + public GoogleCloudStorageWriteChannel( + ExecutorService threadPool, Storage gcs, String bucketName, + String objectName, String contentType) + throws IOException { + init(threadPool, gcs, bucketName, objectName, contentType); + } + + /** + * Sets the ClientRequestHelper to be used instead of calling final methods in client requests. + */ + static void setClientRequestHelper(ClientRequestHelper helper) { + clientRequestHelper = helper; + } + + /** + * Writes contents of the given buffer to this channel. + * + * Note: The data that one writes gets written to a pipe which may not block + * if the pipe has sufficient buffer space. A success code returned from this method + * does not mean that the specific data was successfully written to the underlying + * storage. It simply means that there is no error at present. The data upload + * may encounter an error on a separate thread. Such error is not ignored; + * it shows up as an exception during a subsequent call to write() or close(). + * The only way to be sure of successful upload is when the close() method + * returns successfully. + * + * @param buffer buffer to write + * @throws IOException on IO error + */ + @Override + public int write(ByteBuffer buffer) + throws IOException { + throwIfNotOpen(); + + // No point in writing further if upload failed on another thread. + throwIfUploadFailed(); + + return pipeSinkChannel.write(buffer); + } + + /** + * Tells whether this channel is open. + * + * @return a value indicating whether this channel is open + */ + @Override + public boolean isOpen() { + return (pipeSinkChannel != null) && pipeSinkChannel.isOpen(); + } + + /** + * Closes this channel. + * + * Note: + * The method returns only after all data has been successfully written to GCS + * or if there is a non-retry-able error. + * + * @throws IOException on IO error + */ + @Override + public void close() + throws IOException { + throwIfNotOpen(); + try { + pipeSinkChannel.close(); + uploadOperation.waitForCompletion(); + throwIfUploadFailed(); + } finally { + pipeSinkChannel = null; + pipeSink = null; + pipeSource = null; + uploadOperation = null; + } + } + + /** + * Sets size of upload buffer used. + */ + public static void setUploadBufferSize(int bufferSize) { + Preconditions.checkArgument(bufferSize > 0, + "Upload buffer size must be great than 0."); + if (bufferSize % GCS_UPLOAD_GRANULARITY != 0) { + LOG.warn("Upload buffer size should be a multiple of {} for best performance, got {}", + GCS_UPLOAD_GRANULARITY, bufferSize); + } + GoogleCloudStorageWriteChannel.uploadBufferSize = bufferSize; + } + + /** + * Enables or disables hard limit of 250GB on size of uploaded files. + * + * If enabled, we get very high write throughput but writing files larger than UPLOAD_MAX_SIZE + * will not succeed. Set it to false to allow larger files at lower throughput. + */ + public static void enableFileSizeLimit250Gb(boolean enableLimit) { + GoogleCloudStorageWriteChannel.limitFileSizeTo250Gb = enableLimit; + } + + /** + * Initializes an instance of GoogleCloudStorageWriteChannel. + * + * @param threadPool thread pool to use for running the upload operation + * @param gcs storage object instance + * @param bucketName name of the bucket in which to create object + * @param objectName name of the object to create + * @throws IOException on IO error + */ + private void init( + ExecutorService threadPool, Storage gcs, String bucketName, + String objectName, String contentType) + throws IOException { + + // Create object with the given name. + StorageObject object = (new StorageObject()).setName(objectName); + + // Create a pipe such that its one end is connected to the input stream used by + // the uploader and the other end is the write channel used by the caller. + pipeSource = new PipedInputStream(pipeBufferSize); + pipeSink = new PipedOutputStream(pipeSource); + pipeSinkChannel = Channels.newChannel(pipeSink); + + // Connect pipe-source to the stream used by uploader. + InputStreamContent objectContentStream = + new InputStreamContent(contentType, pipeSource); + // Indicate that we do not know length of file in advance. + objectContentStream.setLength(-1); + objectContentStream.setCloseInputStream(false); + Storage.Objects.Insert insertObject = + gcs.objects().insert(bucketName, object, objectContentStream); + insertObject.setDisableGZipContent(true); + insertObject.getMediaHttpUploader().setProgressListener( + new LoggingMediaHttpUploaderProgressListener(objectName, MIN_LOGGING_INTERVAL_MS)); + + // Insert necessary http headers to enable 250GB limit+high throughput if so configured. + if (limitFileSizeTo250Gb) { + HttpHeaders headers = clientRequestHelper.getRequestHeaders(insertObject); + headers.set("X-Goog-Upload-Desired-Chunk-Granularity", GCS_UPLOAD_GRANULARITY); + headers.set("X-Goog-Upload-Max-Raw-Size", UPLOAD_MAX_SIZE); + } + // Change chunk size from default value (10MB) to one that yields higher performance. + clientRequestHelper.setChunkSize(insertObject, uploadBufferSize); + + // Given that the two ends of the pipe must operate asynchronous relative + // to each other, we need to start the upload operation on a separate thread. + uploadOperation = new UploadOperation(insertObject, pipeSource); + threadPool.execute(uploadOperation); + } + + /** + * Throws if this channel is not currently open. + * + * @throws IOException on IO error + */ + private void throwIfNotOpen() + throws IOException { + if (!isOpen()) { + throw new ClosedChannelException(); + } + } + + /** + * Throws if upload operation failed. Propagates any errors. + * + * @throws IOException on IO error + */ + private void throwIfUploadFailed() + throws IOException { + if ((uploadOperation != null) && (uploadOperation.exception() != null)) { + if (uploadOperation.exception() instanceof Error) { + throw (Error) uploadOperation.exception(); + } + throw new IOException(uploadOperation.exception()); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListener.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListener.java new file mode 100644 index 0000000000000..c215f4aeafafc --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListener.java @@ -0,0 +1,91 @@ +/** + * Copyright 2013 Google Inc. All Rights Reserved. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.gcsio; + +import com.google.api.client.googleapis.media.MediaHttpUploader; +import com.google.api.client.googleapis.media.MediaHttpUploader.UploadState; +import com.google.api.client.googleapis.media.MediaHttpUploaderProgressListener; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * Logs the status of uploads. At the beginning, during, and + * at the end of the upload, emits relevant statistics such as how many bytes + * uploaded and the rate at which the upload is progressing. + *

+ * A new instance of this progress listener should be used for each MediaHttpUploader. + */ +class LoggingMediaHttpUploaderProgressListener implements MediaHttpUploaderProgressListener { + private static final Logger LOG = + LoggerFactory.getLogger(MediaHttpUploaderProgressListener.class); + private static final double BYTES_IN_MB = 1048576.0; + private final long minLoggingInterval; + private final String name; + private long startTime; + private long prevTime; + private long prevUploadedBytes; + + /** + * Creates a upload progress listener which emits relevant statistics about the + * progress of the upload. + * @param name The name of the resource being uploaded. + * @param minLoggingInterval The minimum amount of time (millis) between logging upload progress. + */ + LoggingMediaHttpUploaderProgressListener(String name, long minLoggingInterval) { + this.name = name; + this.minLoggingInterval = minLoggingInterval; + } + + @Override + public void progressChanged(MediaHttpUploader uploader) throws IOException { + progressChanged(LOG, + uploader.getUploadState(), + uploader.getNumBytesUploaded(), + System.currentTimeMillis()); + } + + void progressChanged(Logger log, UploadState uploadState, long bytesUploaded, long currentTime) { + switch (uploadState) { + case INITIATION_STARTED: + startTime = currentTime; + prevTime = currentTime; + log.info("Uploading: {}", name); + break; + case MEDIA_IN_PROGRESS: + // Limit messages to be emitted for in progress uploads. + if (currentTime > prevTime + minLoggingInterval) { + double averageRate = (bytesUploaded / BYTES_IN_MB) + / ((currentTime - startTime) / 1000.0); + double currentRate = ((bytesUploaded - prevUploadedBytes) / BYTES_IN_MB) + / ((currentTime - prevTime) / 1000.0); + log.info(String.format( + "Uploading: %s Average Rate: %.3f MiB/s, Current Rate: %.3f MiB/s, Total: %.3f MiB", + name, averageRate, currentRate, bytesUploaded / BYTES_IN_MB)); + prevTime = currentTime; + prevUploadedBytes = bytesUploaded; + } + break; + case MEDIA_COMPLETE: + log.info("Finished Uploading: {}", name); + break; + default: + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/StorageResourceId.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/StorageResourceId.java new file mode 100644 index 0000000000000..b6051a5147d3e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/StorageResourceId.java @@ -0,0 +1,165 @@ +/** + * Copyright 2013 Google Inc. All Rights Reserved. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.gcsio; + +import com.google.api.client.util.Preconditions; +import com.google.common.base.Strings; + +import java.util.Objects; + +/** + * Data struct representing either a GCS StorageObject, a GCS Bucket or the GCS root (gs://). + * If both bucketName and objectName are null, the StorageResourceId refers to GCS root (gs://). + * If bucketName is non-null, and objectName is null, then this refers to a GCS Bucket. Otherwise, + * if bucketName and objectName are both non-null, this refers to a GCS StorageObject. + */ +public class StorageResourceId { + // The singleton instance identifying the GCS root (gs://). Both getObjectName() and + // getBucketName() will return null. + public static final StorageResourceId ROOT = new StorageResourceId(); + + // Bucket name of this storage resource to be used with the Google Cloud Storage API. + private final String bucketName; + + // Object name of this storage resource to be used with the Google Cloud Storage API. + private final String objectName; + + // Human-readable String to be returned by toString(); kept as 'final' member for efficiency. + private final String readableString; + + /** + * Constructor for a StorageResourceId which refers to the GCS root (gs://). Private because + * all external users should just use the singleton StorageResourceId.ROOT. + */ + private StorageResourceId() { + this.bucketName = null; + this.objectName = null; + this.readableString = createReadableString(bucketName, objectName); + } + + /** + * Constructor for a StorageResourceId representing a Bucket; {@code getObjectName()} will return + * null for a StorageResourceId which represents a Bucket. + * + * @param bucketName The bucket name of the resource. Must be non-empty and non-null. + */ + public StorageResourceId(String bucketName) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(bucketName), + "bucketName must not be null or empty"); + + this.bucketName = bucketName; + this.objectName = null; + this.readableString = createReadableString(bucketName, objectName); + } + + /** + * Constructor for a StorageResourceId representing a full StorageObject, including bucketName + * and objectName. + * + * @param bucketName The bucket name of the resource. Must be non-empty and non-null. + * @param objectName The object name of the resource. Must be non-empty and non-null. + */ + public StorageResourceId(String bucketName, String objectName) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(bucketName), + "bucketName must not be null or empty"); + Preconditions.checkArgument(!Strings.isNullOrEmpty(objectName), + "objectName must not be null or empty"); + + this.bucketName = bucketName; + this.objectName = objectName; + this.readableString = createReadableString(bucketName, objectName); + } + + /** + * Returns true if this StorageResourceId represents a GCS StorageObject; if true, both + * {@code getBucketName} and {@code getObjectName} will be non-empty and non-null. + */ + public boolean isStorageObject() { + return bucketName != null && objectName != null; + } + + /** + * Returns true if this StorageResourceId represents a GCS Bucket; if true, then {@code + * getObjectName} will return null. + */ + public boolean isBucket() { + return bucketName != null && objectName == null; + } + + /** + * Returns true if this StorageResourceId represents the GCS root (gs://); if true, then + * both {@code getBucketName} and {@code getObjectName} will be null. + */ + public boolean isRoot() { + return bucketName == null && objectName == null; + } + + /** + * Gets the bucket name component of this resource identifier. + */ + public String getBucketName() { + return bucketName; + } + + /** + * Gets the object name component of this resource identifier. + */ + public String getObjectName() { + return objectName; + } + + /** + * Returns a string of the form gs:///. + */ + @Override + public String toString() { + return readableString; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof StorageResourceId) { + StorageResourceId other = (StorageResourceId) obj; + return Objects.equals(bucketName, other.bucketName) + && Objects.equals(objectName, other.objectName); + } + return false; + } + + @Override + public int hashCode() { + return readableString.hashCode(); + } + + /** + * Helper for standardizing the way various human-readable messages in logs/exceptions which refer + * to a bucket/object pair. + */ + public static String createReadableString(String bucketName, String objectName) { + if (bucketName == null && objectName == null) { + // TODO: Unify this method with other methods which convert bucketName/objectName + // to a URI; maybe use the single slash for compatibility. + return "gs://"; + } else if (bucketName != null && objectName == null) { + return String.format("gs://%s", bucketName); + } else if (bucketName != null && objectName != null) { + return String.format("gs://%s/%s", bucketName, objectName); + } + throw new IllegalArgumentException( + String.format("Invalid bucketName/objectName pair: gs://%s/%s", bucketName, objectName)); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/package-info.java new file mode 100644 index 0000000000000..98fdc44113a34 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** Defines utilities used by the Dataflow SDK. **/ +package com.google.cloud.dataflow.sdk.util; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTag.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTag.java new file mode 100644 index 0000000000000..3caed1a8bcce9 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTag.java @@ -0,0 +1,72 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import com.google.cloud.dataflow.sdk.coders.Coder; + +/** + * A {@link TupleTag} combined with the {@link Coder} to use for + * values associated with the tag. + * + *

Used as tags in + * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.KeyedState}. + * + * @param the type of the values associated with this tag + */ +public class CodedTupleTag extends TupleTag { + /** + * Returns a {@code CodedTupleTag} with the given id which uses the + * given {@code Coder} whenever a value associated with the tag + * needs to be serialized. + * + *

It is up to the user to ensure that two + * {@code CodedTupleTag}s with the same id actually mean the same + * tag and carry the same generic type parameter. Violating this + * invariant can lead to hard-to-diagnose runtime type errors. + * + *

(An explicit id is required so that persistent keyed state + * saved by one run of a streaming program can be reused if that + * streaming program is upgraded to a new version.) + * + * @param the type of the values associated with the tag + */ + public static CodedTupleTag of(String id, Coder coder) { + return new CodedTupleTag(id, coder); + } + + /** + * Returns the {@code Coder} used for values associated with this tag. + */ + public Coder getCoder() { + return coder; + } + + + /////////////////////////////////////////////// + + private final Coder coder; + + CodedTupleTag(String id, Coder coder) { + super(id); + this.coder = coder; + } + + @Override + public String toString() { + return "CodedTupleTag<" + getId() + ", " + coder + ">"; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTagMap.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTagMap.java new file mode 100644 index 0000000000000..6f96c694ea2e6 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTagMap.java @@ -0,0 +1,59 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import java.util.Map; + +/** + * A mapping of {@link CodedTupleTag}s to associated values. + * + *

Returned by + * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.KeyedState#lookup(java.util.List)}. + */ +public class CodedTupleTagMap { + /** + * Returns a {@code CodedTupleTagMap} containing the given mappings. + * + *

It is up to the caller to ensure that the value associated + * with each CodedTupleTag in the map has the static type specified + * by that tag. + * + *

Intended for internal use only. + */ + public static CodedTupleTagMap of(Map, Object> map) { + // TODO: Should we copy the Map here, to insulate this + // map from any changes to the original argument? + return new CodedTupleTagMap(map); + } + + /** + * Returns the value associated with the given tag in this + * {@code CodedTupleTagMap}, or {@code null} if the tag has no + * asssociated value. + */ + public T get(CodedTupleTag tag) { + return (T) map.get(tag); + } + + ////////////////////////////////////////////// + + private Map, Object> map; + + CodedTupleTagMap(Map, Object> map) { + this.map = map; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java new file mode 100644 index 0000000000000..d354707ebb0c0 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java @@ -0,0 +1,117 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import java.io.Serializable; +import java.util.Comparator; + +/** + * An immutable key/value pair. + * + *

Various + * {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s like + * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} and + * {@link com.google.cloud.dataflow.sdk.transforms.Combine#perKey} + * work on {@link PCollection}s of KVs. + * + * @param the type of the key + * @param the type of the value + */ +public class KV implements Serializable { + /** Returns a KV with the given key and value. */ + public static KV of(K key, V value) { + return new KV<>(key, value); + } + + /** Returns the key of this KV. */ + public K getKey() { + return key; + } + + /** Returns the value of this KV. */ + public V getValue() { + return value; + } + + + ///////////////////////////////////////////////////////////////////////////// + + final K key; + final V value; + + private KV(K key, V value) { + this.key = key; + this.value = value; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o instanceof KV) { + KV that = (KV) o; + return (this.key == null ? that.key == null + : this.key.equals(that.key)) + && (this.value == null ? that.value == null + : this.value.equals(that.value)); + } + return false; + } + + /** Orders the KV by the key. A null key is less than any non-null key. */ + public static class OrderByKey, V> implements + Comparator>, Serializable { + @Override + public int compare(KV a, KV b) { + if (a.key == null) { + return b.key == null ? 0 : -1; + } else if (b.key == null) { + return 1; + } else { + return a.key.compareTo(b.key); + } + } + } + + /** Orders the KV by the value. A null value is less than any non-null value. */ + public static class OrderByValue> + implements Comparator>, Serializable { + @Override + public int compare(KV a, KV b) { + if (a.value == null) { + return b.value == null ? 0 : -1; + } else if (b.value == null) { + return 1; + } else { + return a.value.compareTo(b.value); + } + } + } + + @Override + public int hashCode() { + return getClass().hashCode() + + (key == null ? 0 : key.hashCode()) + + (value == null ? 0 : value.hashCode()); + } + + @Override + public String toString() { + return "KV(" + key + ", " + value + ")"; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PBegin.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PBegin.java new file mode 100644 index 0000000000000..fc3f179fc1765 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PBegin.java @@ -0,0 +1,77 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.transforms.PTransform; + +import java.util.Collection; +import java.util.Collections; + +/** + * {@code PBegin} is used as the "input" to a root + * {@link com.google.cloud.dataflow.sdk.transforms.PTransform} which + * is the first operation in a {@link Pipeline}, such as + * {@link com.google.cloud.dataflow.sdk.io.TextIO.Read} or + * {@link com.google.cloud.dataflow.sdk.transforms.Create}. + * + *

Typically created by calling {@link Pipeline#begin} on a Pipeline. + */ +public class PBegin implements PInput { + /** + * Returns a {@code PBegin} in the given {@code Pipeline}. + */ + public static PBegin in(Pipeline pipeline) { + return new PBegin(pipeline); + } + + /** + * Applies the given PTransform to this input PBegin, and + * returns the PTransform's Output. + */ + public Output apply( + PTransform t) { + return Pipeline.applyTransform(this, t); + } + + @Override + public Pipeline getPipeline() { + return pipeline; + } + + @Override + public Collection expand() { + // A PBegin contains no PValues. + return Collections.emptyList(); + } + + @Override + public void finishSpecifying() { + // Nothing more to be done. + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Constructs a {@code PBegin} in the given {@code Pipeline}. + */ + protected PBegin(Pipeline pipeline) { + this.pipeline = pipeline; + } + + private Pipeline pipeline; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java new file mode 100644 index 0000000000000..fc4b0886b7d5b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java @@ -0,0 +1,240 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.common.reflect.TypeToken; + +/** + * A {@code PCollection} is an immutable collection of values of type + * {@code T}. A {@code PCollection} can contain either a bounded or unbounded + * number of elements. Bounded and unbounded {@code PCollection}s are produced + * as the output of {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s + * (including root PTransforms like + * {@link com.google.cloud.dataflow.sdk.io.TextIO.Read}, + * {@link com.google.cloud.dataflow.sdk.io.PubsubIO.Read} and + * {@link com.google.cloud.dataflow.sdk.transforms.Create}), and can + * be passed as the inputs of other PTransforms. + * + *

Some root transforms produce bounded {@code PCollections} and others + * produce unbounded ones. For example, + * {@link com.google.cloud.dataflow.sdk.io.TextIO.Read} reads a static set + * of files, so it produces a bounded {@code PCollection}. + * {@link com.google.cloud.dataflow.sdk.io.PubsubIO.Read}, on the other hand, + * receives a potentially infinite stream of Pubsub messages, so it produces + * an unbounded {@code PCollection}. + * + *

Each element in a {@code PCollection} may have an associated implicit + * timestamp. Sources assign timestamps to elements when they create + * {@code PCollection}s, and other {@code PTransform}s propagate these + * timestamps from their input to their output. For example, PubsubIO.Read + * assigns pubsub message timestamps to elements, and TextIO.Read assigns + * the default value {@code Long.MIN_VALUE} to elements. User code can + * explicitly assign timestamps to elements with + * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#outputWithTimestamp}. + * + *

Additionally, a {@code PCollection} has an associated + * {@link WindowingFn} and each element is assigned to a set of windows. + * By default, the windowing function is + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow} + * and all elements are assigned into a single default window. + * This default can be overridden with the + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} + * {@code PTransform}. Dataflow pipelines run in classic batch MapReduce style + * with the default GlobalWindow strategy if timestamps are ignored. + * + *

See the individual {@code PTransform} subclasses for specific information + * on how they propagate timestamps and windowing. + * + * @param the type of the elements of this PCollection + */ +public class PCollection extends TypedPValue { + /** + * Returns the name of this PCollection. + * + *

By default, the name of a PCollection is based on the name of the + * PTransform that produces it. It can be specified explicitly by + * calling {@link #setName}. + * + * @throws IllegalStateException if the name hasn't been set yet + */ + @Override + public String getName() { + return super.getName(); + } + + /** + * Sets the name of this PCollection. Returns {@code this}. + * + * @throws IllegalStateException if this PCollection has already been + * finalized and is no longer settable, e.g., by having + * {@code apply()} called on it + */ + @Override + public PCollection setName(String name) { + super.setName(name); + return this; + } + + /** + * Returns the Coder used by this PCollection to encode and decode + * the values stored in it. + * + * @throws IllegalStateException if the Coder hasn't been set, and + * couldn't be inferred + */ + @Override + public Coder getCoder() { + return super.getCoder(); + } + + /** + * Sets the Coder used by this PCollection to encode and decode the + * values stored in it. Returns {@code this}. + * + * @throws IllegalStateException if this PCollection has already + * been finalized and is no longer settable, e.g., by having + * {@code apply()} called on it + */ + @Override + public PCollection setCoder(Coder coder) { + super.setCoder(coder); + return this; + } + + /** + * Returns whether or not the elements of this PCollection have a + * well-defined and fixed order, such that subsequent reading of the + * PCollection is guaranteed to process the elements in order. + * + *

Requiring a fixed order can limit optimization opportunities. + * + *

By default, PCollections do not have a well-defined or fixed order. + */ + public boolean isOrdered() { + return isOrdered; + } + + /** + * Sets whether or not this PCollection should preserve the order in + * which elements are put in it, such that subsequent parallel + * reading of the PCollection is guaranteed to process the elements + * in order. + * + *

Requiring a fixed order can limit optimization opportunities. + * + *

Returns {@code this}. + * + * @throws IllegalStateException if this PCollection has already + * been finalized and is no longer settable, e.g., by having + * {@code apply()} called on it + */ + public PCollection setOrdered(boolean isOrdered) { + if (this.isOrdered != isOrdered) { + if (isFinishedSpecifyingInternal()) { + throw new IllegalStateException( + "cannot change the orderedness of " + this + + " once it's been used"); + } + this.isOrdered = isOrdered; + } + return this; + } + + /** + * Applies the given PTransform to this input PCollection, and + * returns the PTransform's Output. + */ + public Output apply( + PTransform, Output> t) { + return Pipeline.applyTransform(this, t); + } + + /** + * Returns the {@link WindowingFn} of this {@code PCollection}. + */ + public WindowingFn getWindowingFn() { + return windowingFn; + } + + ///////////////////////////////////////////////////////////////////////////// + // Internal details below here. + + /** + * Whether or not the elements of this PCollection have a + * well-defined and fixed order, such that subsequent reading of the + * PCollection is guaranteed to process the elements in order. + */ + private boolean isOrdered = false; + + /** + * {@link WindowingFn} that will be used to merge windows in + * this {@code PCollection} and subsequent {@code PCollection}s produced + * from this one. + * + *

By default, no merging is performed. + */ + private WindowingFn windowingFn; + + private PCollection() {} + + /** + * Sets the {@code TypeToken} for this {@code PCollection}, so that + * the enclosing {@code PCollectionTuple}, {@code PCollectionList}, + * or {@code PTransform>}, etc., can provide + * more detailed reflective information. + */ + @Override + public PCollection setTypeTokenInternal(TypeToken typeToken) { + super.setTypeTokenInternal(typeToken); + return this; + } + + /** + * Sets the {@link WindowingFn} of this {@code PCollection}. + * + *

For use by primitive transformations only. + */ + public PCollection setWindowingFnInternal(WindowingFn windowingFn) { + this.windowingFn = windowingFn; + return this; + } + + /** + * Sets the {@link Pipeline} for this {@code PCollection}. + * + *

For use by primitive transformations only. + */ + @Override + public PCollection setPipelineInternal(Pipeline pipeline) { + super.setPipelineInternal(pipeline); + return this; + } + + /** + * Creates and returns a new PCollection for a primitive output. + * + *

For use by primitive transformations only. + */ + public static PCollection createPrimitiveOutputInternal( + WindowingFn windowingFn) { + return new PCollection().setWindowingFnInternal(windowingFn); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionList.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionList.java new file mode 100644 index 0000000000000..26b7300a9341a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionList.java @@ -0,0 +1,227 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + +/** + * A {@code PCollectionList} is an immutable list of homogeneously + * typed {@code PCollection}s. A PCollectionList is used, for + * instance, as the input to + * {@link com.google.cloud.dataflow.sdk.transforms.Flatten} or the + * output of + * {@link com.google.cloud.dataflow.sdk.transforms.Partition}. + * + *

PCollectionLists can be created and accessed like follows: + *

 {@code
+ * PCollection pc1 = ...;
+ * PCollection pc2 = ...;
+ * PCollection pc3 = ...;
+ *
+ * // Create a PCollectionList with three PCollections:
+ * PCollectionList pcs = PCollectionList.of(pc1).and(pc2).and(pc3);
+ *
+ * // Create an empty PCollectionList:
+ * Pipeline p = ...;
+ * PCollectionList pcs2 = PCollectionList.empty(p);
+ *
+ * // Get PCollections out of a PCollectionList, by index (origin 0):
+ * PCollection pcX = pcs.get(1);
+ * PCollection pcY = pcs.get(0);
+ * PCollection pcZ = pcs.get(2);
+ *
+ * // Get a list of all PCollections in a PCollectionList:
+ * List> allPcs = pcs.getAll();
+ * } 
+ * + * @param the type of the elements of all the PCollections in this list + */ +public class PCollectionList implements PInput, POutput { + /** + * Returns an empty PCollectionList that is part of the given Pipeline. + * + *

Longer PCollectionLists can be created by calling + * {@link #and} on the result. + */ + public static PCollectionList empty(Pipeline pipeline) { + return new PCollectionList<>(pipeline); + } + + /** + * Returns a singleton PCollectionList containing the given PCollection. + * + *

Longer PCollectionLists can be created by calling + * {@link #and} on the result. + */ + public static PCollectionList of(PCollection pc) { + return new PCollectionList(pc.getPipeline()).and(pc); + } + + /** + * Returns a PCollectionList containing the given PCollections, in order. + * + *

The argument list cannot be empty. + * + *

All the PCollections in the resulting PCollectionList must be + * part of the same Pipeline. + * + *

Longer PCollectionLists can be created by calling + * {@link #and} on the result. + */ + public static PCollectionList of(Iterable> pcs) { + Iterator> pcsIter = pcs.iterator(); + if (!pcsIter.hasNext()) { + throw new IllegalArgumentException( + "must either have a non-empty list of PCollections, " + + "or must first call empty(Pipeline)"); + } + return new PCollectionList(pcsIter.next().getPipeline()).and(pcs); + } + + /** + * Returns a new PCollectionList that has all the PCollections of + * this PCollectionList plus the given PCollection appended to the end. + * + *

All the PCollections in the resulting PCollectionList must be + * part of the same Pipeline. + */ + public PCollectionList and(PCollection pc) { + if (pc.getPipeline() != pipeline) { + throw new IllegalArgumentException( + "PCollections come from different Pipelines"); + } + return new PCollectionList<>(pipeline, + new ImmutableList.Builder>() + .addAll(pcollections) + .add(pc) + .build()); + } + + /** + * Returns a new PCollectionList that has all the PCollections of + * this PCollectionList plus the given PCollections appended to the end, + * in order. + * + *

All the PCollections in the resulting PCollectionList must be + * part of the same Pipeline. + */ + public PCollectionList and(Iterable> pcs) { + List> copy = new ArrayList<>(pcollections); + for (PCollection pc : pcs) { + if (pc.getPipeline() != pipeline) { + throw new IllegalArgumentException( + "PCollections come from different Pipelines"); + } + copy.add(pc); + } + return new PCollectionList<>(pipeline, copy); + } + + /** + * Returns the number of PCollections in this PCollectionList. + */ + public int size() { + return pcollections.size(); + } + + /** + * Returns the PCollection at the given index (origin zero). Throws + * IndexOutOfBounds if the index is out of the range + * {@code [0..size()-1]}. + */ + public PCollection get(int index) { + return pcollections.get(index); + } + + /** + * Returns an immutable List of all the PCollections in this PCollectionList. + */ + public List> getAll() { + return pcollections; + } + + /** + * Applies the given PTransform to this input {@code PCollectionList}, + * and returns the PTransform's Output. + */ + public Output apply( + PTransform, Output> t) { + return Pipeline.applyTransform(this, t); + } + + + ///////////////////////////////////////////////////////////////////////////// + // Internal details below here. + + final Pipeline pipeline; + final List> pcollections; + + PCollectionList(Pipeline pipeline) { + this(pipeline, new ArrayList>()); + } + + PCollectionList(Pipeline pipeline, List> pcollections) { + this.pipeline = pipeline; + this.pcollections = Collections.unmodifiableList(pcollections); + } + + @Override + public Pipeline getPipeline() { + return pipeline; + } + + @Override + public Collection expand() { + return pcollections; + } + + @Override + public void recordAsOutput(Pipeline pipeline, + PTransform transform) { + if (this.pipeline != null && this.pipeline != pipeline) { + throw new AssertionError( + "not expecting to change the Pipeline owning a PCollectionList"); + } + int i = 0; + for (PCollection pc : pcollections) { + pc.recordAsOutput(pipeline, transform, "out" + i); + i++; + } + } + + @Override + public void finishSpecifying() { + for (PCollection pc : pcollections) { + pc.finishSpecifying(); + } + } + + @Override + public void finishSpecifyingOutput() { + for (PCollection pc : pcollections) { + pc.finishSpecifyingOutput(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java new file mode 100644 index 0000000000000..fecc175f4d3cc --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java @@ -0,0 +1,252 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.common.collect.ImmutableMap; +import com.google.common.reflect.TypeToken; + +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * A {@code PCollectionTuple} is an immutable tuple of + * heterogeneously-typed {@link PCollection}s, "keyed" by + * {@link TupleTag}s. A PCollectionTuple can be used as the input or + * output of a + * {@link com.google.cloud.dataflow.sdk.transforms.PTransform} taking + * or producing multiple PCollection inputs or outputs that can be of + * different types, for instance a + * {@link com.google.cloud.dataflow.sdk.transforms.ParDo} with side + * outputs. + * + *

PCollectionTuples can be created and accessed like follows: + *

 {@code
+ * PCollection pc1 = ...;
+ * PCollection pc2 = ...;
+ * PCollection> pc3 = ...;
+ *
+ * // Create TupleTags for each of the PCollections to put in the
+ * // PCollectionTuple (the type of the TupleTag enables tracking the
+ * // static type of each of the PCollections in the PCollectionTuple):
+ * TupleTag tag1 = new TupleTag<>();
+ * TupleTag tag2 = new TupleTag<>();
+ * TupleTag> tag3 = new TupleTag<>();
+ *
+ * // Create a PCollectionTuple with three PCollections:
+ * PCollectionTuple pcs =
+ *     PCollectionTuple.of(tag1, pc1)
+ *                     .and(tag2, pc2)
+ *                     .and(tag3, pc3);
+ *
+ * // Create an empty PCollectionTuple:
+ * Pipeline p = ...;
+ * PCollectionTuple pcs2 = PCollectionTuple.empty(p);
+ *
+ * // Get PCollections out of a PCollectionTuple, using the same tags
+ * // that were used to put them in:
+ * PCollection pcX = pcs.get(tag2);
+ * PCollection pcY = pcs.get(tag1);
+ * PCollection> pcZ = pcs.get(tag3);
+ *
+ * // Get a map of all PCollections in a PCollectionTuple:
+ * Map, PCollection> allPcs = pcs.getAll();
+ * } 
+ */ +public class PCollectionTuple implements PInput, POutput { + /** + * Returns an empty PCollectionTuple that is part of the given Pipeline. + * + *

Longer PCollectionTuples can be created by calling + * {@link #and} on the result. + */ + public static PCollectionTuple empty(Pipeline pipeline) { + return new PCollectionTuple(pipeline); + } + + /** + * Returns a singleton PCollectionTuple containing the given + * PCollection keyed by the given TupleTag. + * + *

Longer PCollectionTuples can be created by calling + * {@link #and} on the result. + */ + public static PCollectionTuple of(TupleTag tag, PCollection pc) { + return empty(pc.getPipeline()).and(tag, pc); + } + + /** + * Returns a new PCollectionTuple that has all the PCollections and + * tags of this PCollectionTuple plus the given PCollection and tag. + * + *

The given TupleTag should not already be mapped to a + * PCollection in this PCollectionTuple. + * + *

All the PCollections in the resulting PCollectionTuple must be + * part of the same Pipeline. + */ + public PCollectionTuple and(TupleTag tag, PCollection pc) { + if (pc.getPipeline() != pipeline) { + throw new IllegalArgumentException( + "PCollections come from different Pipelines"); + } + + // The TypeToken in tag will often have good + // reflective information about T + pc.setTypeTokenInternal(tag.getTypeToken()); + return new PCollectionTuple(pipeline, + new ImmutableMap.Builder, PCollection>() + .putAll(pcollectionMap) + .put(tag, pc) + .build()); + } + + /** + * Returns whether this PCollectionTuple contains a PCollection with + * the given tag. + */ + public boolean has(TupleTag tag) { + return pcollectionMap.containsKey(tag); + } + + /** + * Returns the PCollection with the given tag in this + * PCollectionTuple. Throws IllegalArgumentException if there is no + * such PCollection, i.e., {@code !has(tag)}. + */ + public PCollection get(TupleTag tag) { + @SuppressWarnings("unchecked") + PCollection pcollection = (PCollection) pcollectionMap.get(tag); + if (pcollection == null) { + throw new IllegalArgumentException( + "TupleTag not found in this PCollectionTuple tuple"); + } + return pcollection; + } + + /** + * Returns an immutable Map from TupleTag to corresponding + * PCollection, for all the members of this PCollectionTuple. + */ + public Map, PCollection> getAll() { + return pcollectionMap; + } + + /** + * Applies the given PTransform to this input PCollectionTuple, and + * returns the PTransform's Output. + */ + public Output apply( + PTransform t) { + return Pipeline.applyTransform(this, t); + } + + + ///////////////////////////////////////////////////////////////////////////// + // Internal details below here. + + Pipeline pipeline; + final Map, PCollection> pcollectionMap; + + PCollectionTuple(Pipeline pipeline) { + this(pipeline, new LinkedHashMap, PCollection>()); + } + + PCollectionTuple(Pipeline pipeline, + Map, PCollection> pcollectionMap) { + this.pipeline = pipeline; + this.pcollectionMap = Collections.unmodifiableMap(pcollectionMap); + } + + /** + * Returns a PCollectionTuple with each of the given tags mapping to a new + * output PCollection. + * + *

For use by primitive transformations only. + */ + public static PCollectionTuple ofPrimitiveOutputsInternal( + TupleTagList outputTags, WindowingFn windowingFn) { + Map, PCollection> pcollectionMap = new LinkedHashMap<>(); + for (TupleTag outputTag : outputTags.tupleTags) { + if (pcollectionMap.containsKey(outputTag)) { + throw new IllegalArgumentException( + "TupleTag already present in this tuple"); + } + + // In fact, `token` and `outputCollection` should have + // types TypeToken and PCollection for some + // unknown T. It is safe to create `outputCollection` + // with type PCollection because it has the same + // erasure as the correct type. When a transform adds + // elements to `outputCollection` they will be of type T. + @SuppressWarnings("unchecked") + TypeToken token = (TypeToken) outputTag.getTypeToken(); + PCollection outputCollection = PCollection + .createPrimitiveOutputInternal(windowingFn) + .setTypeTokenInternal(token); + + pcollectionMap.put(outputTag, outputCollection); + } + return new PCollectionTuple(null, pcollectionMap); + } + + @Override + public Pipeline getPipeline() { + return pipeline; + } + + @Override + public Collection expand() { + return pcollectionMap.values(); + } + + @Override + public void recordAsOutput(Pipeline pipeline, + PTransform transform) { + if (this.pipeline != null && this.pipeline != pipeline) { + throw new AssertionError( + "not expecting to change the Pipeline owning a PCollectionTuple"); + } + this.pipeline = pipeline; + int i = 0; + for (Map.Entry, PCollection> entry + : pcollectionMap.entrySet()) { + TupleTag tag = entry.getKey(); + PCollection pc = entry.getValue(); + pc.recordAsOutput(pipeline, transform, tag.getOutName(i)); + i++; + } + } + + @Override + public void finishSpecifying() { + for (PCollection pc : pcollectionMap.values()) { + pc.finishSpecifying(); + } + } + + @Override + public void finishSpecifyingOutput() { + for (PCollection pc : pcollectionMap.values()) { + pc.finishSpecifyingOutput(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionView.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionView.java new file mode 100644 index 0000000000000..d19854ccc588b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionView.java @@ -0,0 +1,45 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import com.google.cloud.dataflow.sdk.util.WindowedValue; + +import java.io.Serializable; + +/** + * A {@code PCollectionView} is an immutable view of a + * {@link PCollection} that can be accessed e.g. as a + * side input to a {@link DoFn}. + * + *

A {@PCollectionView} should always be the output of a {@link PTransform}. It is + * the joint responsibility of this transform and each {@link PipelineRunner} to + * implement the view in a runner-specific manner. + * + * @param the type of the value(s) accessible via this {@code PCollectionView} + * @param the type of the windowed value(s) accessible via this {@code PCollectionView} + */ +public interface PCollectionView extends PValue, Serializable { + /** + * A unique identifier, for internal use. + */ + public TupleTag>> getTagInternal(); + + /** + * For internal use only. + */ + public T fromIterableInternal(Iterable> contents); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PDone.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PDone.java new file mode 100644 index 0000000000000..dda48fc530a8c --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PDone.java @@ -0,0 +1,36 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import java.util.Collection; +import java.util.Collections; + +/** + * {@code PDone} is the output of a + * {@link com.google.cloud.dataflow.sdk.transforms.PTransform} that + * doesn't have a non-trival result, e.g., a Write. No more + * transforms can be applied to it. + */ +public class PDone extends POutputValueBase { + public PDone() {} + + @Override + public Collection expand() { + // A PDone contains no PValues. + return Collections.emptyList(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PInput.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PInput.java new file mode 100644 index 0000000000000..6d86fb069535e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PInput.java @@ -0,0 +1,57 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import com.google.cloud.dataflow.sdk.Pipeline; + +import java.util.Collection; + +/** + * The abstract interface of things that might be input to a + * {@link com.google.cloud.dataflow.sdk.transforms.PTransform}. + */ +public interface PInput { + /** + * Returns the owning Pipeline of this PInput. + * + * @throws IllegalStateException if the owning Pipeline hasn't been + * set yet + */ + public Pipeline getPipeline(); + + /** + * Expands this PInput into a list of its component input PValues. + * + *

A PValue expands to itself. + * + *

A tuple or list of PValues (e.g., + * PCollectionTuple, and PCollectionList) expands to its component + * PValues. + * + *

Not intended to be invoked directly by user code. + */ + public Collection expand(); + + /** + *

After building, finalizes this PInput to make it ready for + * being used as an input to a PTransform. + * + *

Automatically invoked whenever {@code apply()} is invoked on + * this PInput, so users do not normally call this explicitly. + */ + public void finishSpecifying(); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutput.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutput.java new file mode 100644 index 0000000000000..3b3264985d559 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutput.java @@ -0,0 +1,72 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.transforms.PTransform; + +import java.util.Collection; + +/** + * The abstract interface of things that might be output from a + * {@link PTransform}. + */ +public interface POutput { + /** + * Expands this {@code POutput} into a list of its component output + * {@code PValue}s. + * + *

A {@link PValue} expands to itself. + * + *

A tuple or list of {@code PValue}s (e.g., + * {@link PCollectionTuple}, and + * {@link PCollectionList}) expands to its component {@code PValue}s. + * + *

Not intended to be invoked directly by user code. + */ + public Collection expand(); + + /** + * Records that this {@code POutput} is an output of the given + * {@code PTransform} in the given {@code Pipeline}. + * + *

Should expand this {@code POutput} and invoke + * {@link PValue#recordAsOutput(Pipeline, + * com.google.cloud.dataflow.sdk.transforms.PTransform, + * String)} on each component output {@code PValue}. + * + *

Automatically invoked as part of applying a + * {@code PTransform}. Not to be invoked directly by user code. + */ + public void recordAsOutput(Pipeline pipeline, + PTransform transform); + + /** + * As part of finishing the producing {@code PTransform}, finalizes this + * {@code PTransform} output to make it ready for being used as an input and + * for running. + * + *

This includes ensuring that all {@code PCollection}s + * have {@code Coder}s specified or defaulted. + * + *

Automatically invoked whenever this {@code POutput} is used + * as a {@code PInput} to another {@code PTransform}, or if never + * used as a {@code PInput}, when {@link Pipeline#run} is called, so + * users do not normally call this explicitly. + */ + public void finishSpecifyingOutput(); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutputValueBase.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutputValueBase.java new file mode 100644 index 0000000000000..0401393f142b8 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutputValueBase.java @@ -0,0 +1,83 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.transforms.PTransform; + +/** + * A {@code POutputValueBase} is the abstract base class of + * {@code PTransform} outputs. + * + *

A {@code PValueBase} that adds tracking of its producing + * {@code PTransform}. + * + *

For internal use. + */ +public abstract class POutputValueBase implements POutput { + + protected POutputValueBase() { } + + /** + * Returns the {@code PTransform} that this {@code POutputValueBase} + * is an output of. + * + *

For internal use only. + */ + public PTransform getProducingTransformInternal() { + return producingTransform; + } + + /** + * Records that this {@code POutputValueBase} is an output with the + * given name of the given {@code PTransform} in the given + * {@code Pipeline}. + * + *

To be invoked only by {@link POutput#recordAsOutput} + * implementations. Not to be invoked directly by user code. + */ + public void recordAsOutput(Pipeline pipeline, + PTransform transform) { + if (producingTransform != null) { + // Already used this POutput as a PTransform output. This can + // happen if the POutput is an output of a transform within a + // composite transform, and is also the result of the composite. + // We want to record the "immediate" atomic transform producing + // this output, and ignore all later composite transforms that + // also produce this output. + // + // Pipeline.applyInternal() uses !hasProducingTransform() to + // avoid calling this operation redundantly, but + // hasProducingTransform() doesn't apply to POutputValueBases + // that aren't PValues or composites of PValues, e.g., PDone. + return; + } + producingTransform = transform; + } + + /** + * Default behavior for {@code finishSpecifyingOutput()} is + * to do nothing. Override if your {@link PValue} requires + * finalization. + */ + public void finishSpecifyingOutput() { } + + /** + * The {@code PTransform} that produces this {@code POutputValueBase}. + */ + private PTransform producingTransform; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValue.java new file mode 100644 index 0000000000000..7e45196af813c --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValue.java @@ -0,0 +1,37 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.transforms.PTransform; + +/** + * A {@code PValue} is the interface to values that can be + * input and output from {@link PTransform}s. + */ +public interface PValue extends POutput, PInput { + public String getName(); + + public PValue setPipelineInternal(Pipeline pipeline); + + /** + * Returns the {@code PTransform} that this {@code PValue} is an output of. + * + *

For internal use only. + */ + public PTransform getProducingTransformInternal(); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValueBase.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValueBase.java new file mode 100644 index 0000000000000..25b1fd6fd9a13 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValueBase.java @@ -0,0 +1,190 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.util.StringUtils; + +import java.util.Collection; +import java.util.Collections; + +/** + * A {@code PValueBase} is an abstract base class that provides + * sensible default implementations for methods of {@link PValue}. + * In particular, this includes functionality for getting/setting: + * + *

    + *
  • The {@code Pipeline} that the {@code PValue} is + * part of. + *
  • Whether the {@code PValue} has bee finalized (as an input + * or an output), after which its properties can + * no longer be changed. + *
+ * + *

For internal use. + */ +public abstract class PValueBase extends POutputValueBase implements PValue { + /** + * Returns the name of this {@code PValueBase}. + * + *

By default, the name of a {@code PValueBase} is based on the + * name of the {@code PTransform} that produces it. It can be + * specified explicitly by calling {@link #setName}. + * + * @throws IllegalStateException if the name hasn't been set yet + */ + public String getName() { + if (name == null) { + throw new IllegalStateException("name not set"); + } + return name; + } + + /** + * Sets the name of this {@code PValueBase}. Returns {@code this}. + * + * @throws IllegalStateException if this {@code PValueBase} has + * already been finalized and is no longer settable, e.g., by having + * {@code apply()} called on it + */ + public PValueBase setName(String name) { + if (finishedSpecifying) { + throw new IllegalStateException( + "cannot change the name of " + this + " once it's been used"); + } + this.name = name; + return this; + } + + ///////////////////////////////////////////////////////////////////////////// + + protected PValueBase() {} + + /** + * The name of this {@code PValueBase}, or null if not yet set. + */ + private String name; + + /** + * The {@code Pipeline} that owns this {@code PValueBase}, or null + * if not yet set. + */ + private Pipeline pipeline; + + /** + * Whether this {@code PValueBase} has been finalized, and its core + * properties, e.g., name, can no longer be changed. + */ + private boolean finishedSpecifying = false; + + + /** + * Returns the owning {@code Pipeline} of this {@code PValueBase}. + * + * @throws IllegalStateException if the owning {@code Pipeline} + * hasn't been set yet + */ + @Override + public Pipeline getPipeline() { + if (pipeline == null) { + throw new IllegalStateException("owning pipeline not set"); + } + return pipeline; + } + + /** + * Sets the owning {@code Pipeline} of this {@code PValueBase}. + * Returns {@code this}. + * + *

For internal use only. + * + * @throws IllegalArgumentException if the owner has already been set + * differently + */ + @Override + public PValue setPipelineInternal(Pipeline pipeline) { + if (this.pipeline != null + && this.pipeline != pipeline) { + throw new IllegalArgumentException( + "owning pipeline cannot be changed once set"); + } + this.pipeline = pipeline; + return this; + } + + @Override + public void recordAsOutput(Pipeline pipeline, + PTransform transform) { + recordAsOutput(pipeline, transform, "out"); + } + + /** + * Records that this {@code POutputValueBase} is an output with the + * given name of the given {@code PTransform} in the given + * {@code Pipeline}. + * + *

To be invoked only by {@link POutput#recordAsOutput} + * implementations. Not to be invoked directly by user code. + */ + protected void recordAsOutput(Pipeline pipeline, + PTransform transform, + String outName) { + super.recordAsOutput(pipeline, transform); + if (name == null) { + name = pipeline.getFullName(transform) + "." + outName; + } + } + + /** + * Returns whether this {@code PValueBase} has been finalized, and + * its core properties, e.g., name, can no longer be changed. + * + *

For internal use only. + */ + public boolean isFinishedSpecifyingInternal() { + return finishedSpecifying; + } + + @Override + public Collection expand() { + return Collections.singletonList(this); + } + + @Override + public void finishSpecifying() { + getProducingTransformInternal().finishSpecifying(); + finishedSpecifying = true; + } + + @Override + public String toString() { + return (name == null ? "" : getName()) + + " [" + getKindString() + "]"; + } + + /** + * Returns a {@code String} capturing the kind of this + * {@code PValueBase}. + * + *

By default, uses the base name of this {@code PValueBase}'s + * class as its kind string. + */ + protected String getKindString() { + return StringUtils.approximateSimpleName(getClass()); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java new file mode 100644 index 0000000000000..9d91a18cb3cf0 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java @@ -0,0 +1,133 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.InstantCoder; +import com.google.cloud.dataflow.sdk.coders.StandardCoder; +import com.google.cloud.dataflow.sdk.util.PropertyNames; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.List; + +/** + * An immutable (value, timestamp) pair. + * + *

Used for assigning initial timestamps to values inserted into a pipeline + * with {@link com.google.cloud.dataflow.sdk.transforms.Create#timestamped}. + * + * @param the type of the value + */ +public class TimestampedValue { + + /** + * Returns a new {@code TimestampedValue} with the given value and timestamp. + */ + public static TimestampedValue of(V value, Instant timestamp) { + return new TimestampedValue<>(value, timestamp); + } + + public V getValue() { + return value; + } + + public Instant getTimestamp() { + return timestamp; + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Coder for {@code TimestampedValue}. + */ + public static class TimestampedValueCoder + extends StandardCoder> { + + private final Coder valueCoder; + + public static TimestampedValueCoder of(Coder valueCoder) { + return new TimestampedValueCoder<>(valueCoder); + } + + @JsonCreator + public static TimestampedValueCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List components) { + checkArgument(components.size() == 1, + "Expecting 1 component, got " + components.size()); + return of((Coder) components.get(0)); + } + + @SuppressWarnings("unchecked") + TimestampedValueCoder(Coder valueCoder) { + this.valueCoder = checkNotNull(valueCoder); + } + + @Override + public void encode(TimestampedValue windowedElem, + OutputStream outStream, + Context context) + throws IOException { + valueCoder.encode(windowedElem.getValue(), outStream, context.nested()); + InstantCoder.of().encode( + windowedElem.getTimestamp(), outStream, context); + } + + @Override + public TimestampedValue decode(InputStream inStream, Context context) + throws IOException { + T value = valueCoder.decode(inStream, context.nested()); + Instant timestamp = InstantCoder.of().decode(inStream, context); + return TimestampedValue.of(value, timestamp); + } + + @Override + public boolean isDeterministic() { + return valueCoder.isDeterministic(); + } + + @Override + public List> getCoderArguments() { + return Arrays.>asList(valueCoder); + } + + public static List getInstanceComponents(TimestampedValue exampleValue) { + return Arrays.asList(exampleValue.getValue()); + } + } + + ///////////////////////////////////////////////////////////////////////////// + + private final V value; + private final Instant timestamp; + + protected TimestampedValue(V value, Instant timestamp) { + this.value = value; + this.timestamp = timestamp; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java new file mode 100644 index 0000000000000..58562163f4a85 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java @@ -0,0 +1,170 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.common.reflect.TypeToken; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.Serializable; +import java.util.Random; + +/** + * A {@code TupleTag} is a typed tag to use as the key of a + * heterogeneously typed tuple, like {@link PCollectionTuple} or + * Its generic type parameter allows tracking + * the static type of things stored in tuples. + * + *

To aid in assigning default {@code Coder}s for results of + * side outputs of {@code ParDo}, an output + * {@code TupleTag} should be instantiated with an extra {@code {}} so + * it is an instance of an anonymous subclass without generic type + * parameters. Input {@code TupleTag}s require no such extra + * instantiation (although it doesn't hurt). For example: + * + *

 {@code
+ * TupleTag inputTag = new TupleTag<>();
+ * TupleTag outputTag = new TupleTag(){};
+ * } 
+ * + * @param the type of the elements or values of the tagged thing, + * e.g., a {@code PCollection}. + */ +public class TupleTag implements Serializable { + /** + * Constructs a new {@code TupleTag}, with a fresh unique id. + * + *

This is the normal way {@code TupleTag}s are constructed. + */ + public TupleTag() { + this.id = genId(); + this.generated = true; + } + + /** + * Constructs a new {@code TupleTag} with the given id. + * + *

It is up to the user to ensure that two {@code TupleTag}s + * with the same id actually mean the same tag and carry the same + * generic type parameter. Violating this invariant can lead to + * hard-to-diagnose runtime type errors. Consequently, this + * operation should be used very sparingly, such as when the + * producer and consumer of {@code TupleTag}s are written in + * separate modules and can only coordinate via ids rather than + * shared {@code TupleTag} instances. Most of the time, + * {@link #TupleTag()} should be preferred. + */ + public TupleTag(String id) { + this.id = id; + this.generated = false; + } + + /** + * Returns the id of this {@code TupleTag}. + * + *

Two {@code TupleTag}s with the same id are considered equal. + * + *

{@code TupleTag}s are not ordered, i.e., the class does not implement + * Comparable interface. TupleTags implement equals and hashCode, making them + * suitable for use as keys in HashMap and HashSet. + */ + public String getId() { return id; } + + /** + * If this {@code TupleTag} is tagging output {@code outputIndex} of + * a {@code PTransform}, returns the name that should be used by + * default for the output. + */ + public String getOutName(int outIndex) { + if (generated) { + return "out" + outIndex; + } else { + return id; + } + } + + /** + * Returns a {@code TypeToken} capturing what is known statically + * about the type of this {@code TupleTag} instance's most-derived + * class. + * + *

This is useful for a {@code TupleTag} constructed as an + * instance of an anonymous subclass with a trailing {@code {}}, + * e.g., {@code new TupleTag(){}}. + */ + public TypeToken getTypeToken() { + return new TypeToken(getClass()) {}; + } + + + ///////////////////////////////////////////////////////////////////////////// + // Internal details below here. + + static final Random RANDOM = new Random(0); + + final String id; + final boolean generated; + + /** Generates and returns a fresh unique id for a TupleTag's id. */ + static String genId() { + long randomLong; + synchronized (RANDOM) { + randomLong = RANDOM.nextLong(); + } + return Long.toHexString(randomLong); + } + + @JsonCreator + private static TupleTag fromJson( + @JsonProperty(PropertyNames.VALUE) String id, + @JsonProperty(PropertyNames.IS_GENERATED) boolean generated) { + return new TupleTag(id, generated); + } + + private TupleTag(String id, boolean generated) { + this.id = id; + this.generated = generated; + } + + public CloudObject asCloudObject() { + CloudObject result = CloudObject.forClass(getClass()); + addString(result, PropertyNames.VALUE, id); + addBoolean(result, PropertyNames.IS_GENERATED, generated); + return result; + } + + @Override + public boolean equals(Object that) { + if (that instanceof TupleTag) { + return this.id.equals(((TupleTag) that).id); + } else { + return false; + } + } + + @Override + public int hashCode() { return id.hashCode(); } + + @Override + public String toString() { return "Tag<" + id + ">"; } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTagList.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTagList.java new file mode 100644 index 0000000000000..27a0683bab5aa --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTagList.java @@ -0,0 +1,146 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import com.google.common.collect.ImmutableList; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * A {@code TupleTagList} is an immutable list of heterogeneously + * typed {@link TupleTag}s. A TupleTagList is used, for instance, to + * specify the tags of the side outputs of a + * {@link com.google.cloud.dataflow.sdk.transforms.ParDo}. + * + *

TupleTagLists can be created and accessed like follows: + *

 {@code
+ * TupleTag tag1 = ...;
+ * TupleTag tag2 = ...;
+ * TupleTag> tag3 = ...;
+ *
+ * // Create a TupleTagList with three TupleTags:
+ * TupleTagList tags = TupleTagList.of(tag1).and(tag2).and(tag3);
+ *
+ * // Create an empty TupleTagList:
+ * Pipeline p = ...;
+ * TupleTagList tags2 = TupleTagList.empty(p);
+ *
+ * // Get TupleTags out of a TupleTagList, by index (origin 0):
+ * TupleTag tagX = tags.get(1);
+ * TupleTag tagY = tags.get(0);
+ * TupleTag tagZ = tags.get(2);
+ *
+ * // Get a list of all TupleTags in a TupleTagList:
+ * List> allTags = tags.getAll();
+ * } 
+ */ +public class TupleTagList implements Serializable { + /** + * Returns an empty TupleTagList. + * + *

Longer TupleTagLists can be created by calling + * {@link #and} on the result. + */ + public static TupleTagList empty() { + return new TupleTagList(); + } + + /** + * Returns a singleton TupleTagList containing the given TupleTag. + * + *

Longer TupleTagLists can be created by calling + * {@link #and} on the result. + */ + public static TupleTagList of(TupleTag tag) { + return empty().and(tag); + } + + /** + * Returns a TupleTagList containing the given TupleTags, in order. + * + *

Longer TupleTagLists can be created by calling + * {@link #and} on the result. + */ + public static TupleTagList of(List> tags) { + return empty().and(tags); + } + + /** + * Returns a new TupleTagList that has all the TupleTags of + * this TupleTagList plus the given TupleTag appended to the end. + */ + public TupleTagList and(TupleTag tag) { + return new TupleTagList( + new ImmutableList.Builder>() + .addAll(tupleTags) + .add(tag) + .build()); + } + + /** + * Returns a new TupleTagList that has all the TupleTags of + * this TupleTagList plus the given TupleTags appended to the end, + * in order. + */ + public TupleTagList and(List> tags) { + return new TupleTagList( + new ImmutableList.Builder>() + .addAll(tupleTags) + .addAll(tags) + .build()); + } + + /** + * Returns the number of TupleTags in this TupleTagList. + */ + public int size() { + return tupleTags.size(); + } + + /** + * Returns the TupleTag at the given index (origin zero). Throws + * IndexOutOfBounds if the index is out of the range + * {@code [0..size()-1]}. + */ + public TupleTag get(int index) { + return tupleTags.get(index); + } + + /** + * Returns an immutable List of all the TupleTags in this TupleTagList. + */ + public List> getAll() { + return tupleTags; + } + + + ///////////////////////////////////////////////////////////////////////////// + // Internal details below here. + + final List> tupleTags; + + TupleTagList() { + this(new ArrayList>()); + } + + TupleTagList(List> tupleTags) { + this.tupleTags = Collections.unmodifiableList(tupleTags); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypedPValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypedPValue.java new file mode 100644 index 0000000000000..95b9b45f53770 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypedPValue.java @@ -0,0 +1,168 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.common.reflect.TypeToken; + +/** + * A {@code TypedPValue} is the abstract base class of things that + * store some number of values of type {@code T}. Because we know + * the type {@code T}, this is the layer of the inheritance hierarchy where + * we store a coder for objects of type {@code T} + * + * @param the type of the values stored in this {@code TypedPValue} + */ +public abstract class TypedPValue extends PValueBase implements PValue { + + /** + * Returns the Coder used by this TypedPValue to encode and decode + * the values stored in it. + * + * @throws IllegalStateException if the Coder hasn't been set, and + * couldn't be inferred + */ + public Coder getCoder() { + if (coder == null) { + throw new IllegalStateException( + "coder for " + this + " not set, and couldn't be inferred; " + + "either register a default Coder for its element type, " + + "or use setCoder() to specify one explicitly"); + } + return coder; + } + + /** + * Sets the Coder used by this TypedPValue to encode and decode the + * values stored in it. Returns {@code this}. + * + * @throws IllegalStateException if this TypedPValue has already + * been finalized and is no longer settable, e.g., by having + * {@code apply()} called on it + */ + public TypedPValue setCoder(Coder coder) { + if (isFinishedSpecifyingInternal()) { + throw new IllegalStateException( + "cannot change the Coder of " + this + " once it's been used"); + } + if (coder == null) { + throw new IllegalArgumentException( + "Cannot setCoder(null)"); + } + this.coder = coder; + return this; + } + + @Override + public void recordAsOutput(Pipeline pipeline, + PTransform transform, + String outName) { + super.recordAsOutput(pipeline, transform, outName); + pipeline.addValueInternal(this); + } + + @Override + public TypedPValue setPipelineInternal(Pipeline pipeline) { + super.setPipelineInternal(pipeline); + return this; + } + + /** + * After building, finalizes this PValue to make it ready for + * running. Automatically invoked whenever the PValue is "used" + * (e.g., when apply() is called on it) and when the Pipeline is + * run (useful if this is a PValue with no consumers). + */ + @Override + public void finishSpecifying() { + if (isFinishedSpecifyingInternal()) { + return; + } + super.finishSpecifying(); + } + + ///////////////////////////////////////////////////////////////////////////// + // Internal details below here. + + /** + * The Coder used by this TypedPValue to encode and decode the + * values stored in it, or null if not specified nor inferred yet. + */ + private Coder coder; + + protected TypedPValue() {} + + private TypeToken typeToken; + + /** + * Returns a {@code TypeToken} with some reflective information + * about {@code T}, if possible. May return {@code null} if no information + * is available. Subclasses may override this to enable better + * {@code Coder} inference. + */ + public TypeToken getTypeToken() { + return typeToken; + } + + /** + * Sets the {@code TypeToken} associated with this class. Better + * reflective type information will lead to better {@code Coder} + * inference. + */ + public TypedPValue setTypeTokenInternal(TypeToken typeToken) { + this.typeToken = typeToken; + return this; + } + + + /** + * If the coder is not explicitly set, this sets the coder for + * this {@code TypedPValue} to the best coder that can be inferred + * based upon the known {@code TypeToken}. By default, this is null, + * but can and should be improved by subclasses. + */ + @Override + public void finishSpecifyingOutput() { + if (coder == null) { + TypeToken token = getTypeToken(); + CoderRegistry registry = getProducingTransformInternal() + .getPipeline() + .getCoderRegistry(); + + if (token != null) { + coder = registry.getDefaultCoder(token); + } + + if (coder == null) { + coder = getProducingTransformInternal().getDefaultOutputCoder(this); + } + + if (coder == null) { + throw new IllegalStateException( + "unable to infer a default Coder for " + this + + "; either register a default Coder for its element type, " + + "or use setCoder() to specify one explicitly. " + + "If a default coder is registered, it may not be found " + + "due to type erasure; again, use setCoder() to specify " + + "a Coder explicitly"); + } + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/package-info.java new file mode 100644 index 0000000000000..ba6e927e0996b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/package-info.java @@ -0,0 +1,42 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Defines {@link com.google.cloud.dataflow.sdk.values.PCollection} and other classes for + * representing data in a {@link com.google.cloud.dataflow.sdk.Pipeline}. + * + *

A {@link com.google.cloud.dataflow.sdk.values.PCollection} is an immutable collection of + * values of type {@code T} and is the main representation for data. + * A {@link com.google.cloud.dataflow.sdk.values.PCollectionTuple} is a tuple of PCollections + * used in cases where PTransforms take or return multiple PCollections. + * + *

A {@link com.google.cloud.dataflow.sdk.values.PCollectionTuple} is an immutable tuple of + * heterogeneously-typed {@link com.google.cloud.dataflow.sdk.values.PCollection}s, "keyed" by + * {@link com.google.cloud.dataflow.sdk.values.TupleTag}s. + * A PCollectionTuple can be used as the input or + * output of a + * {@link com.google.cloud.dataflow.sdk.transforms.PTransform} taking + * or producing multiple PCollection inputs or outputs that can be of + * different types, for instance a + * {@link com.google.cloud.dataflow.sdk.transforms.ParDo} with side + * outputs. + * + *

A {@link com.google.cloud.dataflow.sdk.values.PCollectionView} is an immutable view of a + * PCollection that can be accessed from a DoFn and other user Fns + * as a side input. + * + */ +package com.google.cloud.dataflow.sdk.values; diff --git a/sdk/src/main/resources/com/google/cloud/dataflow/sdk/sdk.properties b/sdk/src/main/resources/com/google/cloud/dataflow/sdk/sdk.properties new file mode 100644 index 0000000000000..5b0a720b215d8 --- /dev/null +++ b/sdk/src/main/resources/com/google/cloud/dataflow/sdk/sdk.properties @@ -0,0 +1,5 @@ +# SDK source version. +version=${pom.version} + +build.date=${timestamp} + diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/PipelineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/PipelineTest.java new file mode 100644 index 0000000000000..13d2b2996cfd1 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/PipelineTest.java @@ -0,0 +1,105 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.hamcrest.core.IsNot.not; +import static org.junit.Assert.fail; + +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; +import com.google.cloud.dataflow.sdk.util.UserCodeException; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for Pipeline. + */ +@RunWith(JUnit4.class) +public class PipelineTest { + + static class PipelineWrapper extends Pipeline { + protected PipelineWrapper(PipelineRunner runner) { + super(runner, PipelineOptionsFactory.create()); + } + } + + // Mock class that throws a user code exception during the call to + // Pipeline.run(). + static class TestPipelineRunnerThrowingUserException + extends PipelineRunner { + @Override + public PipelineResult run(Pipeline pipeline) { + Throwable t = new IllegalStateException("user code exception"); + throw new UserCodeException(t); + } + } + + // Mock class that throws an SDK or API client code exception during + // the call to Pipeline.run(). + static class TestPipelineRunnerThrowingSDKException + extends PipelineRunner { + @Override + public PipelineResult run(Pipeline pipeline) { + throw new IllegalStateException("SDK exception"); + } + } + + @Test + public void testPipelineUserExceptionHandling() { + Pipeline p = new PipelineWrapper( + new TestPipelineRunnerThrowingUserException()); + + // Check pipeline runner correctly catches user errors. + try { + Object results = p.run(); + fail("Should have thrown an exception."); + } catch (RuntimeException exn) { + // Make sure users don't have to worry about the + // UserCodeException wrapper. + Assert.assertThat(exn, not(instanceOf(UserCodeException.class))); + // Assert that the message is correct. + Assert.assertThat( + exn.getMessage(), containsString("user code exception")); + // Cause should be IllegalStateException. + Assert.assertThat( + exn.getCause(), instanceOf(IllegalStateException.class)); + } + } + + @Test + public void testPipelineSDKExceptionHandling() { + Pipeline p = new PipelineWrapper(new TestPipelineRunnerThrowingSDKException()); + + // Check pipeline runner correctly catches SDK errors. + try { + Object results = p.run(); + fail("Should have thrown an exception."); + } catch (RuntimeException exn) { + // Make sure the exception isn't a UserCodeException. + Assert.assertThat(exn, not(instanceOf(UserCodeException.class))); + // Assert that the message is correct. + Assert.assertThat(exn.getMessage(), containsString("SDK exception")); + // RuntimeException should be IllegalStateException. + Assert.assertThat(exn, instanceOf(IllegalStateException.class)); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/TestUtils.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/TestUtils.java new file mode 100644 index 0000000000000..9a92ba0167c4e --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/TestUtils.java @@ -0,0 +1,231 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk; + +import static org.junit.Assert.assertThat; + +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionView; + +import org.hamcrest.CoreMatchers; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * Utilities for tests. + */ +public class TestUtils { + // Do not instantiate. + private TestUtils() {} + + public static final String[] NO_LINES_ARRAY = new String[] { }; + + public static final List NO_LINES = Arrays.asList(NO_LINES_ARRAY); + + public static final String[] LINES_ARRAY = new String[] { + "To be, or not to be: that is the question: ", + "Whether 'tis nobler in the mind to suffer ", + "The slings and arrows of outrageous fortune, ", + "Or to take arms against a sea of troubles, ", + "And by opposing end them? To die: to sleep; ", + "No more; and by a sleep to say we end ", + "The heart-ache and the thousand natural shocks ", + "That flesh is heir to, 'tis a consummation ", + "Devoutly to be wish'd. To die, to sleep; ", + "To sleep: perchance to dream: ay, there's the rub; ", + "For in that sleep of death what dreams may come ", + "When we have shuffled off this mortal coil, ", + "Must give us pause: there's the respect ", + "That makes calamity of so long life; ", + "For who would bear the whips and scorns of time, ", + "The oppressor's wrong, the proud man's contumely, ", + "The pangs of despised love, the law's delay, ", + "The insolence of office and the spurns ", + "That patient merit of the unworthy takes, ", + "When he himself might his quietus make ", + "With a bare bodkin? who would fardels bear, ", + "To grunt and sweat under a weary life, ", + "But that the dread of something after death, ", + "The undiscover'd country from whose bourn ", + "No traveller returns, puzzles the will ", + "And makes us rather bear those ills we have ", + "Than fly to others that we know not of? ", + "Thus conscience does make cowards of us all; ", + "And thus the native hue of resolution ", + "Is sicklied o'er with the pale cast of thought, ", + "And enterprises of great pith and moment ", + "With this regard their currents turn awry, ", + "And lose the name of action.--Soft you now! ", + "The fair Ophelia! Nymph, in thy orisons ", + "Be all my sins remember'd." }; + + public static final List LINES = Arrays.asList(LINES_ARRAY); + + public static final String[] LINES2_ARRAY = new String[] { + "hi", "there", "bob!" }; + + public static final List LINES2 = Arrays.asList(LINES2_ARRAY); + + public static final Integer[] NO_INTS_ARRAY = new Integer[] { }; + + public static final List NO_INTS = Arrays.asList(NO_INTS_ARRAY); + + public static final Integer[] INTS_ARRAY = new Integer[] { + 3, 42, Integer.MAX_VALUE, 0, -1, Integer.MIN_VALUE, 666 }; + + public static final List INTS = Arrays.asList(INTS_ARRAY); + + /** + * Matcher for KVs. + */ + public static class KvMatcher + extends TypeSafeMatcher> { + final Matcher keyMatcher; + final Matcher valueMatcher; + + public static KvMatcher isKv(Matcher keyMatcher, + Matcher valueMatcher) { + return new KvMatcher<>(keyMatcher, valueMatcher); + } + + public KvMatcher(Matcher keyMatcher, + Matcher valueMatcher) { + this.keyMatcher = keyMatcher; + this.valueMatcher = valueMatcher; + } + + @Override + public boolean matchesSafely(KV kv) { + return keyMatcher.matches(kv.getKey()) + && valueMatcher.matches(kv.getValue()); + } + + @Override + public void describeTo(Description description) { + description + .appendText("a KV(").appendValue(keyMatcher) + .appendText(", ").appendValue(valueMatcher) + .appendText(")"); + } + } + + public static PCollection createStrings(Pipeline p, + Iterable values) { + return p.apply(Create.of(values)).setCoder(StringUtf8Coder.of()); + } + + public static PCollection createInts(Pipeline p, + Iterable values) { + return p.apply(Create.of(values)).setCoder(BigEndianIntegerCoder.of()); + } + + public static PCollectionView + createSingletonInt(Pipeline p, Integer value) { + PCollection collection = p.apply(Create.of(value)); + return collection.apply(View.asSingleton()); + } + + //////////////////////////////////////////////////////////////////////////// + // Utilities for testing CombineFns, ensuring they give correct results + // across various permutations and shardings of the input. + + public static void checkCombineFn( + CombineFn fn, List input, final VO expected) { + checkCombineFn(fn, input, CoreMatchers.is(expected)); + } + + public static void checkCombineFn( + CombineFn fn, List input, Matcher matcher) { + checkCombineFnInternal(fn, input, matcher); + Collections.shuffle(input); + checkCombineFnInternal(fn, input, matcher); + } + + private static void checkCombineFnInternal( + CombineFn fn, List input, Matcher matcher) { + int size = input.size(); + checkCombineFnShards(fn, Collections.singletonList(input), matcher); + checkCombineFnShards(fn, shardEvenly(input, 2), matcher); + if (size > 4) { + checkCombineFnShards(fn, shardEvenly(input, size / 2), matcher); + checkCombineFnShards( + fn, shardEvenly(input, (int) (size / Math.sqrt(size))), matcher); + } + checkCombineFnShards(fn, shardExponentially(input, 1.4), matcher); + checkCombineFnShards(fn, shardExponentially(input, 2), matcher); + checkCombineFnShards(fn, shardExponentially(input, Math.E), matcher); + } + + public static void checkCombineFnShards( + CombineFn fn, + List> shards, + Matcher matcher) { + checkCombineFnShardsInternal(fn, shards, matcher); + Collections.shuffle(shards); + checkCombineFnShardsInternal(fn, shards, matcher); + } + + private static void checkCombineFnShardsInternal( + CombineFn fn, + Iterable> shards, + Matcher matcher) { + List accumulators = new ArrayList<>(); + for (Iterable shard : shards) { + VA accumulator = fn.createAccumulator(); + for (VI elem : shard) { + fn.addInput(accumulator, elem); + } + accumulators.add(accumulator); + } + VA merged = fn.mergeAccumulators(accumulators); + assertThat(fn.extractOutput(merged), matcher); + } + + private static List> shardEvenly(List input, int numShards) { + List> shards = new ArrayList<>(numShards); + for (int i = 0; i < numShards; i++) { + shards.add(input.subList(i * input.size() / numShards, + (i + 1) * input.size() / numShards)); + } + return shards; + } + + private static List> shardExponentially( + List input, double base) { + assert base > 1.0; + List> shards = new ArrayList<>(); + int end = input.size(); + while (end > 0) { + int start = (int) (end / base); + shards.add(input.subList(start, end)); + end = start; + } + return shards; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java new file mode 100644 index 0000000000000..725c0e852022d --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java @@ -0,0 +1,189 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder.Context; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; + +/** + * Tests for AvroCoder. + */ +@RunWith(JUnit4.class) +public class AvroCoderTest { + + @DefaultCoder(AvroCoder.class) + private static class Pojo { + public String text; + public int count; + + // Empty constructor required for Avro decoding. + public Pojo() { + } + + public Pojo(String text, int count) { + this.text = text; + this.count = count; + } + + // auto-generated + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + Pojo pojo = (Pojo) o; + + if (count != pojo.count) { + return false; + } + if (text != null + ? !text.equals(pojo.text) + : pojo.text != null) { + return false; + } + + return true; + } + + @Override + public String toString() { + return "Pojo{" + + "text='" + text + '\'' + + ", count=" + count + + '}'; + } + } + + static class GetTextFn extends DoFn { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().text); + } + } + + @Test + public void testAvroCoderEncoding() throws Exception { + AvroCoder coder = AvroCoder.of(Pojo.class); + CloudObject encoding = coder.asCloudObject(); + + Assert.assertThat(encoding.keySet(), + Matchers.containsInAnyOrder("@type", "type", "schema")); + } + + @Test + public void testPojoEncoding() throws Exception { + Pojo before = new Pojo("Hello", 42); + + AvroCoder coder = AvroCoder.of(Pojo.class); + byte[] bytes = CoderUtils.encodeToByteArray(coder, before); + Pojo after = CoderUtils.decodeFromByteArray(coder, bytes); + + Assert.assertEquals(before, after); + } + + @Test + public void testGenericRecordEncoding() throws Exception { + String schemaString = + "{\"namespace\": \"example.avro\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"User\",\n" + + " \"fields\": [\n" + + " {\"name\": \"name\", \"type\": \"string\"},\n" + + " {\"name\": \"favorite_number\", \"type\": [\"int\", \"null\"]},\n" + + " {\"name\": \"favorite_color\", \"type\": [\"string\", \"null\"]}\n" + + " ]\n" + + "}"; + Schema schema = (new Schema.Parser()).parse(schemaString); + + GenericRecord before = new GenericData.Record(schema); + before.put("name", "Bob"); + before.put("favorite_number", 256); + // Leave favorite_color null + + AvroCoder coder = AvroCoder.of(GenericRecord.class, schema); + byte[] bytes = CoderUtils.encodeToByteArray(coder, before); + GenericRecord after = CoderUtils.decodeFromByteArray(coder, bytes); + + Assert.assertEquals(before, after); + + Assert.assertEquals(schema, coder.getSchema()); + } + + @Test + public void testEncodingNotBuffered() throws Exception { + // This test ensures that the coder doesn't read ahead and buffer data. + // Reading ahead causes a problem if the stream consists of records of different + // types. + Pojo before = new Pojo("Hello", 42); + + AvroCoder coder = AvroCoder.of(Pojo.class); + SerializableCoder intCoder = SerializableCoder.of(Integer.class); + + ByteArrayOutputStream outStream = new ByteArrayOutputStream(); + + Context context = Context.NESTED; + coder.encode(before, outStream, context); + intCoder.encode(10, outStream, context); + + ByteArrayInputStream inStream = new ByteArrayInputStream(outStream.toByteArray()); + + Pojo after = coder.decode(inStream, context); + Assert.assertEquals(before, after); + + Integer intAfter = intCoder.decode(inStream, context); + Assert.assertEquals(new Integer(10), intAfter); + } + + @Test + public void testDefaultCoder() throws Exception { + Pipeline p = TestPipeline.create(); + + // Use MyRecord as input and output types without explicitly specifying + // a coder (this uses the default coders, which may not be AvroCoder). + PCollection output = + p.apply(Create.of(new Pojo("hello", 1), new Pojo("world", 2))) + .apply(ParDo.of(new GetTextFn())); + + DataflowAssert.that(output) + .containsInAnyOrder("hello", "world"); + p.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoderTest.java new file mode 100644 index 0000000000000..b6d2b3c657d04 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoderTest.java @@ -0,0 +1,70 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +import com.google.cloud.dataflow.sdk.util.common.CounterTestUtils; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +/** Unit tests for {@link ByteArrayCoder}. */ +@RunWith(JUnit4.class) +public class ByteArrayCoderTest { + @Test public void testOuterContext() throws CoderException, IOException { + byte[] buffer = {0xa, 0xb, 0xc}; + + ByteArrayOutputStream os = new ByteArrayOutputStream(); + ByteArrayCoder.of().encode(buffer, os, Coder.Context.OUTER); + byte[] encoded = os.toByteArray(); + + ByteArrayInputStream is = new ByteArrayInputStream(encoded); + byte[] decoded = ByteArrayCoder.of().decode(is, Coder.Context.OUTER); + assertThat(decoded, equalTo(buffer)); + } + + @Test public void testNestedContext() throws CoderException, IOException { + byte[][] buffers = {{0xa, 0xb, 0xc}, {}, {}, {0xd, 0xe}, {}}; + + ByteArrayOutputStream os = new ByteArrayOutputStream(); + for (byte[] buffer : buffers) { + ByteArrayCoder.of().encode(buffer, os, Coder.Context.NESTED); + } + byte[] encoded = os.toByteArray(); + + ByteArrayInputStream is = new ByteArrayInputStream(encoded); + for (byte[] buffer : buffers) { + byte[] decoded = ByteArrayCoder.of().decode(is, Coder.Context.NESTED); + assertThat(decoded, equalTo(buffer)); + } + } + + @Test public void testRegisterByteSizeObserver() throws Exception { + CounterTestUtils.testByteCount(ByteArrayCoder.of(), Coder.Context.OUTER, + new byte[][]{{ 0xa, 0xb, 0xc }}); + + CounterTestUtils.testByteCount(ByteArrayCoder.of(), Coder.Context.NESTED, + new byte[][]{{ 0xa, 0xb, 0xc }, {}, {}, { 0xd, 0xe }, {}}); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java new file mode 100644 index 0000000000000..ef096eb01c999 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java @@ -0,0 +1,73 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assume.assumeThat; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +/** + * Properties for use in {@link Coder} tests. These are implemented with junit assertions + * rather than as predicates for the sake of error messages. + */ +public class CoderProperties { + + /** + * Verifies that for the given {@link Coder}, {@link Coder.Context}, and values of + * type {@code T}, if the values are equal then the encoded bytes are equal. + */ + public static void coderDeterministic( + Coder coder, Coder.Context context, T value1, T value2) + throws Exception { + assumeThat(value1, equalTo(value2)); + assertArrayEquals(encode(coder, context, value1), encode(coder, context, value2)); + } + + /** + * Verifies that for the given {@link Coder}, {@link Coder.Context}, + * and value of type {@code T}, encoding followed by decoding yields an + * equal of type {@code T}. + */ + public static void coderDecodeEncodeEqual( + Coder coder, Coder.Context context, T value) + throws Exception { + assertEquals( + decode(coder, context, encode(coder, context, value)), + value); + } + + ////////////////////////////////////////////////////////////////////////// + + private static byte[] encode( + Coder coder, Coder.Context context, T value) throws CoderException, IOException { + ByteArrayOutputStream os = new ByteArrayOutputStream(); + coder.encode(value, os, context); + return os.toByteArray(); + } + + private static T decode( + Coder coder, Coder.Context context, byte[] bytes) throws CoderException, IOException { + ByteArrayInputStream is = new ByteArrayInputStream(bytes); + return coder.decode(is, context); + } + +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java new file mode 100644 index 0000000000000..ace3094827335 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java @@ -0,0 +1,230 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.reflect.TypeToken; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Tests for CoderRegistry. + */ +@RunWith(JUnit4.class) +public class CoderRegistryTest { + + public static CoderRegistry getStandardRegistry() { + CoderRegistry registry = new CoderRegistry(); + registry.registerStandardCoders(); + return registry; + } + + @Test + public void testRegisterInstantiatedGenericCoder() { + class MyValueList extends ArrayList { } + + CoderRegistry registry = new CoderRegistry(); + registry.registerCoder(MyValueList.class, ListCoder.of(MyValueCoder.of())); + assertEquals(registry.getDefaultCoder(MyValueList.class), ListCoder.of(MyValueCoder.of())); + } + + @Test + public void testSimpleDefaultCoder() { + CoderRegistry registry = getStandardRegistry(); + assertEquals(StringUtf8Coder.of(), registry.getDefaultCoder(String.class)); + assertEquals(null, registry.getDefaultCoder(UnknownType.class)); + } + + @Test + public void testTemplateDefaultCoder() { + CoderRegistry registry = getStandardRegistry(); + TypeToken> listToken = new TypeToken>() {}; + assertEquals(ListCoder.of(VarIntCoder.of()), + registry.getDefaultCoder(listToken)); + + registry.registerCoder(MyValue.class, MyValueCoder.class); + TypeToken>> kvToken = + new TypeToken>>() {}; + assertEquals(KvCoder.of(StringUtf8Coder.of(), + ListCoder.of(MyValueCoder.of())), + registry.getDefaultCoder(kvToken)); + + TypeToken> listUnknownToken = + new TypeToken>() {}; + assertEquals(null, registry.getDefaultCoder(listUnknownToken)); + } + + @Test + public void testTemplateInference() { + CoderRegistry registry = getStandardRegistry(); + MyTemplateClass> instance = + new MyTemplateClass>() {}; + Coder> expected = ListCoder.of(MyValueCoder.of()); + + // The map method operates on parameter names. + Map> coderMap = registry.getDefaultCoders( + instance.getClass(), + MyTemplateClass.class, + Collections.singletonMap("A", MyValueCoder.of())); + assertEquals(expected, coderMap.get("B")); + + // The array interface operates on position. + Coder[] coders = registry.getDefaultCoders( + instance.getClass(), + MyTemplateClass.class, + new Coder[] { MyValueCoder.of(), null }); + assertEquals(expected, coders[1]); + + // The "last argument" coder handles a common case. + Coder> actual = registry.getDefaultCoder( + instance.getClass(), + MyTemplateClass.class, + MyValueCoder.of()); + assertEquals(expected, actual); + + try { + registry.getDefaultCoder( + instance.getClass(), + MyTemplateClass.class, + BigEndianIntegerCoder.of()); + fail("should have failed"); + } catch (IllegalArgumentException exn) { + assertEquals("Cannot encode elements of type class " + + "com.google.cloud.dataflow.sdk.coders.CoderRegistryTest$MyValue " + + "with BigEndianIntegerCoder", exn.getMessage()); + } + } + + @Test + public void testGetDefaultCoderFromIntegerValue() { + CoderRegistry registry = getStandardRegistry(); + Integer i = 13; + Coder coder = registry.getDefaultCoder(i); + assertEquals(VarIntCoder.of(), coder); + } + + @Test + public void testGetDefaultCoderFromKvValue() { + CoderRegistry registry = getStandardRegistry(); + KV kv = KV.of(13, "hello"); + Coder> coder = registry.getDefaultCoder(kv); + assertEquals(KvCoder.of(VarIntCoder.of(), StringUtf8Coder.of()), + coder); + } + + @Test + public void testGetDefaultCoderFromNestedKvValue() { + CoderRegistry registry = getStandardRegistry(); + KV>> kv = KV.of(13, KV.of(17L, KV.of("hello", "goodbye"))); + Coder>>> coder = registry.getDefaultCoder(kv); + assertEquals( + KvCoder.of(VarIntCoder.of(), + KvCoder.of(VarLongCoder.of(), + KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()))), + coder); + } + + @Test + public void testTypeCompatibility() { + assertTrue(CoderRegistry.isCompatible( + BigEndianIntegerCoder.of(), Integer.class)); + assertFalse(CoderRegistry.isCompatible( + BigEndianIntegerCoder.of(), String.class)); + + assertFalse(CoderRegistry.isCompatible( + ListCoder.of(BigEndianIntegerCoder.of()), Integer.class)); + assertTrue(CoderRegistry.isCompatible( + ListCoder.of(BigEndianIntegerCoder.of()), + new TypeToken>() {}.getType())); + assertFalse(CoderRegistry.isCompatible( + ListCoder.of(BigEndianIntegerCoder.of()), + new TypeToken>() {}.getType())); + } + + static class MyTemplateClass { } + + static class MyValue { } + + static class MyValueCoder implements Coder { + + private static final MyValueCoder INSTANCE = new MyValueCoder(); + + public static MyValueCoder of() { + return INSTANCE; + } + + public static List getInstanceComponents(MyValue exampleValue) { + return Arrays.asList(); + } + + @Override + public void encode(MyValue value, OutputStream outStream, Context context) + throws CoderException, IOException { + } + + @Override + public MyValue decode(InputStream inStream, Context context) + throws CoderException, IOException { + return new MyValue(); + } + + @Override + public List> getCoderArguments() { + return null; + } + + @Override + public CloudObject asCloudObject() { + return null; + } + + @Override + public boolean isDeterministic() { return true; } + + @Override + public boolean isRegisterByteSizeObserverCheap(MyValue value, Context context) { + return true; + } + + @Override + public void registerByteSizeObserver( + MyValue value, ElementByteSizeObserver observer, Context context) + throws Exception { + observer.update(0L); + } + } + + static class UnknownType { } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CustomCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CustomCoderTest.java new file mode 100644 index 0000000000000..e532d44dc66b5 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CustomCoderTest.java @@ -0,0 +1,83 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.values.KV; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** Unit tests for {@link CustomCoder}. */ +@RunWith(JUnit4.class) +public class CustomCoderTest { + + private static class MyCustomCoder extends CustomCoder> { + private final String key; + + public MyCustomCoder(String key) { + this.key = key; + } + + @Override + public void encode(KV kv, OutputStream out, Context context) + throws IOException { + new DataOutputStream(out).writeLong(kv.getValue()); + } + + @Override + public KV decode(InputStream inStream, Context context) + throws IOException { + return KV.of(key, new DataInputStream(inStream).readLong()); + } + + @Override + public boolean equals(Object other) { + return other instanceof MyCustomCoder + && key.equals(((MyCustomCoder) other).key); + } + + @Override + public int hashCode() { + return key.hashCode(); + } + } + + @Test public void testEncodeDecode() throws Exception { + MyCustomCoder coder = new MyCustomCoder("key"); + byte[] encoded = CoderUtils.encodeToByteArray(coder, KV.of("key", 3L)); + Assert.assertEquals( + KV.of("key", 3L), CoderUtils.decodeFromByteArray(coder, encoded)); + + byte[] encoded2 = CoderUtils.encodeToByteArray(coder, KV.of("ignored", 3L)); + Assert.assertEquals( + KV.of("key", 3L), CoderUtils.decodeFromByteArray(coder, encoded2)); + } + + @Test public void testEncodable() throws Exception { + SerializableUtils.ensureSerializable(new MyCustomCoder("key")); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/DefaultCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/DefaultCoderTest.java new file mode 100644 index 0000000000000..769d1e6fb144f --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/DefaultCoderTest.java @@ -0,0 +1,93 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.api.client.util.Preconditions; +import com.google.common.reflect.TypeToken; + +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; + +/** + * Tests of Coder defaults. + */ +@RunWith(JUnit4.class) +public class DefaultCoderTest { + + @DefaultCoder(AvroCoder.class) + private static class AvroRecord { + } + + private static class SerializableBase implements Serializable { + } + + @DefaultCoder(SerializableCoder.class) + private static class SerializableRecord extends SerializableBase { + } + + @DefaultCoder(CustomCoder.class) + private static class CustomRecord extends SerializableBase { + } + + private static class Unknown { + } + + private static class CustomCoder extends SerializableCoder { + // Extending SerializableCoder isn't trivial, but it can be done. + @SuppressWarnings("unchecked") + public static SerializableCoder of(Class recordType) { + Preconditions.checkArgument( + CustomRecord.class.isAssignableFrom(recordType)); + return (SerializableCoder) new CustomCoder(); + } + + protected CustomCoder() { + super(CustomRecord.class); + } + } + + @Test + public void testDefaultCoders() throws Exception { + checkDefault(AvroRecord.class, AvroCoder.class); + checkDefault(SerializableBase.class, SerializableCoder.class); + checkDefault(SerializableRecord.class, SerializableCoder.class); + checkDefault(CustomRecord.class, CustomCoder.class); + } + + @Test + public void testUnknown() throws Exception { + CoderRegistry registery = new CoderRegistry(); + Coder coderType = registery.getDefaultCoder(Unknown.class); + Assert.assertNull(coderType); + } + + /** + * Checks that the default Coder for {@code valueType} is an instance of + * {@code expectedCoder}. + */ + private void checkDefault(Class valueType, + Class expectedCoder) { + CoderRegistry registry = new CoderRegistry(); + Coder coder = registry.getDefaultCoder(TypeToken.of(valueType)); + Assert.assertThat(coder, Matchers.instanceOf(expectedCoder)); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/InstantCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/InstantCoderTest.java new file mode 100644 index 0000000000000..dd719004eab1c --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/InstantCoderTest.java @@ -0,0 +1,67 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.common.primitives.UnsignedBytes; + +import org.joda.time.Instant; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** Unit tests for {@link InstantCoder}. */ +@RunWith(JUnit4.class) +public class InstantCoderTest { + private final InstantCoder coder = InstantCoder.of(); + private final List timestamps = + Arrays.asList(0L, 1L, -1L, -255L, 256L, Long.MIN_VALUE, Long.MAX_VALUE); + + @Test + public void testBasicEncoding() throws Exception { + for (long timestamp : timestamps) { + Assert.assertEquals(new Instant(timestamp), + CoderUtils.decodeFromByteArray(coder, + CoderUtils.encodeToByteArray(coder, new Instant(timestamp)))); + } + } + + @Test + public void testOrderedEncoding() throws Exception { + List sortedTimestamps = new ArrayList<>(timestamps); + Collections.sort(sortedTimestamps); + + List encodings = new ArrayList<>(sortedTimestamps.size()); + for (long timestamp : sortedTimestamps) { + encodings.add(CoderUtils.encodeToByteArray(coder, new Instant(timestamp))); + } + + // Verify that the encodings were already sorted, since they were generated + // in the correct order. + List sortedEncodings = new ArrayList<>(encodings); + Collections.sort(sortedEncodings, UnsignedBytes.lexicographicalComparator()); + + Assert.assertEquals(encodings, sortedEncodings); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/IterableCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/IterableCoderTest.java new file mode 100644 index 0000000000000..993c5d0a5e917 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/IterableCoderTest.java @@ -0,0 +1,46 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** Unit tests for {@link IterableCoder}. */ +@RunWith(JUnit4.class) +public class IterableCoderTest { + @Test + public void testGetInstanceComponentsNonempty() { + Iterable iterable = Arrays.asList(2, 58, 99, 5); + List components = IterableCoder.getInstanceComponents(iterable); + assertEquals(1, components.size()); + assertEquals(2, components.get(0)); + } + + @Test + public void testGetInstanceComponentsEmpty() { + Iterable iterable = Arrays.asList(); + List components = IterableCoder.getInstanceComponents(iterable); + assertNull(components); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ListCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ListCoderTest.java new file mode 100644 index 0000000000000..c04d3e16745b8 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ListCoderTest.java @@ -0,0 +1,46 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** Unit tests for {@link ListCoder}. */ +@RunWith(JUnit4.class) +public class ListCoderTest { + @Test + public void testGetInstanceComponentsNonempty() { + List list = Arrays.asList(21, 5, 3, 5); + List components = ListCoder.getInstanceComponents(list); + assertEquals(1, components.size()); + assertEquals(21, components.get(0)); + } + + @Test + public void testGetInstanceComponentsEmpty() { + List list = Arrays.asList(); + List components = ListCoder.getInstanceComponents(list); + assertNull(components); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/MapCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/MapCoderTest.java new file mode 100644 index 0000000000000..30cd0d8e8100b --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/MapCoderTest.java @@ -0,0 +1,49 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** Unit tests for {@link MapCoder}. */ +@RunWith(JUnit4.class) +public class MapCoderTest { + @Test + public void testGetInstanceComponentsNonempty() { + Map map = new HashMap<>(); + map.put(17, "foozle"); + List components = MapCoder.getInstanceComponents(map); + assertEquals(2, components.size()); + assertEquals(17, components.get(0)); + assertEquals("foozle", components.get(1)); + } + + @Test + public void testGetInstanceComponentsEmpty() { + Map map = new HashMap<>(); + List components = MapCoder.getInstanceComponents(map); + assertNull(components); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/SerializableCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/SerializableCoderTest.java new file mode 100644 index 0000000000000..3e56832a3faad --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/SerializableCoderTest.java @@ -0,0 +1,182 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static org.junit.Assert.assertEquals; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.Serializer; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; + +/** + * Tests SerializableCoder. + */ +@RunWith(JUnit4.class) +public class SerializableCoderTest implements Serializable { + + @DefaultCoder(SerializableCoder.class) + static class MyRecord implements Serializable { + public String value; + + public MyRecord(String value) { + this.value = value; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + MyRecord myRecord = (MyRecord) o; + return value.equals(myRecord.value); + } + + @Override + public int hashCode() { + return value.hashCode(); + } + } + + static class StringToRecord extends DoFn { + @Override + public void processElement(ProcessContext c) { + c.output(new MyRecord(c.element())); + } + } + + static class RecordToString extends DoFn { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().value); + } + } + + static final List LINES = Arrays.asList( + "To be,", + "or not to be"); + + @Test + public void testSerializableCoder() throws Exception { + IterableCoder coder = IterableCoder + .of(SerializableCoder.of(MyRecord.class)); + + List records = new LinkedList<>(); + for (String l : LINES) { + records.add(new MyRecord(l)); + } + + byte[] encoded = CoderUtils.encodeToByteArray(coder, records); + Iterable decoded = CoderUtils.decodeFromByteArray(coder, encoded); + + assertEquals(records, decoded); + } + + @Test + public void testSerializableCoderConstruction() throws Exception { + SerializableCoder coder = SerializableCoder.of(MyRecord.class); + assertEquals(coder.getRecordType(), MyRecord.class); + + CloudObject encoding = coder.asCloudObject(); + Assert.assertThat(encoding.getClassName(), + Matchers.containsString(SerializableCoder.class.getSimpleName())); + + Coder decoded = Serializer.deserialize(encoding, Coder.class); + Assert.assertThat(decoded, Matchers.instanceOf(SerializableCoder.class)); + } + + @Test + public void testDefaultCoder() throws Exception { + Pipeline p = TestPipeline.create(); + + // Use MyRecord as input and output types without explicitly specifying + // a coder (this uses the default coders, which may not be + // SerializableCoder). + PCollection output = + p.apply(Create.of("Hello", "World")) + .apply(ParDo.of(new StringToRecord())) + .apply(ParDo.of(new RecordToString())); + + DataflowAssert.that(output) + .containsInAnyOrder("Hello", "World"); + } + + @Test + public void testLongStringEncoding() throws Exception { + StringUtf8Coder coder = StringUtf8Coder.of(); + + // Java's DataOutputStream.writeUTF fails at 64k, so test well beyond that. + char[] chars = new char[100 * 1024]; + Arrays.fill(chars, 'o'); + String source = new String(chars); + + // Verify OUTER encoding. + assertEquals(source, CoderUtils.decodeFromByteArray(coder, + CoderUtils.encodeToByteArray(coder, source))); + + // Second string uses a UTF8 character. Each codepoint is translated into + // 4 characters in UTF8. + int[] codePoints = new int[20 * 1024]; + Arrays.fill(codePoints, 0x1D50A); // "MATHEMATICAL_FRAKTUR_CAPITAL_G" + String source2 = new String(codePoints, 0, codePoints.length); + + // Verify OUTER encoding. + assertEquals(source2, CoderUtils.decodeFromByteArray(coder, + CoderUtils.encodeToByteArray(coder, source2))); + + + // Encode both strings into NESTED form. + byte[] nestedEncoding; + try (ByteArrayOutputStream os = new ByteArrayOutputStream()) { + coder.encode(source, os, Coder.Context.NESTED); + coder.encode(source2, os, Coder.Context.NESTED); + nestedEncoding = os.toByteArray(); + } + + // Decode from NESTED form. + try (ByteArrayInputStream is = new ByteArrayInputStream(nestedEncoding)) { + String result = coder.decode(is, Coder.Context.NESTED); + String result2 = coder.decode(is, Coder.Context.NESTED); + assertEquals(0, is.available()); + assertEquals(source, result); + assertEquals(source2, result2); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/URICoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/URICoderTest.java new file mode 100644 index 0000000000000..f464e813bf5d4 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/URICoderTest.java @@ -0,0 +1,68 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.net.URI; +import java.util.Arrays; +import java.util.List; + +/** Unit tests for {@link URICoder}. */ +@RunWith(JUnit4.class) +public class URICoderTest { + + // Test data + + private static final List TEST_URI_STRINGS = Arrays.asList( + "http://www.example.com", + "gs://myproject/mybucket/a/gcs/path", + "/just/a/path", + "file:/path/with/no/authority", + "file:///path/with/empty/authority"); + + private static final List TEST_CONTEXTS = Arrays.asList( + Coder.Context.OUTER, + Coder.Context.NESTED); + + // Tests + + @Test + public void testDeterministic() throws Exception { + Coder coder = URICoder.of(); + + for (String uriString : TEST_URI_STRINGS) { + for (Coder.Context context : TEST_CONTEXTS) { + // Obviously equal, but distinct as objects + CoderProperties.coderDeterministic(coder, context, new URI(uriString), new URI(uriString)); + } + } + } + + @Test + public void testDecodeEncodeEqual() throws Exception { + Coder coder = URICoder.of(); + + for (String uriString : TEST_URI_STRINGS) { + for (Coder.Context context : TEST_CONTEXTS) { + CoderProperties.coderDecodeEncodeEqual(coder, context, new URI(uriString)); + } + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java new file mode 100644 index 0000000000000..ad6f16567e92c --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java @@ -0,0 +1,365 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.runners.DirectPipeline; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.EvaluationResults; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PDone; + +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.avro.specific.SpecificDatumWriter; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Tests for AvroIO Read and Write transforms. + */ +@RunWith(JUnit4.class) +public class AvroIOTest { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + private File avroFile; + + @Before + public void prepareAvroFileBeforeAnyTest() throws IOException { + avroFile = tmpFolder.newFile("file.avro"); + } + + private final String schemaString = + "{\"namespace\": \"example.avro\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"User\",\n" + + " \"fields\": [\n" + + " {\"name\": \"name\", \"type\": \"string\"},\n" + + " {\"name\": \"favorite_number\", \"type\": [\"int\", \"null\"]},\n" + + " {\"name\": \"favorite_color\", \"type\": [\"string\", \"null\"]}\n" + + " ]\n" + + "}"; + private final Schema.Parser parser = new Schema.Parser(); + private final Schema schema = parser.parse(schemaString); + + private User[] generateAvroObjects() { + User user1 = new User(); + user1.setName("Bob"); + user1.setFavoriteNumber(256); + + User user2 = new User(); + user2.setName("Alice"); + user2.setFavoriteNumber(128); + + User user3 = new User(); + user3.setName("Ted"); + user3.setFavoriteColor("white"); + + return new User[] { user1, user2, user3 }; + } + + private GenericRecord[] generateAvroGenericRecords() { + GenericRecord user1 = new GenericData.Record(schema); + user1.put("name", "Bob"); + user1.put("favorite_number", 256); + + GenericRecord user2 = new GenericData.Record(schema); + user2.put("name", "Alice"); + user2.put("favorite_number", 128); + + GenericRecord user3 = new GenericData.Record(schema); + user3.put("name", "Ted"); + user3.put("favorite_color", "white"); + + return new GenericRecord[] { user1, user2, user3 }; + } + + private void generateAvroFile(User[] elements) throws IOException { + DatumWriter userDatumWriter = new SpecificDatumWriter<>(User.class); + DataFileWriter dataFileWriter = new DataFileWriter<>(userDatumWriter); + dataFileWriter.create(elements[0].getSchema(), avroFile); + for (User user : elements) { + dataFileWriter.append(user); + } + dataFileWriter.close(); + } + + private List readAvroFile() throws IOException { + DatumReader userDatumReader = new SpecificDatumReader<>(User.class); + DataFileReader dataFileReader = new DataFileReader<>(avroFile, userDatumReader); + List users = new ArrayList<>(); + while (dataFileReader.hasNext()) { + users.add(dataFileReader.next()); + } + return users; + } + + void runTestRead(AvroIO.Read.Bound read, String expectedName, T[] expectedOutput) + throws Exception { + generateAvroFile(generateAvroObjects()); + + DirectPipeline p = DirectPipeline.createForTest(); + PCollection output = p.apply(read); + EvaluationResults results = p.run(); + assertEquals(expectedName, output.getName()); + assertThat(results.getPCollection(output), + containsInAnyOrder(expectedOutput)); + } + + @Test + public void testReadFromGeneratedClass() throws Exception { + runTestRead(AvroIO.Read.from(avroFile.getPath()) + .withSchema(User.class), + "AvroIO.Read.out", generateAvroObjects()); + runTestRead(AvroIO.Read.withSchema(User.class) + .from(avroFile.getPath()), + "AvroIO.Read.out", generateAvroObjects()); + runTestRead(AvroIO.Read.named("MyRead") + .from(avroFile.getPath()) + .withSchema(User.class), + "MyRead.out", generateAvroObjects()); + runTestRead(AvroIO.Read.named("MyRead") + .withSchema(User.class) + .from(avroFile.getPath()), + "MyRead.out", generateAvroObjects()); + runTestRead(AvroIO.Read.from(avroFile.getPath()) + .withSchema(User.class) + .named("HerRead"), + "HerRead.out", generateAvroObjects()); + runTestRead(AvroIO.Read.from(avroFile.getPath()) + .named("HerRead") + .withSchema(User.class), + "HerRead.out", generateAvroObjects()); + runTestRead(AvroIO.Read.withSchema(User.class) + .named("HerRead") + .from(avroFile.getPath()), + "HerRead.out", generateAvroObjects()); + runTestRead(AvroIO.Read.withSchema(User.class) + .from(avroFile.getPath()) + .named("HerRead"), + "HerRead.out", generateAvroObjects()); + } + + @Test + public void testReadFromSchema() throws Exception { + runTestRead(AvroIO.Read.from(avroFile.getPath()) + .withSchema(schema), + "AvroIO.Read.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.withSchema(schema) + .from(avroFile.getPath()), + "AvroIO.Read.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.named("MyRead") + .from(avroFile.getPath()) + .withSchema(schema), + "MyRead.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.named("MyRead") + .withSchema(schema) + .from(avroFile.getPath()), + "MyRead.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.from(avroFile.getPath()) + .withSchema(schema) + .named("HerRead"), + "HerRead.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.from(avroFile.getPath()) + .named("HerRead") + .withSchema(schema), + "HerRead.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.withSchema(schema) + .named("HerRead") + .from(avroFile.getPath()), + "HerRead.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.withSchema(schema) + .from(avroFile.getPath()) + .named("HerRead"), + "HerRead.out", generateAvroGenericRecords()); + } + + @Test + public void testReadFromSchemaString() throws Exception { + runTestRead(AvroIO.Read.from(avroFile.getPath()) + .withSchema(schemaString), + "AvroIO.Read.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.withSchema(schemaString) + .from(avroFile.getPath()), + "AvroIO.Read.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.named("MyRead") + .from(avroFile.getPath()) + .withSchema(schemaString), + "MyRead.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.named("MyRead") + .withSchema(schemaString) + .from(avroFile.getPath()), + "MyRead.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.from(avroFile.getPath()) + .withSchema(schemaString) + .named("HerRead"), + "HerRead.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.from(avroFile.getPath()) + .named("HerRead") + .withSchema(schemaString), + "HerRead.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.withSchema(schemaString) + .named("HerRead") + .from(avroFile.getPath()), + "HerRead.out", generateAvroGenericRecords()); + runTestRead(AvroIO.Read.withSchema(schemaString) + .from(avroFile.getPath()) + .named("HerRead"), + "HerRead.out", generateAvroGenericRecords()); + } + + void runTestWrite(AvroIO.Write.Bound write, String expectedName) + throws Exception { + User[] users = generateAvroObjects(); + + DirectPipeline p = DirectPipeline.createForTest(); + PCollection input = p.apply(Create.of(Arrays.asList((T[]) users))) + .setCoder((Coder) AvroCoder.of(User.class)); + PDone output = input.apply(write.withoutSharding()); + EvaluationResults results = p.run(); + assertEquals(expectedName, write.getName()); + + assertThat(readAvroFile(), containsInAnyOrder(users)); + } + + @Test + public void testWriteFromGeneratedClass() throws Exception { + runTestWrite(AvroIO.Write.to(avroFile.getPath()) + .withSchema(User.class), + "AvroIO.Write"); + runTestWrite(AvroIO.Write.withSchema(User.class) + .to(avroFile.getPath()), + "AvroIO.Write"); + runTestWrite(AvroIO.Write.named("MyWrite") + .to(avroFile.getPath()) + .withSchema(User.class), + "MyWrite"); + runTestWrite(AvroIO.Write.named("MyWrite") + .withSchema(User.class) + .to(avroFile.getPath()), + "MyWrite"); + runTestWrite(AvroIO.Write.to(avroFile.getPath()) + .withSchema(User.class) + .named("HerWrite"), + "HerWrite"); + runTestWrite(AvroIO.Write.to(avroFile.getPath()) + .named("HerWrite") + .withSchema(User.class), + "HerWrite"); + runTestWrite(AvroIO.Write.withSchema(User.class) + .named("HerWrite") + .to(avroFile.getPath()), + "HerWrite"); + runTestWrite(AvroIO.Write.withSchema(User.class) + .to(avroFile.getPath()) + .named("HerWrite"), + "HerWrite"); + } + + @Test + public void testWriteFromSchema() throws Exception { + runTestWrite(AvroIO.Write.to(avroFile.getPath()) + .withSchema(schema), + "AvroIO.Write"); + runTestWrite(AvroIO.Write.withSchema(schema) + .to(avroFile.getPath()), + "AvroIO.Write"); + runTestWrite(AvroIO.Write.named("MyWrite") + .to(avroFile.getPath()) + .withSchema(schema), + "MyWrite"); + runTestWrite(AvroIO.Write.named("MyWrite") + .withSchema(schema) + .to(avroFile.getPath()), + "MyWrite"); + runTestWrite(AvroIO.Write.to(avroFile.getPath()) + .withSchema(schema) + .named("HerWrite"), + "HerWrite"); + runTestWrite(AvroIO.Write.to(avroFile.getPath()) + .named("HerWrite") + .withSchema(schema), + "HerWrite"); + runTestWrite(AvroIO.Write.withSchema(schema) + .named("HerWrite") + .to(avroFile.getPath()), + "HerWrite"); + runTestWrite(AvroIO.Write.withSchema(schema) + .to(avroFile.getPath()) + .named("HerWrite"), + "HerWrite"); + } + + @Test + public void testWriteFromSchemaString() throws Exception { + runTestWrite(AvroIO.Write.to(avroFile.getPath()) + .withSchema(schemaString), + "AvroIO.Write"); + runTestWrite(AvroIO.Write.withSchema(schemaString) + .to(avroFile.getPath()), + "AvroIO.Write"); + runTestWrite(AvroIO.Write.named("MyWrite") + .to(avroFile.getPath()) + .withSchema(schemaString), + "MyWrite"); + runTestWrite(AvroIO.Write.named("MyWrite") + .withSchema(schemaString) + .to(avroFile.getPath()), + "MyWrite"); + runTestWrite(AvroIO.Write.to(avroFile.getPath()) + .withSchema(schemaString) + .named("HerWrite"), + "HerWrite"); + runTestWrite(AvroIO.Write.to(avroFile.getPath()) + .named("HerWrite") + .withSchema(schemaString), + "HerWrite"); + runTestWrite(AvroIO.Write.withSchema(schemaString) + .named("HerWrite") + .to(avroFile.getPath()), + "HerWrite"); + runTestWrite(AvroIO.Write.withSchema(schemaString) + .to(avroFile.getPath()) + .named("HerWrite"), + "HerWrite"); + } + + // TODO: for Write only, test withSuffix, withNumShards, + // withShardNameTemplate and withoutSharding. +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BigQueryIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BigQueryIOTest.java new file mode 100644 index 0000000000000..863e260282a34 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BigQueryIOTest.java @@ -0,0 +1,307 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import static org.junit.Assert.assertEquals; + +import com.google.api.client.util.Data; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder; +import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition; +import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.WriteDisposition; +import com.google.cloud.dataflow.sdk.options.BigQueryOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.util.CoderUtils; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; + +/** + * Tests for BigQueryIO. + */ +@RunWith(JUnit4.class) +public class BigQueryIOTest { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + private void checkReadObject( + BigQueryIO.Read.Bound bound, String project, String dataset, String table) { + checkReadObjectWithValidate(bound, project, dataset, table, true); + } + + private void checkReadObjectWithValidate( + BigQueryIO.Read.Bound bound, String project, String dataset, String table, boolean validate) { + assertEquals(project, bound.table.getProjectId()); + assertEquals(dataset, bound.table.getDatasetId()); + assertEquals(table, bound.table.getTableId()); + assertEquals(validate, bound.validate); + } + + private void checkWriteObject( + BigQueryIO.Write.Bound bound, String project, String dataset, String table, + TableSchema schema, CreateDisposition createDisposition, + WriteDisposition writeDisposition) { + checkWriteObjectWithValidate( + bound, project, dataset, table, schema, createDisposition, writeDisposition, true); + } + + private void checkWriteObjectWithValidate( + BigQueryIO.Write.Bound bound, String project, String dataset, String table, + TableSchema schema, CreateDisposition createDisposition, + WriteDisposition writeDisposition, boolean validate) { + assertEquals(project, bound.table.getProjectId()); + assertEquals(dataset, bound.table.getDatasetId()); + assertEquals(table, bound.table.getTableId()); + assertEquals(schema, bound.schema); + assertEquals(createDisposition, bound.createDisposition); + assertEquals(writeDisposition, bound.writeDisposition); + assertEquals(validate, bound.validate); + } + + @Before + public void setUp() { + BigQueryOptions options = PipelineOptionsFactory.as(BigQueryOptions.class); + options.setProject("defaultProject"); + } + + @Test + public void testBuildSource() throws IOException { + BigQueryIO.Read.Bound bound = BigQueryIO.Read.named("ReadMyTable") + .from("foo.com:project:somedataset.sometable"); + checkReadObject(bound, "foo.com:project", "somedataset", "sometable"); + } + + @Test + public void testBuildSourcewithoutValidation() throws IOException { + // This test just checks that using withoutValidation will not trigger object + // construction errors. + BigQueryIO.Read.Bound bound = BigQueryIO.Read.named("ReadMyTable") + .from("foo.com:project:somedataset.sometable").withoutValidation(); + checkReadObjectWithValidate(bound, "foo.com:project", "somedataset", "sometable", false); + } + + @Test + public void testBuildSourceWithDefaultProject() throws IOException { + BigQueryIO.Read.Bound bound = BigQueryIO.Read.named("ReadMyTable") + .from("somedataset.sometable"); + checkReadObject(bound, null, "somedataset", "sometable"); + } + + @Test + public void testBuildSourceWithTableReference() throws IOException { + TableReference table = new TableReference() + .setProjectId("foo.com:project") + .setDatasetId("somedataset") + .setTableId("sometable"); + BigQueryIO.Read.Bound bound = BigQueryIO.Read.named("ReadMyTable") + .from(table); + checkReadObject(bound, "foo.com:project", "somedataset", "sometable"); + } + + @Test(expected = IllegalStateException.class) + public void testBuildSourceWithoutTable() throws IOException { + Pipeline p = TestPipeline.create(); + p.apply(BigQueryIO.Read.named("ReadMyTable")); + } + + @Test + public void testBuildSink() throws IOException { + BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable") + .to("foo.com:project:somedataset.sometable"); + checkWriteObject( + bound, "foo.com:project", "somedataset", "sometable", + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY); + } + + @Test + public void testBuildSinkwithoutValidation() throws IOException { + // This test just checks that using withoutValidation will not trigger object + // construction errors. + BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable") + .to("foo.com:project:somedataset.sometable").withoutValidation(); + checkWriteObjectWithValidate( + bound, "foo.com:project", "somedataset", "sometable", + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, false); + } + + @Test + public void testBuildSinkDefaultProject() throws IOException { + BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable") + .to("somedataset.sometable"); + checkWriteObject( + bound, null, "somedataset", "sometable", + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY); + } + + @Test + public void testBuildSinkWithTableReference() throws IOException { + TableReference table = new TableReference() + .setProjectId("foo.com:project") + .setDatasetId("somedataset") + .setTableId("sometable"); + BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable") + .to(table); + checkWriteObject( + bound, "foo.com:project", "somedataset", "sometable", + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY); + } + + @Test(expected = IllegalStateException.class) + public void testBuildSinkWithoutTable() throws IOException { + Pipeline p = TestPipeline.create(); + p.apply(Create.of()).setCoder(TableRowJsonCoder.of()) + .apply(BigQueryIO.Write.named("WriteMyTable")); + } + + @Test + public void testBuildSinkWithSchema() throws IOException { + TableSchema schema = new TableSchema(); + BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable") + .to("foo.com:project:somedataset.sometable").withSchema(schema); + checkWriteObject( + bound, "foo.com:project", "somedataset", "sometable", + schema, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY); + } + + @Test + public void testBuildSinkWithCreateDispositionNever() throws IOException { + BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable") + .to("foo.com:project:somedataset.sometable") + .withCreateDisposition(CreateDisposition.CREATE_NEVER); + checkWriteObject( + bound, "foo.com:project", "somedataset", "sometable", + null, CreateDisposition.CREATE_NEVER, WriteDisposition.WRITE_EMPTY); + } + + @Test + public void testBuildSinkWithCreateDispositionIfNeeded() throws IOException { + BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable") + .to("foo.com:project:somedataset.sometable") + .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED); + checkWriteObject( + bound, "foo.com:project", "somedataset", "sometable", + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY); + } + + @Test + public void testBuildSinkWithWriteDispositionTruncate() throws IOException { + BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable") + .to("foo.com:project:somedataset.sometable") + .withWriteDisposition(WriteDisposition.WRITE_TRUNCATE); + checkWriteObject( + bound, "foo.com:project", "somedataset", "sometable", + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_TRUNCATE); + } + + @Test + public void testBuildSinkWithWriteDispositionAppend() throws IOException { + BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable") + .to("foo.com:project:somedataset.sometable") + .withWriteDisposition(WriteDisposition.WRITE_APPEND); + checkWriteObject( + bound, "foo.com:project", "somedataset", "sometable", + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_APPEND); + } + + @Test + public void testBuildSinkWithWriteDispositionEmpty() throws IOException { + BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable") + .to("foo.com:project:somedataset.sometable") + .withWriteDisposition(WriteDisposition.WRITE_EMPTY); + checkWriteObject( + bound, "foo.com:project", "somedataset", "sometable", + null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY); + } + + @Test + public void testTableParsing() { + TableReference ref = BigQueryIO + .parseTableSpec("my-project:data_set.table_name"); + Assert.assertEquals("my-project", ref.getProjectId()); + Assert.assertEquals("data_set", ref.getDatasetId()); + Assert.assertEquals("table_name", ref.getTableId()); + } + + @Test + public void testTableParsing_validPatterns() { + BigQueryIO.parseTableSpec("a123-456:foo_bar.d"); + BigQueryIO.parseTableSpec("a12345:b.c"); + BigQueryIO.parseTableSpec("b12345.c"); + } + + @Test + public void testTableParsing_noProjectId() { + TableReference ref = BigQueryIO + .parseTableSpec("data_set.table_name"); + Assert.assertEquals(null, ref.getProjectId()); + Assert.assertEquals("data_set", ref.getDatasetId()); + Assert.assertEquals("table_name", ref.getTableId()); + } + + @Test + public void testTableParsingError() { + thrown.expect(IllegalArgumentException.class); + BigQueryIO.parseTableSpec("0123456:foo.bar"); + } + + @Test + public void testTableParsingError_2() { + thrown.expect(IllegalArgumentException.class); + BigQueryIO.parseTableSpec("myproject:.bar"); + } + + @Test + public void testTableParsingError_3() { + thrown.expect(IllegalArgumentException.class); + BigQueryIO.parseTableSpec(":a.b"); + } + + @Test + public void testTableParsingError_slash() { + thrown.expect(IllegalArgumentException.class); + BigQueryIO.parseTableSpec("a\\b12345:c.d"); + } + + // Test that BigQuery's special null placeholder objects can be encoded. + @Test + public void testCoder_nullCell() throws CoderException { + TableRow row = new TableRow(); + row.set("temperature", Data.nullOf(Object.class)); + row.set("max_temperature", Data.nullOf(Object.class)); + + byte[] bytes = CoderUtils.encodeToByteArray(TableRowJsonCoder.of(), row); + + TableRow newRow = CoderUtils.decodeFromByteArray(TableRowJsonCoder.of(), bytes); + byte[] newBytes = CoderUtils.encodeToByteArray(TableRowJsonCoder.of(), newRow); + + Assert.assertArrayEquals(bytes, newBytes); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java new file mode 100644 index 0000000000000..e026c58102dae --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java @@ -0,0 +1,126 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import static org.junit.Assert.assertEquals; + +import com.google.api.services.datastore.DatastoreV1.Entity; +import com.google.api.services.datastore.DatastoreV1.Query; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.EntityCoder; +import com.google.cloud.dataflow.sdk.runners.DirectPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; + +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for DatastoreIO Read and Write transforms. + */ +@RunWith(JUnit4.class) +public class DatastoreIOTest { + + private String host; + private String datasetId; + private Query query; + + /** + * Sets the default dataset ID as "shakespearedataset", + * which contains two kinds of records: "food" and "shakespeare". + * The "food" table contains 10 manually constructed entities, + * The "shakespeare" table contains 172948 entities, + * where each entity represents one line in one play in + * Shakespeare collections (e.g. there are 172948 lines in + * all Shakespeare files). + * + *

The function also sets up the datastore agent by creating + * a Datastore object to access the dataset shakespeareddataset. + * + *

Note that the local server must be started to let the agent + * be created normally. + */ + @Before + public void setUp() { + this.host = "http://localhost:1234"; + this.datasetId = "shakespearedataset"; + + Query.Builder q = Query.newBuilder(); + q.addKindBuilder().setName("shakespeare"); + this.query = q.build(); + } + + /** + * Test for reading one entity from kind "food" + */ + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testBuildRead() throws Exception { + DatastoreIO.Read.Bound readQuery = DatastoreIO.Read + .withHost(this.host) + .from(this.datasetId, this.query); + assertEquals(this.query, readQuery.query); + assertEquals(this.datasetId, readQuery.datasetId); + assertEquals(this.host, readQuery.host); + } + + @Test + public void testBuildReadAlt() throws Exception { + DatastoreIO.Read.Bound readQuery = DatastoreIO.Read + .from(this.datasetId, this.query) + .withHost(this.host); + assertEquals(this.query, readQuery.query); + assertEquals(this.datasetId, readQuery.datasetId); + assertEquals(this.host, readQuery.host); + } + + @Test(expected = IllegalStateException.class) + public void testBuildReadWithoutDatastoreSettingToCatchException() + throws Exception { + // create pipeline and run the pipeline to get result + Pipeline p = DirectPipeline.createForTest(); + p.apply(DatastoreIO.Read.named("ReadDatastore")); + } + + @Test + public void testBuildWrite() throws Exception { + DatastoreIO.Write.Bound write = DatastoreIO.Write + .to(this.datasetId) + .withHost(this.host); + assertEquals(this.host, write.host); + assertEquals(this.datasetId, write.datasetId); + } + + @Test + public void testBuildWriteAlt() throws Exception { + DatastoreIO.Write.Bound write = DatastoreIO.Write + .withHost(this.host) + .to(this.datasetId); + assertEquals(this.host, write.host); + assertEquals(this.datasetId, write.datasetId); + } + + @Test(expected = IllegalStateException.class) + public void testBuildWriteWithoutDatastoreToCatchException() throws Exception { + // create pipeline and run the pipeline to get result + Pipeline p = DirectPipeline.createForTest(); + p.apply(Create.of()).setCoder(EntityCoder.of()) + .apply(DatastoreIO.Write.named("WriteDatastore")); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java new file mode 100644 index 0000000000000..b6aaf59b51ad8 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java @@ -0,0 +1,413 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import static com.google.cloud.dataflow.sdk.TestUtils.INTS_ARRAY; +import static com.google.cloud.dataflow.sdk.TestUtils.LINES; +import static com.google.cloud.dataflow.sdk.TestUtils.LINES_ARRAY; +import static com.google.cloud.dataflow.sdk.TestUtils.NO_INTS_ARRAY; +import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES_ARRAY; +import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.TextualIntegerCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.DirectPipeline; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.EvaluationResults; +import com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.GcsUtil; +import com.google.cloud.dataflow.sdk.util.TestCredential; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PDone; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mockito; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileOutputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.PrintStream; +import java.nio.ByteBuffer; +import java.nio.channels.SeekableByteChannel; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Tests for TextIO Read and Write transforms. + */ +@RunWith(JUnit4.class) +public class TextIOTest { + + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + private static class EmptySeekableByteChannel implements SeekableByteChannel { + public long position() { + return 0L; + } + + public SeekableByteChannel position(long newPosition) { + return this; + } + + public long size() { + return 0L; + } + + public SeekableByteChannel truncate(long size) { + return this; + } + + public int write(ByteBuffer src) { + return 0; + } + + public int read(ByteBuffer dst) { + return 0; + } + + public boolean isOpen() { + return true; + } + + public void close() { } + } + + private GcsUtil buildMockGcsUtil() throws IOException { + GcsUtil mockGcsUtil = Mockito.mock(GcsUtil.class); + + // Any request to open gets a new bogus channel + Mockito + .when(mockGcsUtil.open(Mockito.any(GcsPath.class))) + .thenReturn(new EmptySeekableByteChannel()); + + // Any request for expansion gets a single bogus URL + // after we first run the expansion code (which will generally + // return no results, which causes a crash we aren't testing) + Mockito + .when(mockGcsUtil.expand(Mockito.any(GcsPath.class))) + .thenReturn(Arrays.asList(GcsPath.fromUri("gs://bucket/foo"))); + + return mockGcsUtil; + } + + private TestDataflowPipelineOptions buildTestPipelineOptions() { + TestDataflowPipelineOptions options = + PipelineOptionsFactory.as(TestDataflowPipelineOptions.class); + options.setGcpCredential(new TestCredential()); + return options; + } + + void runTestRead(T[] expected, Coder coder) throws Exception { + File tmpFile = tmpFolder.newFile("file.txt"); + String filename = tmpFile.getPath(); + + try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) { + for (T elem : expected) { + byte[] encodedElem = CoderUtils.encodeToByteArray(coder, elem); + String line = new String(encodedElem); + writer.println(line); + } + } + + DirectPipeline p = DirectPipeline.createForTest(); + + TextIO.Read.Bound read; + if (coder.equals(StringUtf8Coder.of())) { + TextIO.Read.Bound readStrings = TextIO.Read.from(filename); + // T==String + read = (TextIO.Read.Bound) readStrings; + } else { + read = TextIO.Read.from(filename).withCoder(coder); + } + + PCollection output = p.apply(read); + + EvaluationResults results = p.run(); + + assertThat(results.getPCollection(output), + containsInAnyOrder(expected)); + } + + @Test + public void testReadStrings() throws Exception { + runTestRead(LINES_ARRAY, StringUtf8Coder.of()); + } + + @Test + public void testReadEmptyStrings() throws Exception { + runTestRead(NO_LINES_ARRAY, StringUtf8Coder.of()); + } + + @Test + public void testReadInts() throws Exception { + runTestRead(INTS_ARRAY, TextualIntegerCoder.of()); + } + + @Test + public void testReadEmptyInts() throws Exception { + runTestRead(NO_INTS_ARRAY, TextualIntegerCoder.of()); + } + + @Test + public void testReadNamed() { + Pipeline p = DirectPipeline.createForTest(); + + { + PCollection output1 = + p.apply(TextIO.Read.from("/tmp/file.txt")); + assertEquals("TextIO.Read.out", output1.getName()); + } + + { + PCollection output2 = + p.apply(TextIO.Read.named("MyRead").from("/tmp/file.txt")); + assertEquals("MyRead.out", output2.getName()); + } + + { + PCollection output3 = + p.apply(TextIO.Read.from("/tmp/file.txt").named("HerRead")); + assertEquals("HerRead.out", output3.getName()); + } + } + + void runTestWrite(T[] elems, Coder coder) throws Exception { + File tmpFile = tmpFolder.newFile("file.txt"); + String filename = tmpFile.getPath(); + + DirectPipeline p = DirectPipeline.createForTest(); + + PCollection input = + p.apply(Create.of(Arrays.asList(elems))).setCoder(coder); + + TextIO.Write.Bound write; + if (coder.equals(StringUtf8Coder.of())) { + TextIO.Write.Bound writeStrings = + TextIO.Write.to(filename).withoutSharding(); + // T==String + write = (TextIO.Write.Bound) writeStrings; + } else { + write = TextIO.Write.to(filename).withCoder(coder).withoutSharding(); + } + + PDone output = input.apply(write); + + EvaluationResults results = p.run(); + + BufferedReader reader = new BufferedReader(new FileReader(tmpFile)); + List actual = new ArrayList<>(); + for (;;) { + String line = reader.readLine(); + if (line == null) { + break; + } + actual.add(line); + } + + String[] expected = new String[elems.length]; + for (int i = 0; i < elems.length; i++) { + T elem = elems[i]; + byte[] encodedElem = CoderUtils.encodeToByteArray(coder, elem); + String line = new String(encodedElem); + expected[i] = line; + } + + assertThat(actual, + containsInAnyOrder(expected)); + } + + @Test + public void testWriteStrings() throws Exception { + runTestWrite(LINES_ARRAY, StringUtf8Coder.of()); + } + + @Test + public void testWriteEmptyStrings() throws Exception { + runTestWrite(NO_LINES_ARRAY, StringUtf8Coder.of()); + } + + @Test + public void testWriteInts() throws Exception { + runTestWrite(INTS_ARRAY, TextualIntegerCoder.of()); + } + + @Test + public void testWriteEmptyInts() throws Exception { + runTestWrite(NO_INTS_ARRAY, TextualIntegerCoder.of()); + } + + @Test + public void testWriteSharded() throws IOException { + File outFolder = tmpFolder.newFolder(); + String filename = outFolder.toPath().resolve("output").toString(); + + DirectPipeline p = DirectPipeline.createForTest(); + + PCollection input = + p.apply(Create.of(Arrays.asList(LINES_ARRAY))) + .setCoder(StringUtf8Coder.of()); + + PDone done = input.apply( + TextIO.Write.to(filename).withNumShards(2).withSuffix(".txt")); + + EvaluationResults results = p.run(); + + String[] files = outFolder.list(); + + assertThat(Arrays.asList(files), + containsInAnyOrder("output-00000-of-00002.txt", + "output-00001-of-00002.txt")); + } + + @Test + public void testWriteNamed() { + Pipeline p = DirectPipeline.createForTest(); + + PCollection input = + p.apply(Create.of(LINES)).setCoder(StringUtf8Coder.of()); + + { + PTransform, PDone> transform1 = + TextIO.Write.to("/tmp/file.txt"); + assertEquals("TextIO.Write", transform1.getName()); + } + + { + PTransform, PDone> transform2 = + TextIO.Write.named("MyWrite").to("/tmp/file.txt"); + assertEquals("MyWrite", transform2.getName()); + } + + { + PTransform, PDone> transform3 = + TextIO.Write.to("/tmp/file.txt").named("HerWrite"); + assertEquals("HerWrite", transform3.getName()); + } + } + + @Test(expected = IllegalArgumentException.class) + public void testUnsupportedFilePattern() throws IOException { + File outFolder = tmpFolder.newFolder(); + String filename = outFolder.toPath().resolve("output@*").toString(); + + DirectPipeline p = DirectPipeline.createForTest(); + + PCollection input = + p.apply(Create.of(Arrays.asList(LINES_ARRAY))) + .setCoder(StringUtf8Coder.of()); + + PDone done = input.apply(TextIO.Write.to(filename)); + + EvaluationResults results = p.run(); + Assert.fail("Expected failure due to unsupported output pattern"); + } + + /** + * The first wildcard must occur after the last directory delimiter. + * This tests a few corner cases that should not crash. + */ + @Test + public void testGoodWildcards() throws Exception { + TestDataflowPipelineOptions options = buildTestPipelineOptions(); + options.setGcsUtil(buildMockGcsUtil()); + + Pipeline pipeline = Pipeline.create(options); + + pipeline.apply(TextIO.Read.from("gs://bucket/foo")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/*")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/?")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/[0-9]")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/*baz*")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/*baz?")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/[0-9]baz?")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/baz/*")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/baz/*wonka*")); + + // Check that running doesn't fail. + pipeline.run(); + } + + /** + * The first wildcard must occur after the last directory delimiter. + * This tests "*". + */ + @Test + public void testBadWildcardStar() throws Exception { + Pipeline pipeline = Pipeline.create(buildTestPipelineOptions()); + + pipeline.apply(TextIO.Read.from("gs://bucket/foo*/baz")); + + // Check that running does fail. + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("wildcard"); + pipeline.run(); + } + + /** + * The first wildcard must occur after the last directory delimiter. + * This tests "?". + */ + @Test + public void testBadWildcardOptional() throws Exception { + Pipeline pipeline = Pipeline.create(buildTestPipelineOptions()); + + pipeline.apply(TextIO.Read.from("gs://bucket/foo?/baz")); + + // Check that running does fail. + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("wildcard"); + pipeline.run(); + } + + /** + * The first wildcard must occur after the last directory delimiter. + * This tests "[]" based character classes. + */ + @Test + public void testBadWildcardBrackets() throws Exception { + Pipeline pipeline = Pipeline.create(buildTestPipelineOptions()); + + pipeline.apply(TextIO.Read.from("gs://bucket/foo[0-9]/baz")); + + // Check that translation does fail. + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("wildcard"); + pipeline.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/user.avsc b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/user.avsc new file mode 100644 index 0000000000000..451a19fa12c32 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/user.avsc @@ -0,0 +1,10 @@ +{ + "namespace": "com.google.cloud.dataflow.sdk.io", + "type": "record", + "name": "User", + "fields": [ + { "name": "name", "type": "string"}, + { "name": "favorite_number", "type": ["int", "null"]}, + { "name": "favorite_color", "type": ["string", "null"]} + ] +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java new file mode 100644 index 0000000000000..f4d6f0499d44f --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java @@ -0,0 +1,94 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.dataflow.sdk.testing.ResetDateTimeProvider; +import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestRule; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link DataflowPipelineOptions}. */ +@RunWith(JUnit4.class) +public class DataflowPipelineOptionsTest { + @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); + @Rule public ResetDateTimeProvider resetDateTimeProviderRule = new ResetDateTimeProvider(); + + @Test + public void testJobNameIsSet() { + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setJobName("TestJobName"); + assertEquals("TestJobName", options.getJobName()); + } + + @Test + public void testUserNameIsNotSet() { + resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z"); + System.getProperties().remove("user.name"); + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setAppName("TestApplication"); + assertEquals("testapplication--1208190706", options.getJobName()); + assertTrue(options.getJobName().length() <= 40); + } + + @Test + public void testAppNameAndUserNameIsTooLong() { + resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z"); + System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde"); + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setAppName("1234567890123456789012345678901234567890"); + assertEquals("a234567890123456789-abcdeabcd-1208190706", options.getJobName()); + assertTrue(options.getJobName().length() <= 40); + } + + @Test + public void testAppNameIsTooLong() { + resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z"); + System.getProperties().put("user.name", "abcde"); + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setAppName("1234567890123456789012345678901234567890"); + assertEquals("a2345678901234567890123-abcde-1208190706", options.getJobName()); + assertTrue(options.getJobName().length() <= 40); + } + + @Test + public void testUserNameIsTooLong() { + resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z"); + System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde"); + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setAppName("1234567890"); + assertEquals("a234567890-abcdeabcdeabcdeabc-1208190706", options.getJobName()); + assertTrue(options.getJobName().length() <= 40); + } + + + @Test + public void testUtf8UserNameAndApplicationNameIsNormalized() { + resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z"); + System.getProperties().put("user.name", "ði ıntəˈnæʃənəl "); + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setAppName("fəˈnɛtık əsoʊsiˈeıʃn"); + assertEquals("f00n0t0k00so0si0e00-0i00nt00n-1208190706", options.getJobName()); + assertTrue(options.getJobName().length() <= 40); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java new file mode 100644 index 0000000000000..ca1e9502bf97d --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java @@ -0,0 +1,502 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner; +import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import com.fasterxml.jackson.annotation.JsonIgnore; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TestRule; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.List; + +/** Tests for {@link PipelineOptionsFactory}. */ +@RunWith(JUnit4.class) +public class PipelineOptionsFactoryTest { + @Rule public ExpectedException expectedException = ExpectedException.none(); + @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); + + @Test + public void testCreationFromSystemProperties() { + System.getProperties().putAll(ImmutableMap + .builder() + .put("root_url", "test_root_url") + .put("service_path", "test_service_path") + .put("temp_gcs_directory", + "gs://tap-testing-30lsaafg6g3zudmjbnsdz6wj/unittesting/staging") + .put("service_account_name", "test_service_account_name") + .put("service_account_keyfile", "test_service_account_keyfile") + .put("worker_id", "test_worker_id") + .put("project_id", "test_project_id") + .put("job_id", "test_job_id") + .build()); + DataflowWorkerHarnessOptions options = PipelineOptionsFactory.createFromSystemProperties(); + assertEquals("test_root_url", options.getApiRootUrl()); + assertEquals("test_service_path", options.getDataflowEndpoint()); + assertEquals("gs://tap-testing-30lsaafg6g3zudmjbnsdz6wj/unittesting/staging", + options.getTempLocation()); + assertEquals("test_service_account_name", options.getServiceAccountName()); + assertEquals("test_service_account_keyfile", options.getServiceAccountKeyfile()); + assertEquals("test_worker_id", options.getWorkerId()); + assertEquals("test_project_id", options.getProject()); + assertEquals("test_job_id", options.getJobId()); + } + + @Test + public void testAppNameIsSet() { + ApplicationNameOptions options = PipelineOptionsFactory.as(ApplicationNameOptions.class); + assertEquals(PipelineOptionsFactoryTest.class.getSimpleName(), options.getAppName()); + } + + /** A simple test interface. */ + public static interface TestPipelineOptions extends PipelineOptions { + String getTestPipelineOption(); + void setTestPipelineOption(String value); + } + + @Test + public void testAppNameIsSetWhenUsingAs() { + TestPipelineOptions options = PipelineOptionsFactory.as(TestPipelineOptions.class); + assertEquals(PipelineOptionsFactoryTest.class.getSimpleName(), + options.as(ApplicationNameOptions.class).getAppName()); + } + + @Test + public void testManualRegistration() { + assertFalse(PipelineOptionsFactory.getRegisteredOptions().contains(TestPipelineOptions.class)); + PipelineOptionsFactory.register(TestPipelineOptions.class); + assertTrue(PipelineOptionsFactory.getRegisteredOptions().contains(TestPipelineOptions.class)); + } + + @Test + public void testDefaultRegistration() { + assertTrue(PipelineOptionsFactory.getRegisteredOptions().contains(PipelineOptions.class)); + } + + /** A test interface missing a getter. */ + public static interface MissingGetter extends PipelineOptions { + void setObject(Object value); + } + + @Test + public void testMissingGetterThrows() throws Exception { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage( + "Expected getter for property [object] of type [java.lang.Object] on " + + "[com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$MissingGetter]."); + + PipelineOptionsFactory.as(MissingGetter.class); + } + + /** A test interface missing a setter. */ + public static interface MissingSetter extends PipelineOptions { + Object getObject(); + } + + @Test + public void testMissingSetterThrows() throws Exception { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage( + "Expected setter for property [object] of type [java.lang.Object] on " + + "[com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$MissingSetter]."); + + PipelineOptionsFactory.as(MissingSetter.class); + } + + /** A test interface representing a composite interface. */ + public static interface CombinedObject extends MissingGetter, MissingSetter { + } + + @Test + public void testHavingSettersGettersFromSeparateInterfacesIsValid() { + PipelineOptionsFactory.as(CombinedObject.class); + } + + /** A test interface which contains a non-bean style method. */ + public static interface ExtraneousMethod extends PipelineOptions { + public String extraneousMethod(int value, String otherValue); + } + + @Test + public void testHavingExtraneousMethodThrows() throws Exception { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage( + "Methods [java.lang.String extraneousMethod(int, java.lang.String)] on " + + "[com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$ExtraneousMethod] " + + "do not conform to being bean properties."); + + PipelineOptionsFactory.as(ExtraneousMethod.class); + } + + /** A test interface which has a conflicting return type with its parent. */ + public static interface ReturnTypeConflict extends CombinedObject { + @Override + String getObject(); + void setObject(String value); + } + + @Test + public void testReturnTypeConflictThrows() throws Exception { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage( + "Method [getObject] has multiple definitions [public abstract java.lang.Object " + + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$MissingSetter" + + ".getObject(), public abstract java.lang.String " + + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$ReturnTypeConflict" + + ".getObject()] with different return types for [" + + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$ReturnTypeConflict]."); + PipelineOptionsFactory.as(ReturnTypeConflict.class); + } + + /** Test interface that has {@link JsonIgnore @JsonIgnore} on a setter for a property. */ + public static interface SetterWithJsonIgnore extends PipelineOptions { + String getValue(); + @JsonIgnore + void setValue(String value); + } + + @Test + public void testSetterAnnotatedWithJsonIgnore() throws Exception { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage( + "Expected setter for property [value] to not be marked with @JsonIgnore on [com." + + "google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$SetterWithJsonIgnore]"); + PipelineOptionsFactory.as(SetterWithJsonIgnore.class); + } + + /** + * This class is has a conflicting field with {@link CombinedObject} that doesn't have + * {@link JsonIgnore @JsonIgnore}. + */ + public static interface GetterWithJsonIgnore extends PipelineOptions { + @JsonIgnore + Object getObject(); + void setObject(Object value); + } + + @Test + public void testNotAllGettersAnnotatedWithJsonIgnore() throws Exception { + // Initial construction is valid. + GetterWithJsonIgnore options = PipelineOptionsFactory.as(GetterWithJsonIgnore.class); + + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage( + "Expected getter for property [object] to be marked with @JsonIgnore on all [com." + + "google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$MissingSetter, " + + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$GetterWithJsonIgnore], " + + "found only on [com.google.cloud.dataflow.sdk.options." + + "PipelineOptionsFactoryTest$GetterWithJsonIgnore]"); + + // When we attempt to convert, we should error at this moment. + options.as(CombinedObject.class); + } + + @Test + public void testAppNameIsNotOverriddenWhenPassedInViaCommandLine() { + ApplicationNameOptions options = PipelineOptionsFactory + .fromArgs(new String[]{ "--appName=testAppName" }) + .as(ApplicationNameOptions.class); + assertEquals("testAppName", options.getAppName()); + } + + @Test + public void testPropertyIsSetOnRegisteredPipelineOptionNotPartOfOriginalInterface() { + PipelineOptions options = PipelineOptionsFactory + .fromArgs(new String[]{ "--project=testProject" }) + .create(); + assertEquals("testProject", options.as(GcpOptions.class).getProject()); + } + + /** A test interface containing all the primitives */ + public static interface Primitives extends PipelineOptions { + boolean getBoolean(); + void setBoolean(boolean value); + char getChar(); + void setChar(char value); + byte getByte(); + void setByte(byte value); + short getShort(); + void setShort(short value); + int getInt(); + void setInt(int value); + long getLong(); + void setLong(long value); + float getFloat(); + void setFloat(float value); + double getDouble(); + void setDouble(double value); + } + + @Test + public void testPrimitives() { + String[] args = new String[] { + "--boolean=true", + "--char=d", + "--byte=12", + "--short=300", + "--int=100000", + "--long=123890123890", + "--float=55.5", + "--double=12.3"}; + + Primitives options = PipelineOptionsFactory.fromArgs(args).as(Primitives.class); + assertTrue(options.getBoolean()); + assertEquals('d', options.getChar()); + assertEquals((byte) 12, options.getByte()); + assertEquals((short) 300, options.getShort()); + assertEquals(100000, options.getInt()); + assertEquals(123890123890L, options.getLong()); + assertEquals(55.5f, options.getFloat(), 0.0f); + assertEquals(12.3, options.getDouble(), 0.0); + } + + @Test + public void testBooleanShorthandArgument() { + String[] args = new String[] {"--boolean"}; + + Primitives options = PipelineOptionsFactory.fromArgs(args).as(Primitives.class); + assertTrue(options.getBoolean()); + } + + /** A test interface containing all supported objects */ + public static interface Objects extends PipelineOptions { + Boolean getBoolean(); + void setBoolean(Boolean value); + Character getChar(); + void setChar(Character value); + Byte getByte(); + void setByte(Byte value); + Short getShort(); + void setShort(Short value); + Integer getInt(); + void setInt(Integer value); + Long getLong(); + void setLong(Long value); + Float getFloat(); + void setFloat(Float value); + Double getDouble(); + void setDouble(Double value); + String getString(); + void setString(String value); + Class getClassValue(); + void setClassValue(Class value); + } + + @Test + public void testObjects() { + String[] args = new String[] { + "--boolean=true", + "--char=d", + "--byte=12", + "--short=300", + "--int=100000", + "--long=123890123890", + "--float=55.5", + "--double=12.3", + "--string=stringValue", + "--classValue=" + PipelineOptionsFactoryTest.class.getName()}; + + Objects options = PipelineOptionsFactory.fromArgs(args).as(Objects.class); + assertTrue(options.getBoolean()); + assertEquals(Character.valueOf('d'), options.getChar()); + assertEquals(Byte.valueOf((byte) 12), options.getByte()); + assertEquals(Short.valueOf((short) 300), options.getShort()); + assertEquals(Integer.valueOf(100000), options.getInt()); + assertEquals(Long.valueOf(123890123890L), options.getLong()); + assertEquals(Float.valueOf(55.5f), options.getFloat(), 0.0f); + assertEquals(Double.valueOf(12.3), options.getDouble(), 0.0); + assertEquals("stringValue", options.getString()); + assertEquals(PipelineOptionsFactoryTest.class, options.getClassValue()); + } + + @Test + public void testMissingArgument() { + String[] args = new String[] {}; + + Objects options = PipelineOptionsFactory.fromArgs(args).as(Objects.class); + assertNull(options.getString()); + } + + /** A test interface containing all supported array return types */ + public static interface Arrays extends PipelineOptions { + boolean[] getBoolean(); + void setBoolean(boolean[] value); + char[] getChar(); + void setChar(char[] value); + short[] getShort(); + void setShort(short[] value); + int[] getInt(); + void setInt(int[] value); + long[] getLong(); + void setLong(long[] value); + float[] getFloat(); + void setFloat(float[] value); + double[] getDouble(); + void setDouble(double[] value); + String[] getString(); + void setString(String[] value); + Class[] getClassValue(); + void setClassValue(Class[] value); + } + + @Test + public void testArrays() { + String[] args = new String[] { + "--boolean=true", + "--boolean=true", + "--boolean=false", + "--char=d", + "--char=e", + "--char=f", + "--short=300", + "--short=301", + "--short=302", + "--int=100000", + "--int=100001", + "--int=100002", + "--long=123890123890", + "--long=123890123891", + "--long=123890123892", + "--float=55.5", + "--float=55.6", + "--float=55.7", + "--double=12.3", + "--double=12.4", + "--double=12.5", + "--string=stringValue1", + "--string=stringValue2", + "--string=stringValue3", + "--classValue=" + PipelineOptionsFactory.class.getName(), + "--classValue=" + PipelineOptionsFactoryTest.class.getName()}; + + Arrays options = PipelineOptionsFactory.fromArgs(args).as(Arrays.class); + boolean[] bools = options.getBoolean(); + assertTrue(bools[0] && bools[1] && !bools[2]); + assertArrayEquals(new char[] {'d', 'e', 'f'}, options.getChar()); + assertArrayEquals(new short[] {300, 301, 302}, options.getShort()); + assertArrayEquals(new int[] {100000, 100001, 100002}, options.getInt()); + assertArrayEquals(new long[] {123890123890L, 123890123891L, 123890123892L}, options.getLong()); + assertArrayEquals(new float[] {55.5f, 55.6f, 55.7f}, options.getFloat(), 0.0f); + assertArrayEquals(new double[] {12.3, 12.4, 12.5}, options.getDouble(), 0.0); + assertArrayEquals(new String[] {"stringValue1", "stringValue2", "stringValue3"}, + options.getString()); + assertArrayEquals(new Class[] {PipelineOptionsFactory.class, + PipelineOptionsFactoryTest.class}, + options.getClassValue()); + } + + @Test + public void testOutOfOrderArrays() { + String[] args = new String[] { + "--char=d", + "--boolean=true", + "--boolean=true", + "--char=e", + "--char=f", + "--boolean=false"}; + + Arrays options = PipelineOptionsFactory.fromArgs(args).as(Arrays.class); + boolean[] bools = options.getBoolean(); + assertTrue(bools[0] && bools[1] && !bools[2]); + assertArrayEquals(new char[] {'d', 'e', 'f'}, options.getChar()); + } + + /** A test interface containing all supported List return types */ + public static interface Lists extends PipelineOptions { + List getString(); + void setString(List value); + } + + @Test + public void testList() { + String[] args = + new String[] {"--string=stringValue1", "--string=stringValue2", "--string=stringValue3"}; + + Lists options = PipelineOptionsFactory.fromArgs(args).as(Lists.class); + assertEquals(ImmutableList.of("stringValue1", "stringValue2", "stringValue3"), + options.getString()); + } + + @Test + public void testListShorthand() { + String[] args = new String[] {"--string=stringValue1,stringValue2,stringValue3"}; + + Lists options = PipelineOptionsFactory.fromArgs(args).as(Lists.class); + assertEquals(ImmutableList.of("stringValue1", "stringValue2", "stringValue3"), + options.getString()); + } + + @Test + public void testMixedShorthandAndLongStyleList() { + String[] args = new String[] { + "--char=d", + "--char=e", + "--char=f", + "--char=g,h,i", + "--char=j", + "--char=k", + "--char=l", + "--char=m,n,o"}; + + Arrays options = PipelineOptionsFactory.fromArgs(args).as(Arrays.class); + assertArrayEquals(new char[] {'d', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o'}, + options.getChar()); + } + + @Test + public void testSetASingularAttributeUsingAListThrowsAnError() { + String[] args = new String[] { + "--diskSizeGb=100", + "--diskSizeGb=200"}; + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("expected one element but was"); + PipelineOptionsFactory.fromArgs(args).create(); + } + + @Test + public void testSettingRunner() { + String[] args = new String[] {"--runner=BlockingDataflowPipelineRunner"}; + + PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create(); + assertEquals(BlockingDataflowPipelineRunner.class, options.getRunner()); + } + + @Test + public void testSettingUnknownRunner() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Unknown 'runner' specified UnknownRunner, supported pipeline " + + "runners [DirectPipelineRunner, DataflowPipelineRunner, BlockingDataflowPipelineRunner]"); + String[] args = new String[] {"--runner=UnknownRunner"}; + + PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create(); + options.getRunner(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsTest.java new file mode 100644 index 0000000000000..9db6a6b754221 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsTest.java @@ -0,0 +1,45 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link PipelineOptions}. */ +@RunWith(JUnit4.class) +public class PipelineOptionsTest { + /** Interface used for testing that {@link PipelineOptions#as(Class)} functions */ + public static interface TestOptions extends PipelineOptions { + } + + @Test + public void testDynamicAs() { + TestOptions options = PipelineOptionsFactory.create().as(TestOptions.class); + assertNotNull(options); + } + + @Test + public void testDefaultRunnerIsSet() { + assertEquals(DirectPipelineRunner.class, PipelineOptionsFactory.create().getRunner()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidatorTest.java new file mode 100644 index 0000000000000..e0decb9f92255 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidatorTest.java @@ -0,0 +1,86 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link PipelineOptionsValidator}. */ +@RunWith(JUnit4.class) +public class PipelineOptionsValidatorTest { + @Rule public ExpectedException expectedException = ExpectedException.none(); + + /** A test interface with an {@link Validation.Required} annotation. */ + public static interface Required extends PipelineOptions { + @Validation.Required + public String getObject(); + public void setObject(String value); + } + + @Test + public void testWhenRequiredOptionIsSet() { + Required required = PipelineOptionsFactory.as(Required.class); + required.setObject("blah"); + PipelineOptionsValidator.validate(Required.class, required); + } + + @Test + public void testWhenRequiredOptionIsSetAndCleared() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Expected non-null property to be set for " + + "[public abstract java.lang.String com.google.cloud.dataflow." + + "sdk.options.PipelineOptionsValidatorTest$Required.getObject()]."); + + Required required = PipelineOptionsFactory.as(Required.class); + required.setObject("blah"); + required.setObject(null); + PipelineOptionsValidator.validate(Required.class, required); + } + + @Test + public void testWhenRequiredOptionIsNeverSet() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Expected non-null property to be set for " + + "[public abstract java.lang.String com.google.cloud.dataflow." + + "sdk.options.PipelineOptionsValidatorTest$Required.getObject()]."); + + Required required = PipelineOptionsFactory.as(Required.class); + PipelineOptionsValidator.validate(Required.class, required); + } + + /** A test interface which overrides the parents method. */ + public static interface SubClassValidation extends Required { + @Override + public String getObject(); + @Override + public void setObject(String value); + } + + @Test + public void testValidationOnOverriddenMethods() throws Exception { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Expected non-null property to be set for " + + "[public abstract java.lang.String com.google.cloud.dataflow." + + "sdk.options.PipelineOptionsValidatorTest$Required.getObject()]."); + + SubClassValidation required = PipelineOptionsFactory.as(SubClassValidation.class); + PipelineOptionsValidator.validate(Required.class, required); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java new file mode 100644 index 0000000000000..b9b07e8626b1d --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java @@ -0,0 +1,625 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Maps; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** Tests for {@link ProxyInvocationHandler}. */ +@RunWith(JUnit4.class) +public class ProxyInvocationHandlerTest { + @Rule public ExpectedException expectedException = ExpectedException.none(); + + /** A test interface with some primitives and objects. */ + public static interface Simple extends PipelineOptions { + boolean isOptionEnabled(); + void setOptionEnabled(boolean value); + int getPrimitive(); + void setPrimitive(int value); + String getString(); + void setString(String value); + } + + @Test + public void testPropertySettingAndGetting() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + Simple proxy = handler.as(Simple.class); + proxy.setString("OBJECT"); + proxy.setOptionEnabled(true); + proxy.setPrimitive(4); + assertEquals("OBJECT", proxy.getString()); + assertTrue(proxy.isOptionEnabled()); + assertEquals(4, proxy.getPrimitive()); + } + + /** A test interface containing all the JLS default values. */ + public static interface JLSDefaults extends PipelineOptions { + boolean getBoolean(); + void setBoolean(boolean value); + char getChar(); + void setChar(char value); + byte getByte(); + void setByte(byte value); + short getShort(); + void setShort(short value); + int getInt(); + void setInt(int value); + long getLong(); + void setLong(long value); + float getFloat(); + void setFloat(float value); + double getDouble(); + void setDouble(double value); + Object getObject(); + void setObject(Object value); + } + + @Test + public void testGettingJLSDefaults() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + JLSDefaults proxy = handler.as(JLSDefaults.class); + assertFalse(proxy.getBoolean()); + assertEquals('\0', proxy.getChar()); + assertEquals((byte) 0, proxy.getByte()); + assertEquals((short) 0, proxy.getShort()); + assertEquals(0, proxy.getInt()); + assertEquals(0L, proxy.getLong()); + assertEquals(0f, proxy.getFloat(), 0f); + assertEquals(0d, proxy.getDouble(), 0d); + assertNull(proxy.getObject()); + } + + /** A {@link DefaultValueFactory} which is used for testing. */ + public static class TestOptionFactory implements DefaultValueFactory { + @Override + public String create(PipelineOptions options) { + return "testOptionFactory"; + } + } + + /** A test interface containing all the {@link Default} annotations. */ + public static interface DefaultAnnotations extends PipelineOptions { + @Default.Boolean(true) + boolean getBoolean(); + void setBoolean(boolean value); + @Default.Character('a') + char getChar(); + void setChar(char value); + @Default.Byte((byte) 4) + byte getByte(); + void setByte(byte value); + @Default.Short((short) 5) + short getShort(); + void setShort(short value); + @Default.Integer(6) + int getInt(); + void setInt(int value); + @Default.Long(7L) + long getLong(); + void setLong(long value); + @Default.Float(8f) + float getFloat(); + void setFloat(float value); + @Default.Double(9d) + double getDouble(); + void setDouble(double value); + @Default.String("testString") + String getString(); + void setString(String value); + @Default.Class(DefaultAnnotations.class) + Class getClassOption(); + void setClassOption(Class value); + @Default.InstanceFactory(TestOptionFactory.class) + String getComplex(); + void setComplex(String value); + } + + @Test + public void testAnnotationDefaults() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + DefaultAnnotations proxy = handler.as(DefaultAnnotations.class); + assertTrue(proxy.getBoolean()); + assertEquals('a', proxy.getChar()); + assertEquals((byte) 4, proxy.getByte()); + assertEquals((short) 5, proxy.getShort()); + assertEquals(6, proxy.getInt()); + assertEquals(7, proxy.getLong()); + assertEquals(8f, proxy.getFloat(), 0f); + assertEquals(9d, proxy.getDouble(), 0d); + assertEquals("testString", proxy.getString()); + assertEquals(DefaultAnnotations.class, proxy.getClassOption()); + assertEquals("testOptionFactory", proxy.getComplex()); + } + + @Test + public void testEquals() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + Simple proxy = handler.as(Simple.class); + JLSDefaults sameAsProxy = proxy.as(JLSDefaults.class); + ProxyInvocationHandler handler2 = new ProxyInvocationHandler(Maps.newHashMap()); + Simple proxy2 = handler2.as(Simple.class); + JLSDefaults sameAsProxy2 = proxy2.as(JLSDefaults.class); + assertTrue(handler.equals(proxy)); + assertTrue(proxy.equals(proxy)); + assertTrue(proxy.equals(sameAsProxy)); + assertFalse(handler.equals(handler2)); + assertFalse(proxy.equals(proxy2)); + assertFalse(proxy.equals(sameAsProxy2)); + } + + @Test + public void testHashCode() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + Simple proxy = handler.as(Simple.class); + JLSDefaults sameAsProxy = proxy.as(JLSDefaults.class); + ProxyInvocationHandler handler2 = new ProxyInvocationHandler(Maps.newHashMap()); + Simple proxy2 = handler.as(Simple.class); + JLSDefaults sameAsProxy2 = proxy.as(JLSDefaults.class); + assertTrue(handler.hashCode() == proxy.hashCode()); + assertTrue(proxy.hashCode() == sameAsProxy.hashCode()); + assertFalse(handler.hashCode() != handler2.hashCode()); + assertFalse(proxy.hashCode() != proxy2.hashCode()); + assertFalse(proxy.hashCode() != sameAsProxy2.hashCode()); + } + + @Test + public void testToString() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + Simple proxy = handler.as(Simple.class); + proxy.setString("stringValue"); + DefaultAnnotations proxy2 = proxy.as(DefaultAnnotations.class); + proxy2.setLong(57L); + assertEquals("Current Settings:\n" + + " long: 57\n" + + " string: stringValue\n", + proxy.toString()); + } + + /** A test interface containing an unknown method. */ + public static interface UnknownMethod { + void unknownMethod(); + } + + @Test + public void testInvokeWithUnknownMethod() throws Exception { + expectedException.expect(RuntimeException.class); + expectedException.expectMessage("Unknown method [public abstract void com.google.cloud." + + "dataflow.sdk.options.ProxyInvocationHandlerTest$UnknownMethod.unknownMethod()] " + + "invoked with args [null]."); + + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + handler.invoke(handler, UnknownMethod.class.getMethod("unknownMethod"), null); + } + + /** A test interface which extends another interface. */ + public static interface SubClass extends Simple { + String getExtended(); + void setExtended(String value); + } + + @Test + public void testSubClassStoresSuperInterfaceValues() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + SubClass extended = handler.as(SubClass.class); + + extended.setString("parentValue"); + assertEquals("parentValue", extended.getString()); + } + + @Test + public void testUpCastRetainsSuperInterfaceValues() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + SubClass extended = handler.as(SubClass.class); + + extended.setString("parentValue"); + Simple simple = extended.as(Simple.class); + assertEquals("parentValue", simple.getString()); + } + + @Test + public void testUpCastRetainsSubClassValues() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + SubClass extended = handler.as(SubClass.class); + + extended.setExtended("subClassValue"); + SubClass extended2 = extended.as(Simple.class).as(SubClass.class); + assertEquals("subClassValue", extended2.getExtended()); + } + + /** A test interface which is a sibling to {@link SubClass}. */ + public static interface Sibling extends Simple { + String getSibling(); + void setSibling(String value); + } + + @Test + public void testAsSiblingRetainsSuperInterfaceValues() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + SubClass extended = handler.as(SubClass.class); + + extended.setString("parentValue"); + Sibling sibling = extended.as(Sibling.class); + assertEquals("parentValue", sibling.getString()); + } + + /** A test interface which has the same methods as the parent. */ + public static interface MethodConflict extends Simple { + @Override + String getString(); + @Override + void setString(String value); + } + + @Test + public void testMethodConflictProvidesSameValue() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + MethodConflict methodConflict = handler.as(MethodConflict.class); + + methodConflict.setString("conflictValue"); + assertEquals("conflictValue", methodConflict.getString()); + assertEquals("conflictValue", methodConflict.as(Simple.class).getString()); + } + + /** A test interface which has the same methods as its parent and grandparent. */ + public static interface DeepMethodConflict extends MethodConflict { + @Override + String getString(); + @Override + void setString(String value); + @Override + int getPrimitive(); + @Override + void setPrimitive(int value); + } + + @Test + public void testDeepMethodConflictProvidesSameValue() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + DeepMethodConflict deepMethodConflict = handler.as(DeepMethodConflict.class); + + // Tests overriding an already overridden method + deepMethodConflict.setString("conflictValue"); + assertEquals("conflictValue", deepMethodConflict.getString()); + assertEquals("conflictValue", deepMethodConflict.as(MethodConflict.class).getString()); + assertEquals("conflictValue", deepMethodConflict.as(Simple.class).getString()); + + // Tests overriding a method from an ancestor class + deepMethodConflict.setPrimitive(5); + assertEquals(5, deepMethodConflict.getPrimitive()); + assertEquals(5, deepMethodConflict.as(MethodConflict.class).getPrimitive()); + assertEquals(5, deepMethodConflict.as(Simple.class).getPrimitive()); + } + + /** A test interface which shares the same methods as {@link Sibling}. */ + public static interface SimpleSibling extends PipelineOptions { + String getString(); + void setString(String value); + } + + @Test + public void testDisjointSiblingsShareValues() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + SimpleSibling proxy = handler.as(SimpleSibling.class); + proxy.setString("siblingValue"); + assertEquals("siblingValue", proxy.getString()); + assertEquals("siblingValue", proxy.as(Simple.class).getString()); + } + + /** A test interface which joins two sibling interfaces which have conflicting methods. */ + public static interface SiblingMethodConflict extends Simple, SimpleSibling { + } + + @Test + public void testSiblingMethodConflict() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + SiblingMethodConflict siblingMethodConflict = handler.as(SiblingMethodConflict.class); + siblingMethodConflict.setString("siblingValue"); + assertEquals("siblingValue", siblingMethodConflict.getString()); + assertEquals("siblingValue", siblingMethodConflict.as(Simple.class).getString()); + assertEquals("siblingValue", siblingMethodConflict.as(SimpleSibling.class).getString()); + } + + /** A test interface which has only the getter and only a setter overriden. */ + public static interface PartialMethodConflict extends Simple { + @Override + String getString(); + @Override + void setPrimitive(int value); + } + + @Test + public void testPartialMethodConflictProvidesSameValue() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + PartialMethodConflict partialMethodConflict = handler.as(PartialMethodConflict.class); + + // Tests overriding a getter property which is only partially bound + partialMethodConflict.setString("conflictValue"); + assertEquals("conflictValue", partialMethodConflict.getString()); + assertEquals("conflictValue", partialMethodConflict.as(Simple.class).getString()); + + // Tests overriding a setter property which is only partially bound + partialMethodConflict.setPrimitive(5); + assertEquals(5, partialMethodConflict.getPrimitive()); + assertEquals(5, partialMethodConflict.as(Simple.class).getPrimitive()); + } + + @Test + public void testJsonConversionForDefault() throws Exception { + PipelineOptions options = PipelineOptionsFactory.create(); + assertNotNull(serializeDeserialize(PipelineOptions.class, options)); + } + + /** Test interface for JSON conversion of simple types */ + private static interface SimpleTypes extends PipelineOptions { + int getInteger(); + void setInteger(int value); + String getString(); + void setString(String value); + } + + @Test + public void testJsonConversionForSimpleTypes() throws Exception { + SimpleTypes options = PipelineOptionsFactory.as(SimpleTypes.class); + options.setString("TestValue"); + options.setInteger(5); + SimpleTypes options2 = serializeDeserialize(SimpleTypes.class, options); + assertEquals(5, options2.getInteger()); + assertEquals("TestValue", options2.getString()); + } + + @Test + public void testJsonConversionOfAJsonConvertedType() throws Exception { + SimpleTypes options = PipelineOptionsFactory.as(SimpleTypes.class); + options.setString("TestValue"); + options.setInteger(5); + SimpleTypes options2 = serializeDeserialize(SimpleTypes.class, + serializeDeserialize(SimpleTypes.class, options)); + assertEquals(5, options2.getInteger()); + assertEquals("TestValue", options2.getString()); + } + + @Test + public void testJsonConversionForPartiallySerializedValues() throws Exception { + SimpleTypes options = PipelineOptionsFactory.as(SimpleTypes.class); + options.setInteger(5); + SimpleTypes options2 = serializeDeserialize(SimpleTypes.class, options); + options2.setString("TestValue"); + SimpleTypes options3 = serializeDeserialize(SimpleTypes.class, options2); + assertEquals(5, options3.getInteger()); + assertEquals("TestValue", options3.getString()); + } + + @Test + public void testJsonConversionForOverriddenSerializedValues() throws Exception { + SimpleTypes options = PipelineOptionsFactory.as(SimpleTypes.class); + options.setInteger(-5); + options.setString("NeedsToBeOverridden"); + SimpleTypes options2 = serializeDeserialize(SimpleTypes.class, options); + options2.setInteger(5); + options2.setString("TestValue"); + SimpleTypes options3 = serializeDeserialize(SimpleTypes.class, options2); + assertEquals(5, options3.getInteger()); + assertEquals("TestValue", options3.getString()); + } + + /** Test interface for JSON conversion of container types */ + private static interface ContainerTypes extends PipelineOptions { + List getList(); + void setList(List values); + Map getMap(); + void setMap(Map values); + Set getSet(); + void setSet(Set values); + } + + @Test + public void testJsonConversionForContainerTypes() throws Exception { + List list = ImmutableList.of("a", "b", "c"); + Map map = ImmutableMap.of("d", "x", "e", "y", "f", "z"); + Set set = ImmutableSet.of("g", "h", "i"); + ContainerTypes options = PipelineOptionsFactory.as(ContainerTypes.class); + options.setList(list); + options.setMap(map); + options.setSet(set); + ContainerTypes options2 = serializeDeserialize(ContainerTypes.class, options); + assertEquals(list, options2.getList()); + assertEquals(map, options2.getMap()); + assertEquals(set, options2.getSet()); + } + + /** Test interface for conversion of inner types */ + private static class InnerType { + public double doubleField; + + static InnerType of(double value) { + InnerType rval = new InnerType(); + rval.doubleField = value; + return rval; + } + + @Override + public boolean equals(Object obj) { + return obj != null + && getClass().equals(obj.getClass()) + && Objects.equals(doubleField, ((InnerType) obj).doubleField); + } + } + + /** Test interface for conversion of generics and inner types */ + private static class ComplexType { + public String stringField; + public Integer intField; + public List genericType; + public InnerType innerType; + + @Override + public boolean equals(Object obj) { + return obj != null + && getClass().equals(obj.getClass()) + && Objects.equals(stringField, ((ComplexType) obj).stringField) + && Objects.equals(intField, ((ComplexType) obj).intField) + && Objects.equals(genericType, ((ComplexType) obj).genericType) + && Objects.equals(innerType, ((ComplexType) obj).innerType); + } + } + + private static interface ComplexTypes extends PipelineOptions { + ComplexType getComplexType(); + void setComplexType(ComplexType value); + } + + @Test + public void testJsonConversionForComplexType() throws Exception { + ComplexType complexType = new ComplexType(); + complexType.stringField = "stringField"; + complexType.intField = 12; + complexType.innerType = InnerType.of(12); + complexType.genericType = ImmutableList.of(InnerType.of(16234), InnerType.of(24)); + + ComplexTypes options = PipelineOptionsFactory.as(ComplexTypes.class); + options.setComplexType(complexType); + ComplexTypes options2 = serializeDeserialize(ComplexTypes.class, options); + assertEquals(complexType, options2.getComplexType()); + } + + /** Test interface for testing ignored properties during serialization. */ + private static interface IgnoredProperty extends PipelineOptions { + @JsonIgnore + String getValue(); + void setValue(String value); + } + + @Test + public void testJsonConversionOfIgnoredProperty() throws Exception { + IgnoredProperty options = PipelineOptionsFactory.as(IgnoredProperty.class); + options.setValue("TestValue"); + + IgnoredProperty options2 = serializeDeserialize(IgnoredProperty.class, options); + assertNull(options2.getValue()); + } + + /** Test class which is not serializable by Jackson. */ + public static class NotSerializable { + private String value; + public NotSerializable(String value) { + this.value = value; + } + + public String getValue() { + return value; + } + } + + /** Test interface containing a class which is not serializable by Jackson. */ + private static interface NotSerializableProperty extends PipelineOptions { + NotSerializable getValue(); + void setValue(NotSerializable value); + } + + @Test(expected = JsonMappingException.class) + public void testJsonConversionOfNotSerializableProperty() throws Exception { + NotSerializableProperty options = PipelineOptionsFactory.as(NotSerializableProperty.class); + options.setValue(new NotSerializable("TestString")); + + serializeDeserialize(NotSerializableProperty.class, options); + } + + /** + * Test interface which has {@link JsonIgnore @JsonIgnore} on a property that Jackson + * can't serialize. + */ + private static interface IgnoredNotSerializableProperty extends PipelineOptions { + @JsonIgnore + NotSerializable getValue(); + void setValue(NotSerializable value); + } + + @Test + public void testJsonConversionOfIgnoredNotSerializableProperty() throws Exception { + IgnoredNotSerializableProperty options = + PipelineOptionsFactory.as(IgnoredNotSerializableProperty.class); + options.setValue(new NotSerializable("TestString")); + + IgnoredNotSerializableProperty options2 = + serializeDeserialize(IgnoredNotSerializableProperty.class, options); + assertNull(options2.getValue()); + } + + /** Test class which is only serializable by Jackson with the added metadata. */ + public static class SerializableWithMetadata { + private String value; + public SerializableWithMetadata(@JsonProperty("value") String value) { + this.value = value; + } + + @JsonProperty("value") + public String getValue() { + return value; + } + } + + /** + * Test interface containing a property which is only serializable by Jackson with + * the additional metadata. + */ + private static interface SerializableWithMetadataProperty extends PipelineOptions { + SerializableWithMetadata getValue(); + void setValue(SerializableWithMetadata value); + } + + @Test + public void testJsonConversionOfSerializableWithMetadataProperty() throws Exception { + SerializableWithMetadataProperty options = + PipelineOptionsFactory.as(SerializableWithMetadataProperty.class); + options.setValue(new SerializableWithMetadata("TestString")); + + SerializableWithMetadataProperty options2 = + serializeDeserialize(SerializableWithMetadataProperty.class, options); + assertEquals("TestString", options2.getValue().getValue()); + } + + private T serializeDeserialize(Class kls, PipelineOptions options) + throws Exception { + ObjectMapper mapper = new ObjectMapper(); + String value = mapper.writeValueAsString(options); + return mapper.readValue(value, PipelineOptions.class).as(kls); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunnerTest.java new file mode 100644 index 0000000000000..398326e8a385e --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunnerTest.java @@ -0,0 +1,137 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.isA; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.testing.ExpectedLogs; +import com.google.cloud.dataflow.sdk.util.MonitoringUtil; +import com.google.cloud.dataflow.sdk.util.MonitoringUtil.JobState; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.io.IOException; +import java.util.Date; +import java.util.concurrent.TimeUnit; + +/** + * Tests for BlockingDataflowPipelineRunner. + */ +@RunWith(JUnit4.class) +public class BlockingDataflowPipelineRunnerTest { + @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(BlockingDataflowPipelineRunner.class); + + // This class mocks a call to DataflowPipelineJob.waitToFinish(): + // it blocks the thread to simulate waiting, + // and releases the blocking once signaled + static class MockWaitToFinish implements Answer { + NotificationHelper jobCompleted = new NotificationHelper(); + + public Object answer(InvocationOnMock invocation) throws InterruptedException { + System.out.println("MockWaitToFinish.answer(): Wait for signaling job completion."); + assertTrue("Test did not receive mock job completion signal", + jobCompleted.waitTillSet(10000)); + + System.out.println("MockWaitToFinish.answer(): job completed."); + return JobState.DONE; + } + + public void signalJobComplete() { + jobCompleted.set(); + } + } + + // Mini helper class for wait-notify + static class NotificationHelper { + private boolean isSet = false; + + public synchronized void set() { + isSet = true; + notifyAll(); + } + + public synchronized boolean check() { + return isSet; + } + + public synchronized boolean waitTillSet(long timeout) throws InterruptedException { + long remainingTimeout = timeout; + long startTime = new Date().getTime(); + while (!isSet && remainingTimeout > 0) { + wait(remainingTimeout); + remainingTimeout = timeout - (new Date().getTime() - startTime); + } + + return isSet; + } + } + + @Test + public void testJobWaitComplete() throws IOException, InterruptedException { + expectedLogs.expectInfo("Job finished with status DONE"); + + DataflowPipelineRunner mockDataflowPipelineRunner = mock(DataflowPipelineRunner.class); + DataflowPipelineJob mockJob = mock(DataflowPipelineJob.class); + MockWaitToFinish mockWait = new MockWaitToFinish(); + + when(mockJob.waitToFinish( + anyLong(), isA(TimeUnit.class), isA(MonitoringUtil.JobMessagesHandler.class))) + .thenAnswer(mockWait); + when(mockDataflowPipelineRunner.run(isA(Pipeline.class))).thenReturn(mockJob); + + // Construct a BlockingDataflowPipelineRunner with mockDataflowPipelineRunner inside + final BlockingDataflowPipelineRunner blockingRunner = + new BlockingDataflowPipelineRunner( + mockDataflowPipelineRunner, + new MonitoringUtil.PrintHandler(System.out)); + + final NotificationHelper executionStarted = new NotificationHelper(); + final NotificationHelper jobCompleted = new NotificationHelper(); + + new Thread() { + public void run() { + executionStarted.set(); + + // Run on an empty test pipeline. + blockingRunner.run(DirectPipeline.createForTest()); + + // Test following code is not reached till mock job completion signal. + jobCompleted.set(); + } + }.start(); + + assertTrue("'executionStarted' event not set till timeout.", + executionStarted.waitTillSet(2000)); + assertFalse("Code after job completion should not be reached before mock signal.", + jobCompleted.check()); + + mockWait.signalJobComplete(); + assertTrue("run() should return after job completion is mocked.", + jobCompleted.waitTillSet(2000)); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJobTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJobTest.java new file mode 100644 index 0000000000000..30697deecc0fe --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJobTest.java @@ -0,0 +1,66 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.Job; +import com.google.cloud.dataflow.sdk.util.MonitoringUtil.JobState; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +/** + * Tests for DataflowPipelineJob. + */ +@RunWith(JUnit4.class) +public class DataflowPipelineJobTest { + private static final String PROJECT_ID = "someProject"; + private static final String JOB_ID = "1234"; + + @Test + public void testWaitToFinish() throws IOException, InterruptedException { + Dataflow mockWorkflowClient = mock(Dataflow.class); + Dataflow.V1b3 mockV1b3 = mock(Dataflow.V1b3.class); + Dataflow.V1b3.Projects mockProjects = mock(Dataflow.V1b3.Projects.class); + Dataflow.V1b3.Projects.Jobs mockJobs = mock(Dataflow.V1b3.Projects.Jobs.class); + Dataflow.V1b3.Projects.Jobs.Get statusRequest = mock(Dataflow.V1b3.Projects.Jobs.Get.class); + + Job statusResponse = new Job(); + statusResponse.setCurrentState(JobState.DONE.getStateName()); + + when(mockWorkflowClient.v1b3()).thenReturn(mockV1b3); + when(mockV1b3.projects()).thenReturn(mockProjects); + when(mockProjects.jobs()).thenReturn(mockJobs); + when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))) + .thenReturn(statusRequest); + when(statusRequest.execute()).thenReturn(statusResponse); + + DataflowPipelineJob job = new DataflowPipelineJob( + PROJECT_ID, JOB_ID, mockWorkflowClient); + + JobState state = job.waitToFinish(1, TimeUnit.MINUTES, null); + assertEquals(JobState.DONE, state); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java new file mode 100644 index 0000000000000..7995445c9869b --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java @@ -0,0 +1,501 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import static org.hamcrest.Matchers.containsString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.api.services.dataflow.model.Job; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo; +import com.google.cloud.dataflow.sdk.util.GcsUtil; +import com.google.cloud.dataflow.sdk.util.PackageUtil; +import com.google.cloud.dataflow.sdk.util.TestCredential; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.collect.ImmutableList; + +import org.hamcrest.Matchers; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.StandardOpenOption; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; + +/** + * Tests for DataflowPipelineRunner. + */ +@RunWith(JUnit4.class) +public class DataflowPipelineRunnerTest { + + @Rule public ExpectedException thrown = ExpectedException.none(); + + // Asserts that the given Job has all expected fields set. + private static void assertValidJob(Job job) { + assertNull(job.getId()); + assertNull(job.getCurrentState()); + } + + private DataflowPipeline buildDataflowPipeline(DataflowPipelineOptions options) { + DataflowPipeline p = DataflowPipeline.create(options); + + p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object")) + .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object")); + + return p; + } + + private static Dataflow buildMockDataflow( + final ArgumentCaptor jobCaptor) throws IOException { + Dataflow mockDataflowClient = mock(Dataflow.class); + Dataflow.V1b3 mockV1b3 = mock(Dataflow.V1b3.class); + Dataflow.V1b3.Projects mockProjects = mock(Dataflow.V1b3.Projects.class); + Dataflow.V1b3.Projects.Jobs mockJobs = mock(Dataflow.V1b3.Projects.Jobs.class); + Dataflow.V1b3.Projects.Jobs.Create mockRequest = + mock(Dataflow.V1b3.Projects.Jobs.Create.class); + + when(mockDataflowClient.v1b3()).thenReturn(mockV1b3); + when(mockV1b3.projects()).thenReturn(mockProjects); + when(mockProjects.jobs()).thenReturn(mockJobs); + when(mockJobs.create(eq("someProject"), jobCaptor.capture())) + .thenReturn(mockRequest); + + Job resultJob = new Job(); + resultJob.setId("newid"); + when(mockRequest.execute()).thenReturn(resultJob); + return mockDataflowClient; + } + + private GcsUtil buildMockGcsUtil() throws IOException { + GcsUtil mockGcsUtil = mock(GcsUtil.class); + when(mockGcsUtil.create( + any(GcsPath.class), anyString())) + .thenReturn(FileChannel.open( + Files.createTempFile("channel-", ".tmp"), + StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE)); + return mockGcsUtil; + } + + private DataflowPipelineOptions buildPipelineOptions( + ArgumentCaptor jobCaptor) throws IOException { + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setProject("someProject"); + options.setTempLocation(DataflowPipelineRunner.verifyGcsPath( + GcsPath.fromComponents("somebucket", "some/path")).toString()); + // Set FILES_PROPERTY to empty to prevent a default value calculated from classpath. + options.setFilesToStage(new LinkedList()); + options.setDataflowClient(buildMockDataflow(jobCaptor)); + options.setGcsUtil(buildMockGcsUtil()); + options.setGcpCredential(new TestCredential()); + return options; + } + + @Test + public void testRun() throws IOException { + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + + DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + DataflowPipeline p = buildDataflowPipeline(options); + DataflowPipelineJob job = p.run(); + assertEquals("newid", job.getJobId()); + assertValidJob(jobCaptor.getValue()); + } + + @Test + public void testRunWithFiles() throws IOException { + // Test that the function DataflowPipelineRunner.stageFiles works as + // expected. + GcsUtil mockGcsUtil = buildMockGcsUtil(); + final GcsPath gcsStaging = + GcsPath.fromComponents("somebucket", "some/path"); + final GcsPath gcsTemp = + GcsPath.fromComponents("somebucket", "some/temp/path"); + final String cloudDataflowDataset = "somedataset"; + + // Create some temporary files. + File temp1 = File.createTempFile("DataflowPipelineRunnerTest", "txt"); + temp1.deleteOnExit(); + File temp2 = File.createTempFile("DataflowPipelineRunnerTest2", "txt"); + temp2.deleteOnExit(); + + DataflowPackage expectedPackage1 = PackageUtil.createPackage( + temp1.getAbsolutePath(), gcsStaging, null); + + String overridePackageName = "alias.txt"; + DataflowPackage expectedPackage2 = PackageUtil.createPackage( + temp2.getAbsolutePath(), gcsStaging, overridePackageName); + + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setFilesToStage(ImmutableList.of( + temp1.getAbsolutePath(), + overridePackageName + "=" + temp2.getAbsolutePath())); + options.setStagingLocation(gcsStaging.toString()); + options.setTempLocation(gcsTemp.toString()); + options.setTempDatasetId(cloudDataflowDataset); + options.setProject("someProject"); + options.setJobName("job"); + options.setDataflowClient(buildMockDataflow(jobCaptor)); + options.setGcsUtil(mockGcsUtil); + options.setGcpCredential(new TestCredential()); + + DataflowPipeline p = buildDataflowPipeline(options); + + DataflowPipelineJob job = p.run(); + assertEquals("newid", job.getJobId()); + + Job workflowJob = jobCaptor.getValue(); + assertValidJob(workflowJob); + + assertEquals( + 2, + workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().size()); + DataflowPackage workflowPackage1 = + workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(0); + assertEquals(expectedPackage1.getName(), workflowPackage1.getName()); + assertEquals(expectedPackage1.getLocation(), workflowPackage1.getLocation()); + DataflowPackage workflowPackage2 = + workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(1); + assertEquals(expectedPackage2.getName(), workflowPackage2.getName()); + assertEquals(expectedPackage2.getLocation(), workflowPackage2.getLocation()); + + assertEquals( + gcsTemp.toResourceName(), + workflowJob.getEnvironment().getTempStoragePrefix()); + assertEquals( + cloudDataflowDataset, + workflowJob.getEnvironment().getDataset()); + assertEquals( + DataflowReleaseInfo.getReleaseInfo().getName(), + workflowJob.getEnvironment().getUserAgent().get("name")); + assertEquals( + DataflowReleaseInfo.getReleaseInfo().getVersion(), + workflowJob.getEnvironment().getUserAgent().get("version")); + } + + @Test + public void runWithDefaultFilesToStage() throws Exception { + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + + DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + options.setFilesToStage(null); + DataflowPipelineRunner.fromOptions(options); + assertTrue(!options.getFilesToStage().isEmpty()); + } + + @Test + public void detectClassPathResourceWithFileResources() throws Exception { + String path = "/tmp/file"; + String path2 = "/tmp/file2"; + URLClassLoader classLoader = new URLClassLoader(new URL[]{ + new URL("file://" + path), + new URL("file://" + path2) + }); + + assertEquals(ImmutableList.of(path, path2), + DataflowPipelineRunner.detectClassPathResourcesToStage(classLoader)); + } + + @Test + public void detectClassPathResourcesWithUnsupportedClassLoader() { + ClassLoader mockClassLoader = Mockito.mock(ClassLoader.class); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Unable to use ClassLoader to detect classpath elements."); + + DataflowPipelineRunner.detectClassPathResourcesToStage(mockClassLoader); + } + + @Test + public void detectClassPathResourceWithNonFileResources() throws Exception { + String url = "http://www.google.com/all-the-secrets.jar"; + URLClassLoader classLoader = new URLClassLoader(new URL[]{ + new URL(url) + }); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Unable to convert url (" + url + ") to file."); + + DataflowPipelineRunner.detectClassPathResourcesToStage(classLoader); + } + + @Test + public void testGcsStagingLocationInitialization() { + // Test that the staging location is initialized correctly. + GcsPath gcsTemp = GcsPath.fromComponents("somebucket", + "some/temp/path"); + + // Set temp location (required), and check that staging location is set. + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setTempLocation(gcsTemp.toString()); + options.setProject("testProject"); + options.setGcpCredential(new TestCredential()); + DataflowPipelineRunner.fromOptions(options); + + assertNotNull(options.getStagingLocation()); + } + + @Test + public void testGcsRequiredTempLocation() { + // Error raised if temp location not set. + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setProject("someProject"); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString("tempLocation")); + DataflowPipelineRunner.fromOptions(options); + } + + @Test + public void testNonGcsFilePathInReadFailure() throws IOException { + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + + Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor)); + p.apply(TextIO.Read.named("ReadMyNonGcsFile").from("/tmp/file")); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString("GCS URI")); + p.run(); + assertValidJob(jobCaptor.getValue()); + } + + @Test + public void testNonGcsFilePathInWriteFailure() throws IOException { + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + + Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor)); + p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object")) + .apply(TextIO.Write.named("WriteMyNonGcsFile").to("/tmp/file")); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString("GCS URI")); + p.run(); + assertValidJob(jobCaptor.getValue()); + } + + @Test + public void testMultiSlashGcsFileReadPath() throws IOException { + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + + Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor)); + p.apply(TextIO.Read.named("ReadInvalidGcsFile") + .from("gs://bucket/tmp//file")); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("consecutive slashes"); + p.run(); + assertValidJob(jobCaptor.getValue()); + } + + @Test + public void testMultiSlashGcsFileWritePath() throws IOException { + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + + Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor)); + p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object")) + .apply(TextIO.Write.named("WriteInvalidGcsFile") + .to("gs://bucket/tmp//file")); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("consecutive slashes"); + p.run(); + assertValidJob(jobCaptor.getValue()); + } + + @Test + public void testInvalidTempLocation() throws IOException { + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + + DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + options.setTempLocation("file://temp/location"); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString("GCS URI")); + DataflowPipelineRunner.fromOptions(options); + assertValidJob(jobCaptor.getValue()); + } + + @Test + public void testInvalidStagingLocation() throws IOException { + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + + DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + options.setStagingLocation("file://my/staging/location"); + try { + DataflowPipelineRunner.fromOptions(options); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("GCS URI")); + } + options.setStagingLocation("my/staging/location"); + try { + DataflowPipelineRunner.fromOptions(options); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), containsString("GCS URI")); + } + } + + @Test + public void testInvalidJobName() throws IOException { + List invalidNames = Arrays.asList( + "invalid_name", + "0invalid", + "invalid-", + "this-one-is-too-long-01234567890123456789"); + List expectedReason = Arrays.asList( + "JobName invalid", + "JobName invalid", + "JobName invalid", + "JobName too long"); + + for (int i = 0; i < invalidNames.size(); ++i) { + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + + DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + options.setJobName(invalidNames.get(i)); + + try { + DataflowPipelineRunner.fromOptions(options); + fail("Expected IllegalArgumentException for jobName " + + options.getJobName()); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), + containsString(expectedReason.get(i))); + } + } + } + + @Test + public void testValidJobName() throws IOException { + List names = Arrays.asList("ok", "Ok", "A-Ok", "ok-123"); + + for (String name : names) { + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + + DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + options.setJobName(name); + + DataflowPipelineRunner runner = DataflowPipelineRunner + .fromOptions(options); + assertNotNull(runner); + } + } + + /** + * A fake PTransform for testing. + */ + public static class TestTransform + extends PTransform, PCollection> { + public boolean translated = false; + + @Override + public PCollection apply(PCollection input) { + return PCollection.createPrimitiveOutputInternal(new GlobalWindow()); + } + + @Override + protected Coder getDefaultOutputCoder() { + return getInput().getCoder(); + } + } + + @Test + public void testTransformTranslatorMissing() throws IOException { + // Test that we throw if we don't provide a translation. + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + + DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + Pipeline p = DataflowPipeline.create(options); + + p.apply(Create.of(Arrays.asList(1, 2, 3))) + .apply(new TestTransform()); + + thrown.expect(IllegalStateException.class); + thrown.expectMessage(Matchers.containsString("no translator registered")); + DataflowPipelineTranslator.fromOptions(options) + .translate(p, Collections.emptyList()); + assertValidJob(jobCaptor.getValue()); + } + + @Test + public void testTransformTranslator() throws IOException { + // Test that we can provide a custom translation + ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); + + DataflowPipelineOptions options = buildPipelineOptions(jobCaptor); + DataflowPipeline p = DataflowPipeline.create(options); + TestTransform transform = new TestTransform(); + + p.apply(Create.of(Arrays.asList(1, 2, 3))) + .apply(transform) + .setCoder(BigEndianIntegerCoder.of()); + + DataflowPipelineTranslator translator = DataflowPipelineRunner + .fromOptions(options).getTranslator(); + + translator.registerTransformTranslator( + TestTransform.class, + new DataflowPipelineTranslator.TransformTranslator() { + @SuppressWarnings("unchecked") + @Override + public void translate( + TestTransform transform, + DataflowPipelineTranslator.TranslationContext context) { + transform.translated = true; + + // Note: This is about the minimum needed to fake out a + // translation. This obviously isn't a real translation. + context.addStep(transform, "TestTranslate"); + context.addOutput("output", transform.getOutput()); + } + }); + + translator.translate(p, Collections.emptyList()); + assertTrue(transform.translated); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java new file mode 100644 index 0000000000000..e2edb9fdc2232 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java @@ -0,0 +1,582 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import static com.google.cloud.dataflow.sdk.util.Structs.addObject; +import static com.google.cloud.dataflow.sdk.util.Structs.getDictionary; +import static com.google.cloud.dataflow.sdk.util.Structs.getString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.argThat; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.Step; +import com.google.api.services.dataflow.model.WorkerPool; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.VarIntCoder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.util.OutputReference; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.TestCredential; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionTuple; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.collect.Iterables; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentMatcher; + +import java.io.IOException; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * Tests for DataflowPipelineTranslator. + */ +@RunWith(JUnit4.class) +public class DataflowPipelineTranslatorTest { + + @Rule public ExpectedException thrown = ExpectedException.none(); + + // A Custom Mockito matcher for an initial Job which checks that all + // expected fields are set. + private static class IsValidCreateRequest extends ArgumentMatcher { + public boolean matches(Object o) { + Job job = (Job) o; + return job.getId() == null + && job.getProjectId() == null + && job.getName() != null + && job.getType() != null + && job.getEnvironment() != null + && job.getSteps() != null + && job.getCurrentState() == null + && job.getCurrentStateTime() == null + && job.getExecutionInfo() == null + && job.getCreateTime() == null; + } + } + + private DataflowPipeline buildPipeline(DataflowPipelineOptions options) + throws IOException { + DataflowPipeline p = DataflowPipeline.create(options); + + p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object")) + .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object")); + + return p; + } + + private static Dataflow buildMockDataflow( + ArgumentMatcher jobMatcher) throws IOException { + Dataflow mockDataflowClient = mock(Dataflow.class); + Dataflow.V1b3 mockV1b3 = mock(Dataflow.V1b3.class); + Dataflow.V1b3.Projects mockProjects = mock(Dataflow.V1b3.Projects.class); + Dataflow.V1b3.Projects.Jobs mockJobs = mock(Dataflow.V1b3.Projects.Jobs.class); + Dataflow.V1b3.Projects.Jobs.Create mockRequest = mock( + Dataflow.V1b3.Projects.Jobs.Create.class); + + when(mockDataflowClient.v1b3()).thenReturn(mockV1b3); + when(mockV1b3.projects()).thenReturn(mockProjects); + when(mockProjects.jobs()).thenReturn(mockJobs); + when(mockJobs.create(eq("someProject"), argThat(jobMatcher))) + .thenReturn(mockRequest); + + Job resultJob = new Job(); + resultJob.setId("newid"); + when(mockRequest.execute()).thenReturn(resultJob); + return mockDataflowClient; + } + + private static DataflowPipelineOptions buildPipelineOptions() throws IOException { + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setGcpCredential(new TestCredential()); + options.setProject("some-project"); + options.setTempLocation(GcsPath.fromComponents("somebucket", "some/path").toString()); + options.setFilesToStage(new LinkedList()); + options.setDataflowClient(buildMockDataflow(new IsValidCreateRequest())); + return options; + } + + @Test + public void testZoneConfig() throws IOException { + final String testZone = "test-zone-1"; + + DataflowPipelineOptions options = buildPipelineOptions(); + options.setZone(testZone); + + Pipeline p = buildPipeline(options); + p.traverseTopologically(new RecordingPipelineVisitor()); + Job job = DataflowPipelineTranslator.fromOptions(options).translate( + p, Collections.emptyList()); + + assertEquals(2, job.getEnvironment().getWorkerPools().size()); + assertEquals(testZone, + job.getEnvironment().getWorkerPools().get(0).getZone()); + assertEquals(testZone, + job.getEnvironment().getWorkerPools().get(1).getZone()); + } + + @Test + public void testWorkerMachineTypeConfig() throws IOException { + final String testMachineType = "test-machine-type"; + + DataflowPipelineOptions options = buildPipelineOptions(); + options.setWorkerMachineType(testMachineType); + + Pipeline p = buildPipeline(options); + p.traverseTopologically(new RecordingPipelineVisitor()); + Job job = DataflowPipelineTranslator.fromOptions(options).translate( + p, Collections.emptyList()); + + assertEquals(2, job.getEnvironment().getWorkerPools().size()); + + WorkerPool workerPool = null; + + if (job + .getEnvironment() + .getWorkerPools() + .get(0) + .getKind() + .equals(DataflowPipelineTranslator.HARNESS_WORKER_POOL)) { + workerPool = job.getEnvironment().getWorkerPools().get(0); + } else if (job + .getEnvironment() + .getWorkerPools() + .get(1) + .getKind() + .equals(DataflowPipelineTranslator.HARNESS_WORKER_POOL)) { + workerPool = job.getEnvironment().getWorkerPools().get(1); + } else { + fail("Missing worker pool."); + } + assertEquals(testMachineType, workerPool.getMachineType()); + } + + @Test + public void testDiskSizeGbConfig() throws IOException { + final Integer diskSizeGb = 1234; + + DataflowPipelineOptions options = buildPipelineOptions(); + options.setDiskSizeGb(diskSizeGb); + + Pipeline p = buildPipeline(options); + p.traverseTopologically(new RecordingPipelineVisitor()); + Job job = DataflowPipelineTranslator.fromOptions(options).translate( + p, Collections.emptyList()); + + assertEquals(2, job.getEnvironment().getWorkerPools().size()); + assertEquals(diskSizeGb, + job.getEnvironment().getWorkerPools().get(0).getDiskSizeGb()); + assertEquals(diskSizeGb, + job.getEnvironment().getWorkerPools().get(1).getDiskSizeGb()); + } + + @Test + public void testShufflePoolConfig() throws IOException { + final Integer numWorkers = 10; + final String diskSource = "test-disk-source"; + final Integer diskSizeGb = 12345; + final String zone = "test-zone-1"; + + DataflowPipelineOptions options = buildPipelineOptions(); + options.setShuffleNumWorkers(numWorkers); + options.setShuffleDiskSourceImage(diskSource); + options.setShuffleDiskSizeGb(diskSizeGb); + options.setShuffleZone(zone); + + Pipeline p = buildPipeline(options); + p.traverseTopologically(new RecordingPipelineVisitor()); + Job job = DataflowPipelineTranslator.fromOptions(options).translate( + p, Collections.emptyList()); + + assertEquals(2, job.getEnvironment().getWorkerPools().size()); + WorkerPool shufflePool = + job.getEnvironment().getWorkerPools().get(1); + assertEquals(shufflePool.getKind(), + DataflowPipelineTranslator.SHUFFLE_WORKER_POOL); + assertEquals(numWorkers, shufflePool.getNumWorkers()); + assertEquals(diskSource, shufflePool.getDiskSourceImage()); + assertEquals(diskSizeGb, shufflePool.getDiskSizeGb()); + assertEquals(zone, shufflePool.getZone()); + } + + @Test + public void testPredefinedAddStep() throws Exception { + DataflowPipelineOptions options = buildPipelineOptions(); + + DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options); + DataflowPipelineTranslator.registerTransformTranslator( + EmbeddedTransform.class, new EmbeddedTranslator()); + + // Create a predefined step using another pipeline + Step predefinedStep = createPredefinedStep(); + + // Create a pipeline that the predefined step will be embedded into + DataflowPipeline pipeline = DataflowPipeline.create(options); + pipeline.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/in")) + .apply(ParDo.of(new NoOpFn())) + .apply(new EmbeddedTransform(predefinedStep.clone())) + .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/out")); + Job job = translator.translate(pipeline, Collections.emptyList()); + + List steps = job.getSteps(); + assertEquals(4, steps.size()); + + // The input to the embedded step should match the output of the step before + Map step1Out = getOutputPortReference(steps.get(1)); + Map step2In = getDictionary( + steps.get(2).getProperties(), PropertyNames.PARALLEL_INPUT); + assertEquals(step1Out, step2In); + + // The output from the embedded step should match the input of the step after + Map step2Out = getOutputPortReference(steps.get(2)); + Map step3In = getDictionary( + steps.get(3).getProperties(), PropertyNames.PARALLEL_INPUT); + assertEquals(step2Out, step3In); + + // The step should not have been modified other than remapping the input + Step predefinedStepClone = predefinedStep.clone(); + Step embeddedStepClone = steps.get(2).clone(); + predefinedStepClone.getProperties().remove(PropertyNames.PARALLEL_INPUT); + embeddedStepClone.getProperties().remove(PropertyNames.PARALLEL_INPUT); + assertEquals(predefinedStepClone, embeddedStepClone); + } + + /** + * Construct a OutputReference for the output of the step. + */ + private static OutputReference getOutputPortReference(Step step) throws Exception { + // TODO: This should be done via a Structs accessor. + List> output = + (List>) step.getProperties().get(PropertyNames.OUTPUT_INFO); + String outputTagId = getString(Iterables.getOnlyElement(output), PropertyNames.OUTPUT_NAME); + return new OutputReference(step.getName(), outputTagId); + } + + /** + * Returns a Step for a DoFn by creating and translating a pipeline. + */ + private static Step createPredefinedStep() throws Exception { + DataflowPipelineOptions options = buildPipelineOptions(); + DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options); + DataflowPipeline pipeline = DataflowPipeline.create(options); + String stepName = "DoFn1"; + pipeline.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/in")) + .apply(ParDo.of(new NoOpFn()).named(stepName)) + .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/out")); + Job job = translator.translate(pipeline, Collections.emptyList()); + + assertEquals(3, job.getSteps().size()); + Step step = job.getSteps().get(1); + assertEquals(stepName, getString(step.getProperties(), PropertyNames.USER_NAME)); + return step; + } + + private static class NoOpFn extends DoFn{ + @Override public void processElement(ProcessContext c) throws Exception { + c.output(c.element()); + } + } + + /** + * A placeholder transform that will be used to substitute a predefined Step. + */ + private static class EmbeddedTransform + extends PTransform, PCollection> { + private final Step step; + + public EmbeddedTransform(Step step) { + this.step = step; + } + + @Override + public PCollection apply(PCollection input) { + return PCollection.createPrimitiveOutputInternal(new GlobalWindow()); + } + + @Override + protected Coder getDefaultOutputCoder() { + return StringUtf8Coder.of(); + } + } + + /** + * A TransformTranslator that adds the predefined Step using + * {@link TranslationContext#addStep} and remaps the input port reference. + */ + private static class EmbeddedTranslator + implements DataflowPipelineTranslator.TransformTranslator { + @Override public void translate(EmbeddedTransform transform, TranslationContext context) { + addObject(transform.step.getProperties(), PropertyNames.PARALLEL_INPUT, + context.asOutputReference(transform.getInput())); + context.addStep(transform, transform.step); + } + } + + /** + * A composite transform which returns an output that is unrelated to + * the input. + */ + private static class UnrelatedOutputCreator + extends PTransform, PCollection> { + + @Override + public PCollection apply(PCollection input) { + // Apply an operation so that this is a composite transform. + input.apply(Count.perElement()); + + // Return a value unrelated to the input. + return input.getPipeline().apply(Create.of(1, 2, 3, 4)); + } + + @Override + protected Coder getDefaultOutputCoder() { + return VarIntCoder.of(); + } + } + + /** + * A composite transform which returns an output which is unbound. + */ + private static class UnboundOutputCreator + extends PTransform, PDone> { + + @Override + public PDone apply(PCollection input) { + // Apply an operation so that this is a composite transform. + input.apply(Count.perElement()); + + return new PDone(); + } + + @Override + protected Coder getDefaultOutputCoder() { + return VoidCoder.of(); + } + } + + /** + * A composite transform which returns a partially bound output. + * + *

This is not allowed and will result in a failure. + */ + private static class PartiallyBoundOutputCreator + extends PTransform, PCollectionTuple> { + + public final TupleTag sumTag = new TupleTag<>("sum"); + public final TupleTag doneTag = new TupleTag<>("done"); + + @Override + public PCollectionTuple apply(PCollection input) { + PCollection sum = input.apply(Sum.integersGlobally()); + + // Fails here when attempting to construct a tuple with an unbound object. + return PCollectionTuple.of(sumTag, sum) + .and(doneTag, PCollection.createPrimitiveOutputInternal( + new GlobalWindow())); + } + } + + @Test + public void testMultiGraphPipelineSerialization() throws IOException { + Pipeline p = DataflowPipeline.create(buildPipelineOptions()); + + PCollection input = p.begin() + .apply(Create.of(1, 2, 3)); + + input.apply(new UnrelatedOutputCreator()); + input.apply(new UnboundOutputCreator()); + + DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions( + PipelineOptionsFactory.as(DataflowPipelineOptions.class)); + + // Check that translation doesn't fail. + t.translate(p, Collections.emptyList()); + } + + @Test + public void testPartiallyBoundFailure() throws IOException { + Pipeline p = DataflowPipeline.create(buildPipelineOptions()); + + PCollection input = p.begin() + .apply(Create.of(1, 2, 3)); + + thrown.expect(IllegalStateException.class); + input.apply(new PartiallyBoundOutputCreator()); + + Assert.fail("Failure expected from use of partially bound output"); + } + + /** + * The first wildcard must occur after the last directory delimiter. + * This tests a few corner cases that should not crash. + */ + @Test + public void testGoodWildcards() throws Exception { + DataflowPipelineOptions options = buildPipelineOptions(); + Pipeline pipeline = DataflowPipeline.create(options); + DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options); + + pipeline.apply(TextIO.Read.from("gs://bucket/foo")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/*")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/?")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/[0-9]")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/*baz*")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/*baz?")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/[0-9]baz?")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/baz/*")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo/baz/*wonka*")); + + // Check that translation doesn't fail. + t.translate(pipeline, Collections.emptyList()); + } + + /** + * The first wildcard must occur after the last directory delimiter. + * This tests "*". + */ + @Test + public void testBadWildcardStar() throws Exception { + DataflowPipelineOptions options = buildPipelineOptions(); + Pipeline pipeline = DataflowPipeline.create(options); + DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options); + + pipeline.apply(TextIO.Read.from("gs://bucket/foo*/baz")); + + // Check that translation does fail. + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Unsupported wildcard usage"); + t.translate(pipeline, Collections.emptyList()); + } + + /** + * The first wildcard must occur after the last directory delimiter. + * This tests "?". + */ + @Test + public void testBadWildcardOptional() throws Exception { + DataflowPipelineOptions options = buildPipelineOptions(); + Pipeline pipeline = DataflowPipeline.create(options); + DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options); + + pipeline.apply(TextIO.Read.from("gs://bucket/foo?/baz")); + + // Check that translation does fail. + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Unsupported wildcard usage"); + t.translate(pipeline, Collections.emptyList()); + } + + /** + * The first wildcard must occur after the last directory delimiter. + * This tests "[]" based character classes. + */ + @Test + public void testBadWildcardBrackets() throws Exception { + DataflowPipelineOptions options = buildPipelineOptions(); + Pipeline pipeline = DataflowPipeline.create(options); + DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options); + + pipeline.apply(TextIO.Read.from("gs://bucket/foo[0-9]/baz")); + + // Check that translation does fail. + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Unsupported wildcard usage"); + t.translate(pipeline, Collections.emptyList()); + } + + @Test + public void testToSingletonTranslation() throws Exception { + // A "change detector" test that makes sure the translation + // of getting a PCollectionView does not change + // in bad ways during refactor + + DataflowPipelineOptions options = buildPipelineOptions(); + DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options); + + DataflowPipeline pipeline = DataflowPipeline.create(options); + PCollectionView view = pipeline + .apply(Create.of(1)) + .apply(View.asSingleton()); + Job job = translator.translate(pipeline, Collections.emptyList()); + + List steps = job.getSteps(); + assertEquals(2, steps.size()); + + Step createStep = steps.get(0); + assertEquals("CreateCollection", createStep.getKind()); + + Step collectionToSingletonStep = steps.get(1); + assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind()); + + } + + @Test + public void testToIterableTranslation() throws Exception { + // A "change detector" test that makes sure the translation + // of getting a PCollectionView, ...> does not change + // in bad ways during refactor + + DataflowPipelineOptions options = buildPipelineOptions(); + DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options); + + DataflowPipeline pipeline = DataflowPipeline.create(options); + PCollectionView, ?> view = pipeline + .apply(Create.of(1, 2, 3)) + .apply(View.asIterable()); + Job job = translator.translate(pipeline, Collections.emptyList()); + + List steps = job.getSteps(); + assertEquals(2, steps.size()); + + Step createStep = steps.get(0); + assertEquals("CreateCollection", createStep.getKind()); + + Step collectionToSingletonStep = steps.get(1); + assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerTest.java new file mode 100644 index 0000000000000..520e03e28b9dd --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerTest.java @@ -0,0 +1,84 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import static org.junit.Assert.assertTrue; + +import com.google.api.services.dataflow.Dataflow; +import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions; +import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.GcsUtil; +import com.google.cloud.dataflow.sdk.util.TestCredential; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.io.IOException; + +/** + * Tests for DataflowPipelineRunner. + */ +@RunWith(JUnit4.class) +public class PipelineRunnerTest { + + @Mock private Dataflow mockDataflow; + @Mock private GcsUtil mockGcsUtil; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + } + + @Test + public void testLongName() throws IOException { + // Check we can create a pipeline runner using the full class name. + DirectPipelineOptions options = PipelineOptionsFactory.as(DirectPipelineOptions.class); + options.setAppName("test"); + options.setProject("test"); + options.setGcsUtil(mockGcsUtil); + options.setRunner(DirectPipelineRunner.class); + options.setGcpCredential(new TestCredential()); + PipelineRunner runner = PipelineRunner.fromOptions(options); + assertTrue(runner instanceof DirectPipelineRunner); + } + + @Test + public void testShortName() throws IOException { + // Check we can create a pipeline runner using the short class name. + DirectPipelineOptions options = PipelineOptionsFactory.as(DirectPipelineOptions.class); + options.setAppName("test"); + options.setProject("test"); + options.setGcsUtil(mockGcsUtil); + options.setRunner(DirectPipelineRunner.class); + options.setGcpCredential(new TestCredential()); + PipelineRunner runner = PipelineRunner.fromOptions(options); + assertTrue(runner instanceof DirectPipelineRunner); + } + + @Test + public void testAppNameDefault() throws IOException { + ApplicationNameOptions options = PipelineOptionsFactory.as(ApplicationNameOptions.class); + Assert.assertEquals(PipelineRunnerTest.class.getSimpleName(), + options.getAppName()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/TransformTreeTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/TransformTreeTest.java new file mode 100644 index 0000000000000..d0308e87a33f5 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/TransformTreeTest.java @@ -0,0 +1,179 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.not; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.First; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.values.PBegin; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.cloud.dataflow.sdk.values.PValue; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.util.Arrays; +import java.util.EnumSet; + +/** + * Tests for {@link TransformTreeNode} and {@link TransformHierarchy}. + */ +@RunWith(JUnit4.class) +public class TransformTreeTest { + + enum TransformsSeen { + READ, + WRITE, + FIRST + } + + /** + * INVALID TRANSFORM, DO NOT COPY. + * + *

This is an invalid composite transform, which returns unbound outputs. + * This should never happen, and is here to test that it is properly rejected. + */ + private static class InvalidCompositeTransform + extends PTransform> { + + @Override + public PCollectionList apply(PBegin b) { + // Composite transform: apply delegates to other transformations, + // here a Create transform. + PCollection result = b.apply(Create.of("hello", "world")); + + // Issue below: PCollection.createPrimitiveOutput should not be used + // from within a composite transform. + return PCollectionList.of( + Arrays.asList(result, PCollection.createPrimitiveOutputInternal( + new GlobalWindow()))); + } + } + + /** + * A composite transform which returns an output which is unbound. + */ + private static class UnboundOutputCreator + extends PTransform, PDone> { + + @Override + public PDone apply(PCollection input) { + // Apply an operation so that this is a composite transform. + input.apply(Count.perElement()); + + return new PDone(); + } + + @Override + protected Coder getDefaultOutputCoder() { + return VoidCoder.of(); + } + } + + // Builds a pipeline containing a composite operation (First), then + // visits the nodes and verifies that the hierarchy was captured. + @Test + public void testCompositeCapture() throws Exception { + Pipeline p = DirectPipeline.createForTest(); + + p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object")) + .apply(First.of(10)) + .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object")); + + final EnumSet visited = + EnumSet.noneOf(TransformsSeen.class); + final EnumSet left = + EnumSet.noneOf(TransformsSeen.class); + + p.traverseTopologically(new Pipeline.PipelineVisitor() { + @Override + public void enterCompositeTransform(TransformTreeNode node) { + PTransform transform = node.getTransform(); + if (transform instanceof First) { + Assert.assertTrue(visited.add(TransformsSeen.FIRST)); + Assert.assertNotNull(node.getEnclosingNode()); + Assert.assertTrue(node.isCompositeNode()); + } + Assert.assertThat(transform, not(instanceOf(TextIO.Read.Bound.class))); + Assert.assertThat(transform, not(instanceOf(TextIO.Write.Bound.class))); + } + + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + PTransform transform = node.getTransform(); + if (transform instanceof First) { + Assert.assertTrue(left.add(TransformsSeen.FIRST)); + } + } + + @Override + public void visitTransform(TransformTreeNode node) { + PTransform transform = node.getTransform(); + // First is a composite, should not be visited here. + Assert.assertThat(transform, not(instanceOf(First.class))); + if (transform instanceof TextIO.Read.Bound) { + Assert.assertTrue(visited.add(TransformsSeen.READ)); + } else if (transform instanceof TextIO.Write.Bound) { + Assert.assertTrue(visited.add(TransformsSeen.WRITE)); + } + } + + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + } + }); + + Assert.assertTrue(visited.equals(EnumSet.allOf(TransformsSeen.class))); + Assert.assertTrue(left.equals(EnumSet.of(TransformsSeen.FIRST))); + } + + @Test(expected = IllegalStateException.class) + public void testOutputChecking() throws Exception { + Pipeline p = DirectPipeline.createForTest(); + + p.apply(new InvalidCompositeTransform()); + + p.traverseTopologically(new RecordingPipelineVisitor()); + Assert.fail("traversal should have failed with an IllegalStateException"); + } + + @Test + public void testMultiGraphSetup() throws IOException { + Pipeline p = DirectPipeline.createForTest(); + + PCollection input = p.begin() + .apply(Create.of(1, 2, 3)); + + input.apply(new UnboundOutputCreator()); + + p.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSinkTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSinkTest.java new file mode 100644 index 0000000000000..3c7f29b40a8c1 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSinkTest.java @@ -0,0 +1,114 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.TestUtils; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; + +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.file.SeekableInput; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.io.DatumReader; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.File; +import java.nio.ByteBuffer; +import java.nio.channels.SeekableByteChannel; +import java.util.ArrayList; +import java.util.List; + +/** + * Tests for AvroByteSink. + */ +@RunWith(JUnit4.class) +public class AvroByteSinkTest { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + void runTestWriteFile(List elems, Coder coder) throws Exception { + File tmpFile = tmpFolder.newFile("file.avro"); + String filename = tmpFile.getPath(); + + // Write the file. + + AvroByteSink avroSink = new AvroByteSink<>(filename, coder); + List actualSizes = new ArrayList<>(); + try (Sink.SinkWriter writer = avroSink.writer()) { + for (T elem : elems) { + actualSizes.add(writer.add(elem)); + } + } + + // Read back the file. + + SeekableByteChannel inChannel = (SeekableByteChannel) + IOChannelUtils.getFactory(filename).open(filename); + + SeekableInput seekableInput = + new AvroSource.SeekableByteChannelInput(inChannel); + + Schema schema = Schema.create(Schema.Type.BYTES); + + DatumReader datumReader = new GenericDatumReader<>(schema); + + DataFileReader fileReader = new DataFileReader<>( + seekableInput, datumReader); + + List actual = new ArrayList<>(); + List expectedSizes = new ArrayList<>(); + ByteBuffer inBuffer = ByteBuffer.allocate(10 * 1024); + while (fileReader.hasNext()) { + inBuffer = fileReader.next(inBuffer); + byte[] encodedElem = new byte[inBuffer.remaining()]; + inBuffer.get(encodedElem); + assert inBuffer.remaining() == 0; + inBuffer.clear(); + T elem = CoderUtils.decodeFromByteArray(coder, encodedElem); + actual.add(elem); + expectedSizes.add((long) encodedElem.length); + } + + fileReader.close(); + + // Compare the expected and the actual elements. + Assert.assertEquals(elems, actual); + Assert.assertEquals(expectedSizes, actualSizes); + } + + @Test + public void testWriteFile() throws Exception { + runTestWriteFile(TestUtils.INTS, BigEndianIntegerCoder.of()); + } + + @Test + public void testWriteEmptyFile() throws Exception { + runTestWriteFile(TestUtils.NO_INTS, BigEndianIntegerCoder.of()); + } + + // TODO: sharded filenames + // TODO: writing to GCS +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSourceTest.java new file mode 100644 index 0000000000000..e6bfffdcb68ab --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSourceTest.java @@ -0,0 +1,200 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.TestUtils; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.MimeTypes; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.io.DatumWriter; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.File; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +import javax.annotation.Nullable; + +/** + * Tests for AvroByteSource. + */ +@RunWith(JUnit4.class) +public class AvroByteSourceTest { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + private void runTestRead(List> elemsList, + Coder coder, + boolean requireExactMatch) + throws Exception { + File tmpFile = tmpFolder.newFile("file.avro"); + String filename = tmpFile.getPath(); + + // Write the data. + OutputStream outStream = Channels.newOutputStream( + IOChannelUtils.create(filename, MimeTypes.BINARY)); + Schema schema = Schema.create(Schema.Type.BYTES); + DatumWriter datumWriter = new GenericDatumWriter<>(schema); + DataFileWriter fileWriter = new DataFileWriter<>(datumWriter); + fileWriter.create(schema, outStream); + boolean first = true; + List syncPoints = new ArrayList<>(); + List expectedSizes = new ArrayList<>(); + for (List elems : elemsList) { + if (first) { + first = false; + } else { + // Ensure a block boundary here. + long syncPoint = fileWriter.sync(); + syncPoints.add(syncPoint); + } + for (T elem : elems) { + byte[] encodedElem = CoderUtils.encodeToByteArray(coder, elem); + fileWriter.append(ByteBuffer.wrap(encodedElem)); + expectedSizes.add(encodedElem.length); + } + } + fileWriter.close(); + + // Test reading the data back. + List> actualElemsList = new ArrayList<>(); + List actualSizes = new ArrayList<>(); + Long startOffset = null; + Long endOffset; + long prevSyncPoint = 0; + for (long syncPoint : syncPoints) { + endOffset = (prevSyncPoint + syncPoint) / 2; + actualElemsList.add(readElems(filename, startOffset, endOffset, coder, actualSizes)); + startOffset = endOffset; + prevSyncPoint = syncPoint; + } + actualElemsList.add(readElems(filename, startOffset, null, coder, actualSizes)); + + // Compare the expected and the actual elements. + if (requireExactMatch) { + // Require the blocks to match exactly. (This works only for + // small block sizes. Large block sizes, bigger than Avro's + // internal sizes, lead to different splits.) + Assert.assertEquals(elemsList, actualElemsList); + } else { + // Just require the overall elements to be the same. (This + // works for any block size.) + List expected = new ArrayList<>(); + for (List elems : elemsList) { + expected.addAll(elems); + } + List actual = new ArrayList<>(); + for (List actualElems : actualElemsList) { + actual.addAll(actualElems); + } + Assert.assertEquals(expected, actual); + } + + Assert.assertEquals(expectedSizes, actualSizes); + } + + private List readElems(String filename, + @Nullable Long startOffset, + @Nullable Long endOffset, + Coder coder, + List actualSizes) + throws Exception { + AvroByteSource avroSource = + new AvroByteSource<>(filename, startOffset, endOffset, coder); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(avroSource, actualSizes); + + List actualElems = new ArrayList<>(); + try (Source.SourceIterator iterator = avroSource.iterator()) { + while (iterator.hasNext()) { + actualElems.add(iterator.next()); + } + } + return actualElems; + } + + @Test + public void testRead() throws Exception { + runTestRead(Collections.singletonList(TestUtils.INTS), + BigEndianIntegerCoder.of(), + true /* require exact match */); + } + + @Test + public void testReadEmpty() throws Exception { + runTestRead(Collections.singletonList(TestUtils.NO_INTS), + BigEndianIntegerCoder.of(), + true /* require exact match */); + } + + private List> generateInputBlocks(int numBlocks, + int blockSizeBytes, + int averageLineSizeBytes) { + Random random = new Random(0); + List> blocks = new ArrayList<>(numBlocks); + for (int blockNum = 0; blockNum < numBlocks; blockNum++) { + int numLines = blockSizeBytes / averageLineSizeBytes; + List lines = new ArrayList<>(numLines); + for (int lineNum = 0; lineNum < numLines; lineNum++) { + int numChars = random.nextInt(averageLineSizeBytes * 2); + StringBuilder sb = new StringBuilder(); + for (int charNum = 0; charNum < numChars; charNum++) { + sb.appendCodePoint(random.nextInt('z' - 'a' + 1) + 'a'); + } + lines.add(sb.toString()); + } + blocks.add(lines); + } + return blocks; + } + + @Test + public void testReadSmallRanges() throws Exception { + runTestRead(generateInputBlocks(3, 50, 5), + StringUtf8Coder.of(), + true /* require exact match */); + } + + @Test + public void testReadBigRanges() throws Exception { + runTestRead(generateInputBlocks(10, 128 * 1024, 100), + StringUtf8Coder.of(), + false /* don't require exact match */); + } + + // TODO: sharded filenames + // TODO: reading from GCS +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactoryTest.java new file mode 100644 index 0000000000000..79653feabc4ce --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactoryTest.java @@ -0,0 +1,83 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; + +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for AvroSinkFactory. + */ +@RunWith(JUnit4.class) +public class AvroSinkFactoryTest { + private final String pathToAvroFile = "/path/to/file.avro"; + + Sink runTestCreateAvroSink(String filename, + CloudObject encoding) + throws Exception { + CloudObject spec = CloudObject.forClassName("AvroSink"); + addString(spec, "filename", filename); + + com.google.api.services.dataflow.model.Sink cloudSink = + new com.google.api.services.dataflow.model.Sink(); + cloudSink.setSpec(spec); + cloudSink.setCodec(encoding); + + Sink sink = SinkFactory.create(PipelineOptionsFactory.create(), cloudSink, + new BatchModeExecutionContext()); + return sink; + } + + @Test + public void testCreateAvroByteSink() throws Exception { + Coder coder = + WindowedValue.getValueOnlyCoder(BigEndianIntegerCoder.of()); + Sink sink = runTestCreateAvroSink( + pathToAvroFile, coder.asCloudObject()); + + Assert.assertThat(sink, new IsInstanceOf(AvroByteSink.class)); + AvroByteSink avroSink = (AvroByteSink) sink; + Assert.assertEquals(pathToAvroFile, avroSink.avroSink.filenamePrefix); + Assert.assertEquals(coder, avroSink.coder); + } + + @Test + public void testCreateAvroSink() throws Exception { + WindowedValue.WindowedValueCoder coder = + WindowedValue.getValueOnlyCoder(AvroCoder.of(Integer.class)); + Sink sink = runTestCreateAvroSink(pathToAvroFile, coder.asCloudObject()); + + Assert.assertThat(sink, new IsInstanceOf(AvroSink.class)); + AvroSink avroSink = (AvroSink) sink; + Assert.assertEquals(pathToAvroFile, avroSink.filenamePrefix); + Assert.assertEquals(coder.getValueCoder(), avroSink.avroCoder); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkTest.java new file mode 100644 index 0000000000000..5f22d2774f4be --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkTest.java @@ -0,0 +1,104 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.TestUtils; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; + +import org.apache.avro.file.DataFileReader; +import org.apache.avro.file.SeekableInput; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.io.DatumReader; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.File; +import java.nio.channels.SeekableByteChannel; +import java.util.ArrayList; +import java.util.List; + +/** + * Tests for AvroSink. + */ +@RunWith(JUnit4.class) +public class AvroSinkTest { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + void runTestWriteFile(List elems, AvroCoder coder) throws Exception { + File tmpFile = tmpFolder.newFile("file.avro"); + String filename = tmpFile.getPath(); + + // Write the file. + + AvroSink avroSink = new AvroSink<>(filename, WindowedValue.getValueOnlyCoder(coder)); + List actualSizes = new ArrayList<>(); + try (Sink.SinkWriter> writer = avroSink.writer()) { + for (T elem : elems) { + actualSizes.add(writer.add(WindowedValue.valueInGlobalWindow(elem))); + } + } + + // Read back the file. + + SeekableByteChannel inChannel = (SeekableByteChannel) + IOChannelUtils.getFactory(filename).open(filename); + + SeekableInput seekableInput = + new AvroSource.SeekableByteChannelInput(inChannel); + + DatumReader datumReader = new GenericDatumReader<>(coder.getSchema()); + + DataFileReader fileReader = new DataFileReader<>( + seekableInput, datumReader); + + List actual = new ArrayList<>(); + List expectedSizes = new ArrayList<>(); + while (fileReader.hasNext()) { + T next = fileReader.next(); + actual.add(next); + expectedSizes.add((long) CoderUtils.encodeToByteArray(coder, next).length); + } + + fileReader.close(); + + // Compare the expected and the actual elements. + Assert.assertEquals(elems, actual); + Assert.assertEquals(expectedSizes, actualSizes); + } + + @Test + public void testWriteFile() throws Exception { + runTestWriteFile(TestUtils.INTS, AvroCoder.of(Integer.class)); + } + + @Test + public void testWriteEmptyFile() throws Exception { + runTestWriteFile(TestUtils.NO_INTS, AvroCoder.of(Integer.class)); + } + + // TODO: sharded filenames + // TODO: writing to GCS +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactoryTest.java new file mode 100644 index 0000000000000..3c81950fd29d6 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactoryTest.java @@ -0,0 +1,115 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.addLong; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import javax.annotation.Nullable; + +/** + * Tests for AvroSourceFactory. + */ +@RunWith(JUnit4.class) +public class AvroSourceFactoryTest { + private final String pathToAvroFile = "/path/to/file.avro"; + + Source runTestCreateAvroSource(String filename, + @Nullable Long start, + @Nullable Long end, + CloudObject encoding) + throws Exception { + CloudObject spec = CloudObject.forClassName("AvroSource"); + addString(spec, "filename", filename); + if (start != null) { + addLong(spec, "start_offset", start); + } + if (end != null) { + addLong(spec, "end_offset", end); + } + + com.google.api.services.dataflow.model.Source cloudSource = + new com.google.api.services.dataflow.model.Source(); + cloudSource.setSpec(spec); + cloudSource.setCodec(encoding); + + Source source = SourceFactory.create(PipelineOptionsFactory.create(), + cloudSource, + new BatchModeExecutionContext()); + return source; + } + + @Test + public void testCreatePlainAvroByteSource() throws Exception { + Coder coder = + WindowedValue.getValueOnlyCoder(BigEndianIntegerCoder.of()); + Source source = runTestCreateAvroSource( + pathToAvroFile, null, null, coder.asCloudObject()); + + Assert.assertThat(source, new IsInstanceOf(AvroByteSource.class)); + AvroByteSource avroSource = (AvroByteSource) source; + Assert.assertEquals(pathToAvroFile, avroSource.avroSource.filename); + Assert.assertEquals(null, avroSource.avroSource.startPosition); + Assert.assertEquals(null, avroSource.avroSource.endPosition); + Assert.assertEquals(coder, avroSource.coder); + } + + @Test + public void testCreateRichAvroByteSource() throws Exception { + Coder coder = + WindowedValue.getValueOnlyCoder(BigEndianIntegerCoder.of()); + Source source = runTestCreateAvroSource( + pathToAvroFile, 200L, 500L, coder.asCloudObject()); + + Assert.assertThat(source, new IsInstanceOf(AvroByteSource.class)); + AvroByteSource avroSource = (AvroByteSource) source; + Assert.assertEquals(pathToAvroFile, avroSource.avroSource.filename); + Assert.assertEquals(200L, (long) avroSource.avroSource.startPosition); + Assert.assertEquals(500L, (long) avroSource.avroSource.endPosition); + Assert.assertEquals(coder, avroSource.coder); + } + + @Test + public void testCreateRichAvroSource() throws Exception { + WindowedValue.WindowedValueCoder coder = + WindowedValue.getValueOnlyCoder(AvroCoder.of(Integer.class)); + Source source = runTestCreateAvroSource( + pathToAvroFile, 200L, 500L, coder.asCloudObject()); + + Assert.assertThat(source, new IsInstanceOf(AvroSource.class)); + AvroSource avroSource = (AvroSource) source; + Assert.assertEquals(pathToAvroFile, avroSource.filename); + Assert.assertEquals(200L, (long) avroSource.startPosition); + Assert.assertEquals(500L, (long) avroSource.endPosition); + Assert.assertEquals(coder.getValueCoder(), avroSource.avroCoder); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceTest.java new file mode 100644 index 0000000000000..4855ef92e4d96 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceTest.java @@ -0,0 +1,196 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.TestUtils; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.MimeTypes; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.io.DatumWriter; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.File; +import java.io.OutputStream; +import java.nio.channels.Channels; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +import javax.annotation.Nullable; + +/** + * Tests for AvroSource. + */ +@RunWith(JUnit4.class) +public class AvroSourceTest { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + private void runTestRead(List> elemsList, + AvroCoder coder, + boolean requireExactMatch) + throws Exception { + File tmpFile = tmpFolder.newFile("file.avro"); + String filename = tmpFile.getPath(); + + // Write the data. + OutputStream outStream = Channels.newOutputStream( + IOChannelUtils.create(filename, MimeTypes.BINARY)); + DatumWriter datumWriter = coder.createDatumWriter(); + DataFileWriter fileWriter = new DataFileWriter<>(datumWriter); + fileWriter.create(coder.getSchema(), outStream); + boolean first = true; + List syncPoints = new ArrayList<>(); + List expectedSizes = new ArrayList<>(); + for (List elems : elemsList) { + if (first) { + first = false; + } else { + // Ensure a block boundary here. + long syncPoint = fileWriter.sync(); + syncPoints.add(syncPoint); + } + for (T elem : elems) { + fileWriter.append(elem); + expectedSizes.add(CoderUtils.encodeToByteArray(coder, elem).length); + } + } + fileWriter.close(); + + // Test reading the data back. + List> actualElemsList = new ArrayList<>(); + List actualSizes = new ArrayList<>(); + Long startOffset = null; + Long endOffset; + long prevSyncPoint = 0; + for (long syncPoint : syncPoints) { + endOffset = (prevSyncPoint + syncPoint) / 2; + actualElemsList.add(readElems(filename, startOffset, endOffset, coder, actualSizes)); + startOffset = endOffset; + prevSyncPoint = syncPoint; + } + actualElemsList.add(readElems(filename, startOffset, null, coder, actualSizes)); + + // Compare the expected and the actual elements. + if (requireExactMatch) { + // Require the blocks to match exactly. (This works only for + // small block sizes. Large block sizes, bigger than Avro's + // internal sizes, lead to different splits.) + Assert.assertEquals(elemsList, actualElemsList); + } else { + // Just require the overall elements to be the same. (This + // works for any block size.) + List expected = new ArrayList<>(); + for (List elems : elemsList) { + expected.addAll(elems); + } + List actual = new ArrayList<>(); + for (List actualElems : actualElemsList) { + actual.addAll(actualElems); + } + Assert.assertEquals(expected, actual); + } + + Assert.assertEquals(expectedSizes, actualSizes); + } + + private List readElems(String filename, + @Nullable Long startOffset, + @Nullable Long endOffset, + Coder coder, + List actualSizes) + throws Exception { + AvroSource avroSource = + new AvroSource<>(filename, startOffset, endOffset, WindowedValue.getValueOnlyCoder(coder)); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(avroSource, actualSizes); + + List actualElems = new ArrayList<>(); + try (Source.SourceIterator> iterator = avroSource.iterator()) { + while (iterator.hasNext()) { + actualElems.add(iterator.next().getValue()); + } + } + return actualElems; + } + + @Test + public void testRead() throws Exception { + runTestRead(Collections.singletonList(TestUtils.INTS), + AvroCoder.of(Integer.class), + true /* require exact match */); + } + + @Test + public void testReadEmpty() throws Exception { + runTestRead(Collections.singletonList(TestUtils.NO_INTS), + AvroCoder.of(Integer.class), + true /* require exact match */); + } + + private List> generateInputBlocks(int numBlocks, + int blockSizeBytes, + int averageLineSizeBytes) { + Random random = new Random(0); + List> blocks = new ArrayList<>(numBlocks); + for (int blockNum = 0; blockNum < numBlocks; blockNum++) { + int numLines = blockSizeBytes / averageLineSizeBytes; + List lines = new ArrayList<>(numLines); + for (int lineNum = 0; lineNum < numLines; lineNum++) { + int numChars = random.nextInt(averageLineSizeBytes * 2); + StringBuilder sb = new StringBuilder(); + for (int charNum = 0; charNum < numChars; charNum++) { + sb.appendCodePoint(random.nextInt('z' - 'a' + 1) + 'a'); + } + lines.add(sb.toString()); + } + blocks.add(lines); + } + return blocks; + } + + @Test + public void testReadSmallRanges() throws Exception { + runTestRead(generateInputBlocks(3, 50, 5), + AvroCoder.of(String.class), + true /* require exact match */); + } + + @Test + public void testReadBigRanges() throws Exception { + runTestRead(generateInputBlocks(10, 128 * 1024, 100), + AvroCoder.of(String.class), + false /* don't require exact match */); + } + + // TODO: sharded filenames + // TODO: reading from GCS +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactoryTest.java new file mode 100644 index 0000000000000..0eb95c70205ca --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactoryTest.java @@ -0,0 +1,78 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for BigQuerySourceFactory. + */ +@RunWith(JUnit4.class) +public class BigQuerySourceFactoryTest { + void runTestCreateBigQuerySource(String project, + String dataset, + String table, + CloudObject encoding) + throws Exception { + CloudObject spec = CloudObject.forClassName("BigQuerySource"); + addString(spec, "project", project); + addString(spec, "dataset", dataset); + addString(spec, "table", table); + + com.google.api.services.dataflow.model.Source cloudSource = + new com.google.api.services.dataflow.model.Source(); + cloudSource.setSpec(spec); + cloudSource.setCodec(encoding); + + Source source = SourceFactory.create(PipelineOptionsFactory.create(), + cloudSource, + new BatchModeExecutionContext()); + Assert.assertThat(source, new IsInstanceOf(BigQuerySource.class)); + BigQuerySource bigQuerySource = (BigQuerySource) source; + Assert.assertEquals(project, bigQuerySource.tableRef.getProjectId()); + Assert.assertEquals(dataset, bigQuerySource.tableRef.getDatasetId()); + Assert.assertEquals(table, bigQuerySource.tableRef.getTableId()); + } + + @Test + public void testCreateBigQuerySource() throws Exception { + runTestCreateBigQuerySource( + "someproject", "somedataset", "sometable", + makeCloudEncoding("TableRowJsonCoder")); + } + + @Test + public void testCreateBigQuerySourceCoderIgnored() throws Exception { + // BigQuery sources do not need a coder because the TableRow objects are read directly from + // the table using the BigQuery API. + runTestCreateBigQuerySource( + "someproject", "somedataset", "sometable", + makeCloudEncoding("BigEndianIntegerCoder")); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceTest.java new file mode 100644 index 0000000000000..2ed4635e8c10b --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceTest.java @@ -0,0 +1,183 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableCell; +import com.google.api.services.bigquery.model.TableDataList; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.io.IOException; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; + +/** + * Tests for BigQuerySource. + * + *

The tests just make sure a basic scenario of reading works because the class itself is a + * thin wrapper over {@code BigQueryTableRowIterator}. The tests for the wrapped class have + * comprehensive coverage. + */ +@RunWith(JUnit4.class) +public class BigQuerySourceTest { + + @Mock private Bigquery mockClient; + @Mock private Bigquery.Tables mockTables; + @Mock private Bigquery.Tables.Get mockTablesGet; + @Mock private Bigquery.Tabledata mockTabledata; + @Mock private Bigquery.Tabledata.List mockTabledataList; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + } + + @After + public void tearDown() { + verifyNoMoreInteractions(mockClient); + verifyNoMoreInteractions(mockTables); + verifyNoMoreInteractions(mockTablesGet); + verifyNoMoreInteractions(mockTabledata); + verifyNoMoreInteractions(mockTabledataList); + } + + private void onTableGet(Table table) throws IOException { + when(mockClient.tables()) + .thenReturn(mockTables); + when(mockTables.get(anyString(), anyString(), anyString())) + .thenReturn(mockTablesGet); + when(mockTablesGet.execute()) + .thenReturn(table); + } + + private void verifyTableGet() throws IOException { + verify(mockClient).tables(); + verify(mockTables).get("project", "dataset", "table"); + verify(mockTablesGet).execute(); + } + + private void onTableList(TableDataList result) throws IOException { + when(mockClient.tabledata()) + .thenReturn(mockTabledata); + when(mockTabledata.list(anyString(), anyString(), anyString())) + .thenReturn(mockTabledataList); + when(mockTabledataList.execute()) + .thenReturn(result); + } + + private void verifyTabledataList() throws IOException { + verify(mockClient, atLeastOnce()).tabledata(); + verify(mockTabledata, atLeastOnce()).list("project", "dataset", "table"); + verify(mockTabledataList, atLeastOnce()).execute(); + // Max results may be set when testing for an empty table. + verify(mockTabledataList, atLeast(0)).setMaxResults(anyLong()); + } + + private Table basicTableSchema() { + return new Table() + .setSchema(new TableSchema() + .setFields(Arrays.asList( + new TableFieldSchema() + .setName("name") + .setType("STRING"), + new TableFieldSchema() + .setName("integer") + .setType("INTEGER"), + new TableFieldSchema() + .setName("float") + .setType("FLOAT"), + new TableFieldSchema() + .setName("bool") + .setType("BOOLEAN") + ))); + } + + private TableRow rawRow(Object...args) { + List cells = new LinkedList<>(); + for (Object a : args) { + cells.add(new TableCell().setV(a)); + } + return new TableRow().setF(cells); + } + + private TableDataList rawDataList(TableRow...rows) { + return new TableDataList() + .setRows(Arrays.asList(rows)); + } + + @Test + public void testRead() throws IOException { + onTableGet(basicTableSchema()); + + // BQ API data is always encoded as a string + TableDataList dataList = rawDataList( + rawRow("Arthur", "42", "3.14159", "false"), + rawRow("Allison", "79", "2.71828", "true") + ); + onTableList(dataList); + + BigQuerySource source = new BigQuerySource( + mockClient, + new TableReference() + .setProjectId("project") + .setDatasetId("dataset") + .setTableId("table")); + + BigQuerySource.SourceIterator iterator = source.iterator(); + Assert.assertTrue(iterator.hasNext()); + TableRow row = iterator.next(); + + Assert.assertEquals("Arthur", row.get("name")); + Assert.assertEquals("42", row.get("integer")); + Assert.assertEquals(3.14159, row.get("float")); + Assert.assertEquals(false, row.get("bool")); + + row = iterator.next(); + + Assert.assertEquals("Allison", row.get("name")); + Assert.assertEquals("79", row.get("integer")); + Assert.assertEquals(2.71828, row.get("float")); + Assert.assertEquals(true, row.get("bool")); + + Assert.assertFalse(iterator.hasNext()); + + verifyTableGet(); + verifyTabledataList(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java new file mode 100644 index 0000000000000..b616f6d75f372 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java @@ -0,0 +1,337 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; +import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray; +import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; +import com.google.cloud.dataflow.sdk.util.common.worker.ParDoFn; +import com.google.cloud.dataflow.sdk.util.common.worker.Receiver; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.reflect.TypeToken; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Tests for CombineValuesFn. + */ +@RunWith(JUnit4.class) +public class CombineValuesFnTest { + /** Example AccumulatingCombineFn. */ + public static class MeanInts extends + Combine.AccumulatingCombineFn { + + class CountSum extends + Combine.AccumulatingCombineFn.Accumulator { + + long count; + double sum; + + @Override + public void addInput(Integer element) { + count++; + sum += element.doubleValue(); + } + + @Override + public void mergeAccumulator(CountSum accumulator) { + count += accumulator.count; + sum += accumulator.sum; + } + + @Override + public String extractOutput() { + return String.format("%.1f", count == 0 ? 0.0 : sum / count); + } + + public CountSum(long count, double sum) { + this.count = count; + this.sum = sum; + } + + @Override + public int hashCode() { + return KV.of(count, sum).hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj == null || !(obj instanceof CountSum)) { + return false; + } + if (obj == this) { + return true; + } + + CountSum other = (CountSum) obj; + return (this.count == other.count) + && (Math.abs(this.sum - other.sum) < 0.1); + } + } + + @Override + public CountSum createAccumulator() { + return new CountSum(0, 0.0); + } + + @Override + public Coder getAccumulatorCoder( + CoderRegistry registry, Coder inputCoder) { + return new CountSumCoder(); + } + } + + /** + * An example "cheap" accumulator coder. + */ + public static class CountSumCoder implements Coder { + public CountSumCoder() { } + + @Override + public void encode( + MeanInts.CountSum value, OutputStream outStream, Context context) + throws CoderException, IOException { + DataOutputStream dataStream = new DataOutputStream(outStream); + dataStream.writeLong(value.count); + dataStream.writeDouble(value.sum); + } + + @Override + public MeanInts.CountSum decode(InputStream inStream, Context context) + throws CoderException, IOException { + DataInputStream dataStream = new DataInputStream(inStream); + long count = dataStream.readLong(); + double sum = dataStream.readDouble(); + return (new MeanInts ()).new CountSum(count, sum); + } + + @Override + public boolean isDeterministic() { return true; } + + public CloudObject asCloudObject() { + return makeCloudEncoding(this.getClass().getName()); + } + + @Override + public List> getCoderArguments() { return null; } + + public List getInstanceComponents(MeanInts.CountSum exampleValue) { + return null; + } + + @Override + public boolean isRegisterByteSizeObserverCheap( + MeanInts.CountSum value, Context context) { + return true; + } + + @Override + public void registerByteSizeObserver( + MeanInts.CountSum value, ElementByteSizeObserver observer, Context ctx) + throws Exception { + observer.update((long) 16); + } + } + + static class TestReceiver implements Receiver { + List receivedElems = new ArrayList<>(); + + @Override + public void process(Object outputElem) { + receivedElems.add(outputElem); + } + } + + private static ParDoFn createCombineValuesFn( + String phase, Combine.KeyedCombineFn combineFn) throws Exception { + // This partially mirrors the work that + // com.google.cloud.dataflow.sdk.transforms.Combine.translateHelper + // does, at least for the KeyedCombineFn. The phase is generated + // by the back-end. + CloudObject spec = CloudObject.forClassName("CombineValuesFn"); + addString(spec, PropertyNames.SERIALIZED_FN, + byteArrayToJsonString(serializeToByteArray(combineFn))); + addString(spec, PropertyNames.PHASE, phase); + + return CombineValuesFn.create( + PipelineOptionsFactory.create(), + spec, + "name", + null, // no side inputs + null, // no side outputs + 1, // single main output + new BatchModeExecutionContext(), + (new CounterSet()).getAddCounterMutator(), + null); + } + + @Test + public void testCombineValuesFnAll() throws Exception { + TestReceiver receiver = new TestReceiver(); + + Combine.KeyedCombineFn combiner = + (new MeanInts()).asKeyedFn(); + + ParDoFn combineParDoFn = createCombineValuesFn( + CombineValuesFn.CombinePhase.ALL, combiner); + + combineParDoFn.startBundle(receiver); + combineParDoFn.processElement(WindowedValue.valueInGlobalWindow( + KV.of("a", Arrays.asList(5, 6, 7)))); + combineParDoFn.processElement(WindowedValue.valueInGlobalWindow( + KV.of("b", Arrays.asList(1, 3, 7)))); + combineParDoFn.processElement(WindowedValue.valueInGlobalWindow( + KV.of("c", Arrays.asList(3, 6, 8, 9)))); + combineParDoFn.finishBundle(); + + Object[] expectedReceivedElems = { + WindowedValue.valueInGlobalWindow(KV.of("a", "6.0")), + WindowedValue.valueInGlobalWindow(KV.of("b", "3.7")), + WindowedValue.valueInGlobalWindow(KV.of("c", "6.5")), + }; + assertArrayEquals(expectedReceivedElems, receiver.receivedElems.toArray()); + } + + @Test + public void testCombineValuesFnAdd() throws Exception { + TestReceiver receiver = new TestReceiver(); + MeanInts mean = new MeanInts(); + + Combine.KeyedCombineFn combiner = mean.asKeyedFn(); + + ParDoFn combineParDoFn = createCombineValuesFn( + CombineValuesFn.CombinePhase.ADD, combiner); + + combineParDoFn.startBundle(receiver); + combineParDoFn.processElement(WindowedValue.valueInGlobalWindow( + KV.of("a", Arrays.asList(5, 6, 7)))); + combineParDoFn.processElement(WindowedValue.valueInGlobalWindow( + KV.of("b", Arrays.asList(1, 3, 7)))); + combineParDoFn.processElement(WindowedValue.valueInGlobalWindow( + KV.of("c", Arrays.asList(3, 6, 8, 9)))); + combineParDoFn.finishBundle(); + + Object[] expectedReceivedElems = { + WindowedValue.valueInGlobalWindow(KV.of("a", mean.new CountSum(3, 18))), + WindowedValue.valueInGlobalWindow(KV.of("b", mean.new CountSum(3, 11))), + WindowedValue.valueInGlobalWindow(KV.of("c", mean.new CountSum(4, 26))) + }; + assertArrayEquals(expectedReceivedElems, receiver.receivedElems.toArray()); + } + + @Test + public void testCombineValuesFnMerge() throws Exception { + TestReceiver receiver = new TestReceiver(); + MeanInts mean = new MeanInts(); + + Combine.KeyedCombineFn combiner = mean.asKeyedFn(); + + ParDoFn combineParDoFn = createCombineValuesFn( + CombineValuesFn.CombinePhase.MERGE, combiner); + + combineParDoFn.startBundle(receiver); + combineParDoFn.processElement(WindowedValue.valueInGlobalWindow( + KV.of("a", + Arrays.asList( + mean.new CountSum(3, 6), + mean.new CountSum(2, 9), + mean.new CountSum(1, 12))))); + combineParDoFn.processElement(WindowedValue.valueInGlobalWindow( + KV.of("b", + Arrays.asList( + mean.new CountSum(2, 20), + mean.new CountSum(1, 1))))); + combineParDoFn.finishBundle(); + + Object[] expectedReceivedElems = { + WindowedValue.valueInGlobalWindow(KV.of("a", mean.new CountSum(6, 27))), + WindowedValue.valueInGlobalWindow(KV.of("b", mean.new CountSum(3, 21))), + }; + assertArrayEquals(expectedReceivedElems, receiver.receivedElems.toArray()); + } + + @Test + public void testCombineValuesFnExtract() throws Exception { + TestReceiver receiver = new TestReceiver(); + MeanInts mean = new MeanInts(); + + Combine.KeyedCombineFn combiner = mean.asKeyedFn(); + + ParDoFn combineParDoFn = createCombineValuesFn( + CombineValuesFn.CombinePhase.EXTRACT, combiner); + + combineParDoFn.startBundle(receiver); + combineParDoFn.processElement(WindowedValue.valueInGlobalWindow( + KV.of("a", mean.new CountSum(6, 27)))); + combineParDoFn.processElement(WindowedValue.valueInGlobalWindow( + KV.of("b", mean.new CountSum(3, 21)))); + combineParDoFn.finishBundle(); + + assertArrayEquals( + new Object[]{ WindowedValue.valueInGlobalWindow(KV.of("a", "4.5")), + WindowedValue.valueInGlobalWindow(KV.of("b", "7.0")) }, + receiver.receivedElems.toArray()); + } + + @Test + public void testCombineValuesFnCoders() throws Exception { + CoderRegistry registry = new CoderRegistry(); + registry.registerStandardCoders(); + + MeanInts meanInts = new MeanInts(); + MeanInts.CountSum countSum = meanInts.new CountSum(6, 27); + + Coder coder = meanInts.getAccumulatorCoder( + registry, registry.getDefaultCoder(TypeToken.of(Integer.class))); + + assertEquals( + countSum, + CoderUtils.decodeFromByteArray(coder, + CoderUtils.encodeToByteArray(coder, countSum))); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CopyableSeekableByteChannelTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CopyableSeekableByteChannelTest.java new file mode 100644 index 0000000000000..e27fa1832870a --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CopyableSeekableByteChannelTest.java @@ -0,0 +1,152 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Preconditions.checkArgument; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; +import java.nio.channels.SeekableByteChannel; + +/** Unit tests for {@link CopyableSeekableByteChannel}. */ +@RunWith(JUnit4.class) +public final class CopyableSeekableByteChannelTest { + @Test + public void copiedChannelShouldMaintainIndependentPosition() + throws IOException { + ByteBuffer dst = ByteBuffer.allocate(6); + SeekableByteChannel base = + new FakeSeekableByteChannel("Hello, world! :-)".getBytes()); + base.position(1); + + CopyableSeekableByteChannel chan = new CopyableSeekableByteChannel(base); + assertThat(chan.position(), equalTo((long) 1)); + + CopyableSeekableByteChannel copy = chan.copy(); + assertThat(copy.position(), equalTo((long) 1)); + + assertThat(chan.read(dst), equalTo(6)); + assertThat(chan.position(), equalTo((long) 7)); + assertThat(new String(dst.array()), equalTo("ello, ")); + dst.rewind(); + + assertThat(copy.position(), equalTo((long) 1)); + copy.position(3); + assertThat(copy.read(dst), equalTo(6)); + assertThat(copy.position(), equalTo((long) 9)); + assertThat(new String(dst.array()), equalTo("lo, wo")); + dst.rewind(); + + assertThat(chan.read(dst), equalTo(6)); + assertThat(chan.position(), equalTo((long) 13)); + assertThat(new String(dst.array()), equalTo("world!")); + dst.rewind(); + + assertThat(chan.read(dst), equalTo(4)); + assertThat(chan.position(), equalTo((long) 17)); + assertThat(new String(dst.array()), equalTo(" :-)d!")); + dst.rewind(); + + assertThat(copy.position(), equalTo((long) 9)); + assertThat(copy.read(dst), equalTo(6)); + assertThat(new String(dst.array()), equalTo("rld! :")); + } + + private static final class FakeSeekableByteChannel + implements SeekableByteChannel { + private boolean closed = false; + private ByteBuffer data; + + public FakeSeekableByteChannel(byte[] data) { + this.data = ByteBuffer.wrap(data); + } + + @Override + public long position() throws IOException { + checkClosed(); + return data.position(); + } + + @Override + public SeekableByteChannel position(long newPosition) throws IOException { + checkArgument(newPosition >= 0); + checkClosed(); + data.position((int) newPosition); + return this; + } + + @Override + public int read(ByteBuffer dst) throws IOException { + checkClosed(); + if (!data.hasRemaining()) { + return -1; + } + int count = Math.min(data.remaining(), dst.remaining()); + ByteBuffer src = data.slice(); + src.limit(count); + dst.put(src); + data.position(data.position() + count); + return count; + } + + @Override + public long size() throws IOException { + checkClosed(); + return data.limit(); + } + + @Override + public SeekableByteChannel truncate(long size) throws IOException { + checkClosed(); + data.limit((int) size); + return this; + } + + @Override + public int write(ByteBuffer src) throws IOException { + checkClosed(); + int count = Math.min(data.remaining(), src.remaining()); + ByteBuffer copySrc = src.slice(); + copySrc.limit(count); + data.put(copySrc); + return count; + } + + @Override + public boolean isOpen() { + return !closed; + } + + @Override + public void close() { + closed = true; + } + + private void checkClosed() throws ClosedChannelException { + if (closed) { + throw new ClosedChannelException(); + } + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java new file mode 100644 index 0000000000000..2167a504183fc --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java @@ -0,0 +1,438 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToSourcePosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.util.CloudCounterUtils.extractCounter; +import static com.google.cloud.dataflow.sdk.util.CloudMetricUtils.extractCloudMetric; +import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; +import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudTime; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MAX; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SET; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.timeout; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.api.services.dataflow.model.Position; +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.cloud.dataflow.sdk.options.DataflowWorkerHarnessOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.Transport; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.CounterTestUtils; +import com.google.cloud.dataflow.sdk.util.common.Metric; +import com.google.cloud.dataflow.sdk.util.common.Metric.DoubleMetric; +import com.google.cloud.dataflow.sdk.util.common.worker.MapTaskExecutor; +import com.google.cloud.dataflow.sdk.util.common.worker.Operation; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; + +import org.hamcrest.Description; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentMatcher; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import javax.annotation.Nullable; + +/** Unit tests for {@link DataflowWorkProgressUpdater}. */ +@RunWith(JUnit4.class) +public class DataflowWorkProgressUpdaterTest { + static class TestMapTaskExecutor extends MapTaskExecutor { + ApproximateProgress progress = null; + + public TestMapTaskExecutor(CounterSet counters) { + super(new ArrayList(), + counters, + new StateSampler("test", counters.getAddCounterMutator())); + } + + @Override + public Source.Progress getWorkerProgress() { + return cloudProgressToSourceProgress(progress); + } + + @Override + public Source.Position proposeStopPosition( + Source.Progress suggestedStopPoint) { + @Nullable ApproximateProgress progress = sourceProgressToCloudProgress(suggestedStopPoint); + if (progress == null) { + return null; + } + return cloudPositionToSourcePosition(progress.getPosition()); + } + + public void setWorkerProgress(ApproximateProgress progress) { + this.progress = progress; + } + } + + static { + // To shorten wait times during testing. + System.setProperty("minimum_worker_update_interval_millis", "100"); + System.setProperty("worker_lease_renewal_latency_margin", "100"); + } + + private static final String PROJECT_ID = "TEST_PROJECT_ID"; + private static final String JOB_ID = "TEST_JOB_ID"; + private static final String WORKER_ID = "TEST_WORKER_ID"; + private static final Long WORK_ID = 1234567890L; + private static final String COUNTER_NAME = "test-counter-"; + private static final AggregationKind[] COUNTER_KINDS = {SUM, MAX, SET}; + private static final Long COUNTER_VALUE1 = 12345L; + private static final Double COUNTER_VALUE2 = Math.PI; + private static final String COUNTER_VALUE3 = "value"; + + @Rule public final ExpectedException thrown = ExpectedException.none(); + @Mock private DataflowWorker.WorkUnitClient workUnitClient; + private CounterSet counters; + private List> metrics; + private TestMapTaskExecutor worker; + private WorkItem workItem; + private DataflowWorkerHarnessOptions options; + private DataflowWorkProgressUpdater progressUpdater; + private long nowMillis; + + @Before + public void initMocksAndWorkflowServiceAndWorkerAndWork() throws IOException { + MockitoAnnotations.initMocks(this); + + options = PipelineOptionsFactory.createFromSystemProperties(); + options.setProject(PROJECT_ID); + options.setJobId(JOB_ID); + options.setWorkerId(WORKER_ID); + + metrics = new ArrayList<>(); + counters = new CounterSet(); + worker = new TestMapTaskExecutor(counters) { + @Override + public Collection> getOutputMetrics() { + return metrics; + } + }; + nowMillis = System.currentTimeMillis(); + + workItem = new WorkItem(); + workItem.setProjectId(PROJECT_ID); + workItem.setJobId(JOB_ID); + workItem.setId(WORK_ID); + workItem.setLeaseExpireTime(toCloudTime(new Instant(nowMillis + 1000))); + workItem.setReportStatusInterval(toCloudDuration(Duration.millis(500))); + + progressUpdater = new DataflowWorkProgressUpdater( + workItem, worker, workUnitClient, options); + } + + // TODO: Remove sleeps from this test by using a mock sleeper. This + // requires a redesign of the WorkProgressUpdater to use a Sleeper and + // not use a ScheduledThreadExecutor which relies on real time passing. + @Test(timeout = 2000) + public void workProgressUpdaterUpdates() throws Exception { + when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))).thenReturn( + generateServiceState(nowMillis + 2000, 1000, null)); + setUpCounters(2); + setUpMetrics(3); + setUpProgress(makeRecordIndexProgress(1L)); + progressUpdater.startReportingProgress(); + // The initial update should be sent after leaseRemainingTime / 2. + verify(workUnitClient, timeout(600)).reportWorkItemStatus(argThat( + new ExpectedDataflowProgress() + .withCounters(2) + .withMetrics(3) + .withProgress(makeRecordIndexProgress(1L)))); + progressUpdater.stopReportingProgress(); + } + + // Verifies that ReportWorkItemStatusRequest contains correct progress report + // and actual stop position report. + @Test(timeout = 5000) + public void workProgressUpdaterAdaptsProgressInterval() throws Exception { + // Mock that the next reportProgress call will return a response that asks + // us to truncate the task at index 3, and the next two will not ask us to + // truncate at all. + when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) + .thenReturn(generateServiceState(nowMillis + 2000, 1000, + makeRecordIndexPosition(3L))) + .thenReturn(generateServiceState(nowMillis + 3000, 2000, null)) + .thenReturn(generateServiceState(nowMillis + 4000, 3000, null)); + + setUpCounters(3); + setUpMetrics(2); + setUpProgress(makeRecordIndexProgress(1L)); + progressUpdater.startReportingProgress(); + // The initial update should be sent after + // leaseRemainingTime (1000) / 2 = 500. + verify(workUnitClient, timeout(600)).reportWorkItemStatus(argThat( + new ExpectedDataflowProgress() + .withCounters(3) + .withMetrics(2) + .withProgress(makeRecordIndexProgress(1L)))); + + setUpCounters(5); + setUpMetrics(6); + setUpProgress(makeRecordIndexProgress(2L)); + // The second update should be sent after one second (2000 / 2). + verify(workUnitClient, timeout(1100)).reportWorkItemStatus(argThat( + new ExpectedDataflowProgress() + .withCounters(5) + .withMetrics(6) + .withProgress(makeRecordIndexProgress(2L)) + .withStopPosition(makeRecordIndexPosition(3L)))); + + // After the request is sent, reset stop position cache to null. + assertNull(progressUpdater.getStopPosition()); + + setUpProgress(makeRecordIndexProgress(3L)); + + // The third update should be sent after one and half seconds (3000 / 2). + verify(workUnitClient, timeout(1600)).reportWorkItemStatus(argThat( + new ExpectedDataflowProgress() + .withProgress(makeRecordIndexProgress(3L)))); + + progressUpdater.stopReportingProgress(); + } + + // Verifies that a last update is sent when there is an unacknowledged split request. + @Test(timeout = 3000) + public void workProgressUpdaterLastUpdate() throws Exception { + when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) + .thenReturn(generateServiceState(nowMillis + 2000, 1000, + makeRecordIndexPosition(2L))) + .thenReturn(generateServiceState(nowMillis + 3000, 2000, null)); + + setUpProgress(makeRecordIndexProgress(1L)); + progressUpdater.startReportingProgress(); + // The initial update should be sent after leaseRemainingTime / 2 = 500 msec. + Thread.sleep(600); + verify(workUnitClient, timeout(200)).reportWorkItemStatus(argThat( + new ExpectedDataflowProgress() + .withProgress(makeRecordIndexProgress(1L)))); + + // The first update should include the new actual stop position. + // Verify that the progressUpdater has recorded it. + assertEquals(makeRecordIndexPosition(2L), + sourcePositionToCloudPosition(progressUpdater.getStopPosition())); + + setUpProgress(makeRecordIndexProgress(2L)); + // The second update should be sent after one second (2000 / 2). + Thread.sleep(200); // not enough time for an update so the latest stop position is not + // acknowledged. + // Check that the progressUpdater still has a pending stop position to send + assertEquals(makeRecordIndexPosition(2L), + sourcePositionToCloudPosition(progressUpdater.getStopPosition())); + + progressUpdater.stopReportingProgress(); // should send the last update + // check that the progressUpdater is done with reporting its latest stop position + assertNull(progressUpdater.getStopPosition()); + + // Verify that the last update contained the latest stop position + verify(workUnitClient, timeout(1000)).reportWorkItemStatus(argThat( + new ExpectedDataflowProgress() + .withStopPosition(makeRecordIndexPosition(2L)))); + } + + private void setUpCounters(int n) { + counters.clear(); + for (int i = 0; i < n; i++) { + counters.add(makeCounter(i)); + } + } + + private static Counter makeCounter(int i) { + if (i % 3 == 0) { + return Counter.longs(COUNTER_NAME + i, COUNTER_KINDS[0]) + .addValue(COUNTER_VALUE1 + i).addValue(COUNTER_VALUE1 + i * 2); + } else if (i % 3 == 1) { + return Counter.doubles(COUNTER_NAME + i, COUNTER_KINDS[1]) + .addValue(COUNTER_VALUE2 + i).addValue(COUNTER_VALUE2 + i * 3); + } else { + return Counter.strings(COUNTER_NAME + i, COUNTER_KINDS[2]) + .addValue(COUNTER_VALUE3 + i).addValue(COUNTER_NAME + i * 5); + } + } + + private static Metric makeMetric(int i) { + return new DoubleMetric(String.valueOf(i), (double) i); + } + + private void setUpMetrics(int n) { + metrics = new ArrayList<>(); + for (int i = 0; i < n; i++) { + metrics.add(makeMetric(i)); + } + } + + private void setUpProgress(ApproximateProgress progress) { + worker.setWorkerProgress(progress); + } + + private com.google.api.services.dataflow.model.Position makeRecordIndexPosition(Long index) { + com.google.api.services.dataflow.model.Position position = + new com.google.api.services.dataflow.model.Position(); + position.setRecordIndex(index); + return position; + } + + private ApproximateProgress makeRecordIndexProgress(Long index) { + return new ApproximateProgress().setPosition(makeRecordIndexPosition(index)); + } + + private WorkItemServiceState generateServiceState( + long leaseExpirationTimestamp, int progressReportIntervalMs, + Position suggestedStopPosition) + throws IOException { + WorkItemServiceState responseState = new WorkItemServiceState(); + responseState.setFactory(Transport.getJsonFactory()); + responseState.setLeaseExpireTime(toCloudTime(new Instant(leaseExpirationTimestamp))); + responseState.setReportStatusInterval( + toCloudDuration(Duration.millis(progressReportIntervalMs))); + + if (suggestedStopPosition != null) { + responseState.setSuggestedStopPosition(suggestedStopPosition); + } + + return responseState; + } + + private static final class ExpectedDataflowProgress extends ArgumentMatcher { + @Nullable Integer counterCount; + @Nullable Integer metricCount; + @Nullable ApproximateProgress expectedProgress; + @Nullable Position expectedStopPosition; + + public ExpectedDataflowProgress withCounters(Integer counterCount) { + this.counterCount = counterCount; + return this; + } + + public ExpectedDataflowProgress withMetrics(Integer metricCount) { + this.metricCount = metricCount; + return this; + } + + public ExpectedDataflowProgress withProgress(ApproximateProgress expectedProgress) { + this.expectedProgress = expectedProgress; + return this; + } + + public ExpectedDataflowProgress withStopPosition(Position expectedStopPosition) { + this.expectedStopPosition = expectedStopPosition; + return this; + } + + @Override + public void describeTo(Description description) { + List values = new ArrayList<>(); + if (this.counterCount != null) { + for (int i = 0; i < counterCount; i++) { + values.add(extractCounter(makeCounter(i), false).toString()); + } + } + if (this.metricCount != null) { + for (int i = 0; i < metricCount; i++) { + values.add(extractCloudMetric(makeMetric(i), WORKER_ID).toString()); + } + } + if (this.expectedProgress != null) { + values.add("progress " + this.expectedProgress); + } + if (this.expectedStopPosition != null) { + values.add("stop position " + this.expectedStopPosition); + } else { + values.add("no stop position present"); + } + description.appendValueList("Dataflow progress with ", ", ", ".", values); + } + + @Override + public boolean matches(Object status) { + WorkItemStatus st = (WorkItemStatus) status; + return matchCountersAndMetrics(st) + && matchProgress(st) + && matchStopPosition(st); + } + + private boolean matchCountersAndMetrics(WorkItemStatus status) { + if (counterCount == null && metricCount == null) { + return true; + } + + List sentUpdates = status.getMetricUpdates(); + + if (counterCount + metricCount != sentUpdates.size()) { + return false; + } + + for (int i = 0; i < counterCount; i++) { + if (!sentUpdates.contains( + CounterTestUtils.extractCounterUpdate(makeCounter(i), false))) { + return false; + } + } + + for (int i = 0; i < metricCount; i++) { + if (!sentUpdates.contains(extractCloudMetric(makeMetric(i), WORKER_ID))) { + return false; + } + } + + return true; + } + + private boolean matchProgress(WorkItemStatus status) { + if (expectedProgress == null) { + return true; + } + ApproximateProgress progress = status.getProgress(); + return expectedProgress.equals(progress); + } + + private boolean matchStopPosition(WorkItemStatus status) { + Position actualStopPosition = status.getStopPosition(); + if (expectedStopPosition == null) { + return actualStopPosition == null; + } + return expectedStopPosition.equals(actualStopPosition); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java new file mode 100644 index 0000000000000..d1d369fe99acd --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java @@ -0,0 +1,243 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.doCallRealMethod; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +import com.google.api.client.http.LowLevelHttpResponse; +import com.google.api.client.json.Json; +import com.google.api.client.testing.http.MockHttpTransport; +import com.google.api.client.testing.http.MockLowLevelHttpRequest; +import com.google.api.client.testing.http.MockLowLevelHttpResponse; +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.WorkItem; +import com.google.cloud.dataflow.sdk.options.DataflowWorkerHarnessOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.testing.RestoreMappedDiagnosticContext; +import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties; +import com.google.cloud.dataflow.sdk.util.TestCredential; +import com.google.cloud.dataflow.sdk.util.Transport; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TestRule; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.slf4j.MDC; + +import java.io.IOException; + +/** Unit tests for {@link DataflowWorkerHarness}. */ +@RunWith(JUnit4.class) +public class DataflowWorkerHarnessTest { + @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); + @Rule public TestRule restoreMDC = new RestoreMappedDiagnosticContext(); + @Rule public ExpectedException expectedException = ExpectedException.none(); + @Mock private MockHttpTransport transport; + @Mock private MockLowLevelHttpRequest request; + @Mock private DataflowWorker mockDataflowWorker; + + private Dataflow service; + + @Before + public void setUp() throws Exception { + MockitoAnnotations.initMocks(this); + when(transport.buildRequest(anyString(), anyString())).thenReturn(request); + doCallRealMethod().when(request).getContentAsString(); + + service = new Dataflow(transport, Transport.getJsonFactory(), null); + } + + @Test + public void testThatWeOnlyProcessWorkOnce() throws Exception { + when(mockDataflowWorker.getAndPerformWork()).thenReturn(true); + DataflowWorkerHarness.processWork(mockDataflowWorker); + verify(mockDataflowWorker).getAndPerformWork(); + verifyNoMoreInteractions(mockDataflowWorker); + } + + @Test + public void testThatWeOnlyProcessWorkOnceEvenWhenFailing() throws Exception { + when(mockDataflowWorker.getAndPerformWork()).thenReturn(false); + DataflowWorkerHarness.processWork(mockDataflowWorker); + verify(mockDataflowWorker).getAndPerformWork(); + verifyNoMoreInteractions(mockDataflowWorker); + } + + @Test + public void testCreationOfWorkerHarness() throws Exception { + System.getProperties().putAll(ImmutableMap + .builder() + .put("project_id", "projectId") + .put("job_id", "jobId") + .put("worker_id", "workerId") + .build()); + DataflowWorkerHarnessOptions options = PipelineOptionsFactory.createFromSystemProperties(); + options.setGcpCredential(new TestCredential()); + assertNotNull(DataflowWorkerHarness.create(options)); + assertEquals("jobId", MDC.get("dataflow.jobId")); + assertEquals("workerId", MDC.get("dataflow.workerId")); + } + + @Test + public void testCloudServiceCall() throws Exception { + System.getProperties().putAll(ImmutableMap + .builder() + .put("project_id", "projectId") + .put("job_id", "jobId") + .put("worker_id", "workerId") + .build()); + WorkItem workItem = createWorkItem("projectId", "jobId"); + + when(request.execute()).thenReturn(generateMockResponse(workItem)); + + DataflowWorkerHarnessOptions options = PipelineOptionsFactory.createFromSystemProperties(); + + DataflowWorker.WorkUnitClient client = + new DataflowWorkerHarness.DataflowWorkUnitClient(service, options); + + assertEquals(workItem, client.getWorkItem()); + + LeaseWorkItemRequest actualRequest = Transport.getJsonFactory().fromString( + request.getContentAsString(), LeaseWorkItemRequest.class); + assertEquals("workerId", actualRequest.getWorkerId()); + assertEquals(ImmutableList.of("workerId", "remote_source", "custom_source"), + actualRequest.getWorkerCapabilities()); + assertEquals(ImmutableList.of("map_task", "seq_map_task", "remote_source_task"), + actualRequest.getWorkItemTypes()); + assertEquals("1234", MDC.get("dataflow.workId")); + } + + @Test + public void testCloudServiceCallNoWorkId() throws Exception { + System.getProperties().putAll(ImmutableMap + .builder() + .put("project_id", "projectId") + .put("job_id", "jobId") + .put("worker_id", "workerId") + .build()); + + // If there's no work the service should return an empty work item. + WorkItem workItem = new WorkItem(); + + when(request.execute()).thenReturn(generateMockResponse(workItem)); + + DataflowWorkerHarnessOptions options = PipelineOptionsFactory.createFromSystemProperties(); + + DataflowWorker.WorkUnitClient client = + new DataflowWorkerHarness.DataflowWorkUnitClient(service, options); + + assertNull(client.getWorkItem()); + + LeaseWorkItemRequest actualRequest = Transport.getJsonFactory().fromString( + request.getContentAsString(), LeaseWorkItemRequest.class); + assertEquals("workerId", actualRequest.getWorkerId()); + assertEquals(ImmutableList.of("workerId", "remote_source", "custom_source"), + actualRequest.getWorkerCapabilities()); + assertEquals(ImmutableList.of("map_task", "seq_map_task", "remote_source_task"), + actualRequest.getWorkItemTypes()); + } + + @Test + public void testCloudServiceCallNoWorkItem() throws Exception { + System.getProperties().putAll(ImmutableMap + .builder() + .put("project_id", "projectId") + .put("job_id", "jobId") + .put("worker_id", "workerId") + .build()); + + when(request.execute()).thenReturn(generateMockResponse()); + + DataflowWorkerHarnessOptions options = PipelineOptionsFactory.createFromSystemProperties(); + + DataflowWorker.WorkUnitClient client = + new DataflowWorkerHarness.DataflowWorkUnitClient(service, options); + + assertNull(client.getWorkItem()); + + LeaseWorkItemRequest actualRequest = Transport.getJsonFactory().fromString( + request.getContentAsString(), LeaseWorkItemRequest.class); + assertEquals("workerId", actualRequest.getWorkerId()); + assertEquals(ImmutableList.of("workerId", "remote_source", "custom_source"), + actualRequest.getWorkerCapabilities()); + assertEquals(ImmutableList.of("map_task", "seq_map_task", "remote_source_task"), + actualRequest.getWorkItemTypes()); + } + + @Test + public void testCloudServiceCallMultipleWorkItems() throws Exception { + expectedException.expect(IOException.class); + expectedException.expectMessage( + "This version of the SDK expects no more than one work item from the service"); + System.getProperties().putAll(ImmutableMap + .builder() + .put("project_id", "projectId") + .put("job_id", "jobId") + .put("worker_id", "workerId") + .build()); + + WorkItem workItem1 = createWorkItem("projectId", "jobId"); + WorkItem workItem2 = createWorkItem("projectId", "jobId"); + + when(request.execute()).thenReturn(generateMockResponse(workItem1, workItem2)); + + DataflowWorkerHarnessOptions options = PipelineOptionsFactory.createFromSystemProperties(); + + DataflowWorker.WorkUnitClient client = + new DataflowWorkerHarness.DataflowWorkUnitClient(service, options); + + client.getWorkItem(); + } + + private LowLevelHttpResponse generateMockResponse(WorkItem ... workItems) throws Exception { + MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); + response.setContentType(Json.MEDIA_TYPE); + LeaseWorkItemResponse lease = new LeaseWorkItemResponse(); + lease.setWorkItems(Lists.newArrayList(workItems)); + // N.B. Setting the factory is necessary in order to get valid JSON. + lease.setFactory(Transport.getJsonFactory()); + response.setContent(lease.toPrettyString()); + return response; + } + + private WorkItem createWorkItem(String projectId, String jobId) { + WorkItem workItem = new WorkItem(); + workItem.setFactory(Transport.getJsonFactory()); + workItem.setProjectId(projectId); + workItem.setJobId(jobId); + + // We need to set a work id because otherwise the client will treat the response as + // indicating no work is available. + workItem.setId(1234L); + return workItem; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java new file mode 100644 index 0000000000000..2d51fb2838954 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java @@ -0,0 +1,85 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static org.junit.Assert.assertFalse; +import static org.mockito.Matchers.argThat; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.api.services.dataflow.model.WorkItem; +import com.google.api.services.dataflow.model.WorkItemStatus; +import com.google.cloud.dataflow.sdk.options.DataflowWorkerHarnessOptions; +import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper; + +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +/** Unit tests for {@link DataflowWorker}. */ +@RunWith(JUnit4.class) +public class DataflowWorkerTest { + @Rule + public FastNanoClockAndSleeper clockAndSleeper = new FastNanoClockAndSleeper(); + + @Mock + DataflowWorker.WorkUnitClient mockWorkUnitClient; + + @Mock + DataflowWorkerHarnessOptions options; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + } + + @Test + public void testWhenNoWorkThatWeReturnFalse() throws Exception { + DataflowWorker worker = new DataflowWorker(mockWorkUnitClient, options); + when(mockWorkUnitClient.getWorkItem()).thenReturn(null); + + assertFalse(worker.getAndPerformWork()); + } + + @Test + public void testWhenProcessingWorkUnitFailsWeReportStatus() throws Exception { + DataflowWorker worker = new DataflowWorker(mockWorkUnitClient, options); + when(mockWorkUnitClient.getWorkItem()).thenReturn(new WorkItem().setId(1L)).thenReturn(null); + + assertFalse(worker.getAndPerformWork()); + verify(mockWorkUnitClient).reportWorkItemStatus(argThat(cloudWorkHasErrors())); + } + + private Matcher cloudWorkHasErrors() { + return new TypeSafeMatcher() { + @Override + public void describeTo(Description description) { + description.appendText("WorkItemStatus expected to have errors"); + } + + @Override + protected boolean matchesSafely(WorkItemStatus status) { + return status.getCompleted() && !status.getErrors().isEmpty(); + } + }; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java new file mode 100644 index 0000000000000..b41bd1b2e2914 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java @@ -0,0 +1,499 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Base64.encodeBase64URLSafeString; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudDuration; + +import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.Position; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.InstantCoder; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.Reiterable; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Source.SourceIterator; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.collect.Lists; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * Tests for GroupingShuffleSource. + */ +@RunWith(JUnit4.class) +public class GroupingShuffleSourceTest { + static final List>> NO_KVS = Collections.emptyList(); + + static final Instant timestamp = new Instant(123000); + static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); + + static final List>> KVS = Arrays.asList( + KV.of(1, Arrays.asList("in 1a", "in 1b")), + KV.of(2, Arrays.asList("in 2a", "in 2b")), + KV.of(3, Arrays.asList("in 3")), + KV.of(4, Arrays.asList("in 4a", "in 4b", "in 4c", "in 4d")), + KV.of(5, Arrays.asList("in 5"))); + + /** How many of the values with each key are to be read. */ + enum ValuesToRead { + /** Don't even ask for the values iterator. */ + SKIP_VALUES, + /** Get the iterator, but don't read any values. */ + READ_NO_VALUES, + /** Read just the first value. */ + READ_ONE_VALUE, + /** Read all the values. */ + READ_ALL_VALUES + } + + void runTestReadShuffleSource(List>> input, + ValuesToRead valuesToRead) + throws Exception { + Coder> elemCoder = + WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); + BatchModeExecutionContext context = new BatchModeExecutionContext(); + GroupingShuffleSource> shuffleSource = + new GroupingShuffleSource<>( + PipelineOptionsFactory.create(), + null, null, null, + WindowedValue.getFullCoder( + KvCoder.of( + BigEndianIntegerCoder.of(), + IterableCoder.of( + WindowedValue.getFullCoder(StringUtf8Coder.of(), + IntervalWindow.getCoder()))), + IntervalWindow.getCoder()), + context); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(shuffleSource); + + TestShuffleReader shuffleReader = new TestShuffleReader(); + List expectedSizes = new ArrayList<>(); + for (KV> kvs : input) { + Integer key = kvs.getKey(); + byte[] keyByte = CoderUtils.encodeToByteArray(BigEndianIntegerCoder.of(), key); + + for (String value : kvs.getValue()) { + byte[] valueByte = CoderUtils.encodeToByteArray( + elemCoder, WindowedValue.of(value, timestamp, Lists.newArrayList(window))); + byte[] skey = CoderUtils.encodeToByteArray(InstantCoder.of(), timestamp); + ShuffleEntry shuffleEntry = new ShuffleEntry(keyByte, skey, valueByte); + shuffleReader.addEntry(shuffleEntry); + expectedSizes.add(shuffleEntry.length()); + } + } + + List>>> actual = new ArrayList<>(); + try (SourceIterator>>>> iter = + shuffleSource.iterator(shuffleReader)) { + Iterable> prevValuesIterable = null; + Iterator> prevValuesIterator = null; + while (iter.hasNext()) { + Assert.assertTrue(iter.hasNext()); + Assert.assertTrue(iter.hasNext()); + + KV>> elem = iter.next().getValue(); + Integer key = elem.getKey(); + List> values = new ArrayList<>(); + if (valuesToRead.ordinal() > ValuesToRead.SKIP_VALUES.ordinal()) { + if (prevValuesIterable != null) { + prevValuesIterable.iterator(); // Verifies that this does not throw. + } + if (prevValuesIterator != null) { + prevValuesIterator.hasNext(); // Verifies that this does not throw. + } + + Iterable> valuesIterable = elem.getValue(); + Iterator> valuesIterator = valuesIterable.iterator(); + + if (valuesToRead.ordinal() >= ValuesToRead.READ_ONE_VALUE.ordinal()) { + while (valuesIterator.hasNext()) { + Assert.assertTrue(valuesIterator.hasNext()); + Assert.assertTrue(valuesIterator.hasNext()); + Assert.assertEquals("BatchModeExecutionContext key", + key, context.getKey()); + values.add(valuesIterator.next()); + if (valuesToRead == ValuesToRead.READ_ONE_VALUE) { + break; + } + } + if (valuesToRead == ValuesToRead.READ_ALL_VALUES) { + Assert.assertFalse(valuesIterator.hasNext()); + Assert.assertFalse(valuesIterator.hasNext()); + + try { + valuesIterator.next(); + Assert.fail("Expected NoSuchElementException"); + } catch (NoSuchElementException exn) { + // As expected. + } + valuesIterable.iterator(); // Verifies that this does not throw. + } + } + + prevValuesIterable = valuesIterable; + prevValuesIterator = valuesIterator; + } + + actual.add(KV.of(key, values)); + } + Assert.assertFalse(iter.hasNext()); + Assert.assertFalse(iter.hasNext()); + try { + iter.next(); + Assert.fail("Expected NoSuchElementException"); + } catch (NoSuchElementException exn) { + // As expected. + } + } + + List>>> expected = new ArrayList<>(); + for (KV> kvs : input) { + Integer key = kvs.getKey(); + List> values = new ArrayList<>(); + if (valuesToRead.ordinal() >= ValuesToRead.READ_ONE_VALUE.ordinal()) { + for (String value : kvs.getValue()) { + values.add(WindowedValue.of(value, timestamp, Lists.newArrayList(window))); + if (valuesToRead == ValuesToRead.READ_ONE_VALUE) { + break; + } + } + } + expected.add(KV.of(key, values)); + } + Assert.assertEquals(expected, actual); + Assert.assertEquals(expectedSizes, observer.getActualSizes()); + } + + @Test + public void testReadEmptyShuffleSource() throws Exception { + runTestReadShuffleSource(NO_KVS, ValuesToRead.READ_ALL_VALUES); + } + + @Test + public void testReadEmptyShuffleSourceSkippingValues() throws Exception { + runTestReadShuffleSource(NO_KVS, ValuesToRead.SKIP_VALUES); + } + + @Test + public void testReadNonEmptyShuffleSource() throws Exception { + runTestReadShuffleSource(KVS, ValuesToRead.READ_ALL_VALUES); + } + + @Test + public void testReadNonEmptyShuffleSourceReadingOneValue() throws Exception { + runTestReadShuffleSource(KVS, ValuesToRead.READ_ONE_VALUE); + } + + @Test + public void testReadNonEmptyShuffleSourceReadingNoValues() throws Exception { + runTestReadShuffleSource(KVS, ValuesToRead.READ_NO_VALUES); + } + + @Test + public void testReadNonEmptyShuffleSourceSkippingValues() throws Exception { + runTestReadShuffleSource(KVS, ValuesToRead.SKIP_VALUES); + } + + static byte[] fabricatePosition(int shard, byte[] key) throws Exception { + ByteArrayOutputStream os = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(os); + dos.writeInt(shard); + if (key != null) { + dos.writeInt(Arrays.hashCode(key)); + } + return os.toByteArray(); + } + + @Test + public void testReadFromEmptyShuffleSourceAndUpdateStopPosition() + throws Exception { + BatchModeExecutionContext context = new BatchModeExecutionContext(); + GroupingShuffleSource shuffleSource = + new GroupingShuffleSource<>( + PipelineOptionsFactory.create(), + null, null, null, + WindowedValue.getFullCoder( + KvCoder.of( + BigEndianIntegerCoder.of(), + IterableCoder.of(BigEndianIntegerCoder.of())), + IntervalWindow.getCoder()), + context); + TestShuffleReader shuffleReader = new TestShuffleReader(); + try (Source.SourceIterator>>> iter = + shuffleSource.iterator(shuffleReader)) { + + Position proposedStopPosition = new Position(); + String stop = encodeBase64URLSafeString(fabricatePosition(0, null)); + proposedStopPosition.setShufflePosition(stop); + + // Cannot update stop position since all input was consumed. + Assert.assertEquals(null, iter.updateStopPosition( + cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + } + } + + @Test + public void testReadFromShuffleSourceAndFailToUpdateStopPosition() + throws Exception { + BatchModeExecutionContext context = new BatchModeExecutionContext(); + final int kFirstShard = 0; + + TestShuffleReader shuffleReader = new TestShuffleReader(); + final int kNumRecords = 2; + for (int i = 0; i < kNumRecords; ++i) { + byte[] key = CoderUtils.encodeToByteArray(BigEndianIntegerCoder.of(), i); + shuffleReader.addEntry(new ShuffleEntry( + fabricatePosition(kFirstShard, key), key, null, key)); + } + + // Note that TestShuffleReader start/end positions are in the + // space of keys not the positions (TODO: should probably always + // use positions instead). + String stop = encodeBase64URLSafeString( + fabricatePosition(kNumRecords, null)); + GroupingShuffleSource shuffleSource = + new GroupingShuffleSource<>( + PipelineOptionsFactory.create(), + null, null, stop, + WindowedValue.getFullCoder( + KvCoder.of( + BigEndianIntegerCoder.of(), + IterableCoder.of(BigEndianIntegerCoder.of())), + IntervalWindow.getCoder()), + context); + + try (Source.SourceIterator>>> iter = + shuffleSource.iterator(shuffleReader)) { + + Position proposedStopPosition = new Position(); + proposedStopPosition.setShufflePosition( + encodeBase64URLSafeString(fabricatePosition(kNumRecords + 1, null))); + + // Cannot update the stop position since the value provided is + // past the current stop position. + Assert.assertEquals(null, iter.updateStopPosition( + cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + + int i = 0; + for (; iter.hasNext(); ++i) { + KV> elem = iter.next().getValue(); + if (i == 0) { + // First record + byte[] key = CoderUtils.encodeToByteArray(BigEndianIntegerCoder.of(), i); + proposedStopPosition.setShufflePosition( + encodeBase64URLSafeString(fabricatePosition(kFirstShard, key))); + // Cannot update stop position since it is identical with + // the position of the record that was just returned. + Assert.assertEquals(null, iter.updateStopPosition( + cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + + proposedStopPosition.setShufflePosition( + encodeBase64URLSafeString(fabricatePosition(kFirstShard, null))); + // Cannot update stop position since it comes before current position + Assert.assertEquals(null, iter.updateStopPosition( + cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + } + } + Assert.assertEquals(kNumRecords, i); + + proposedStopPosition.setShufflePosition( + encodeBase64URLSafeString(fabricatePosition(kFirstShard, null))); + // Cannot update stop position since all input was consumed. + Assert.assertEquals(null, iter.updateStopPosition( + cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + } + } + + @Test + public void testReadFromShuffleSourceAndUpdateStopPosition() + throws Exception { + BatchModeExecutionContext context = new BatchModeExecutionContext(); + GroupingShuffleSource shuffleSource = + new GroupingShuffleSource<>( + PipelineOptionsFactory.create(), + null, null, null, + WindowedValue.getFullCoder( + KvCoder.of( + BigEndianIntegerCoder.of(), + IterableCoder.of(BigEndianIntegerCoder.of())), + IntervalWindow.getCoder()), + context); + + TestShuffleReader shuffleReader = new TestShuffleReader(); + final int kNumRecords = 10; + final int kFirstShard = 0; + final int kSecondShard = 1; + + // Setting up two shards with kNumRecords each; keys are unique + // (hence groups of values for the same key are singletons) + // therefore each record comes with a unique position constructed. + for (int i = 0; i < kNumRecords; ++i) { + byte[] keyByte = CoderUtils.encodeToByteArray( + BigEndianIntegerCoder.of(), i); + ShuffleEntry entry = new ShuffleEntry( + fabricatePosition(kFirstShard, keyByte), keyByte, null, keyByte); + shuffleReader.addEntry(entry); + } + + for (int i = kNumRecords; i < 2 * kNumRecords; ++i) { + byte[] keyByte = CoderUtils.encodeToByteArray( + BigEndianIntegerCoder.of(), i); + + ShuffleEntry entry = new ShuffleEntry( + fabricatePosition(kSecondShard, keyByte), keyByte, null, keyByte); + shuffleReader.addEntry(entry); + } + + int i = 0; + try (Source.SourceIterator>>> iter = + shuffleSource.iterator(shuffleReader)) { + + Position proposedStopPosition = new Position(); + + Assert.assertNull(iter.updateStopPosition( + cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + + // Stop at the shard boundary + String stop = encodeBase64URLSafeString(fabricatePosition(kSecondShard, null)); + proposedStopPosition.setShufflePosition(stop); + + Assert.assertEquals( + stop, + sourcePositionToCloudPosition( + iter.updateStopPosition( + cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))) + .getShufflePosition()); + + while (iter.hasNext()) { + Assert.assertTrue(iter.hasNext()); + Assert.assertTrue(iter.hasNext()); + + KV> elem = iter.next().getValue(); + int key = elem.getKey(); + Assert.assertEquals(key, i); + + Iterable valuesIterable = elem.getValue(); + Iterator valuesIterator = valuesIterable.iterator(); + + int j = 0; + while (valuesIterator.hasNext()) { + Assert.assertTrue(valuesIterator.hasNext()); + Assert.assertTrue(valuesIterator.hasNext()); + + int value = valuesIterator.next(); + Assert.assertEquals(value, i); + ++j; + } + Assert.assertEquals(j, 1); + ++i; + } + + ApproximateProgress progress = + sourceProgressToCloudProgress(iter.getProgress()); + Assert.assertEquals(stop, progress.getPosition().getShufflePosition()); + } + Assert.assertEquals(i, kNumRecords); + } + + @Test + public void testGetApproximateProgress() throws Exception { + // Store the positions of all KVs returned. + List positionsList = new ArrayList(); + + BatchModeExecutionContext context = new BatchModeExecutionContext(); + GroupingShuffleSource shuffleSource = + new GroupingShuffleSource<>( + PipelineOptionsFactory.create(), + null, null, null, + WindowedValue.getFullCoder( + KvCoder.of( + BigEndianIntegerCoder.of(), + IterableCoder.of(BigEndianIntegerCoder.of())), + IntervalWindow.getCoder()), + context); + + TestShuffleReader shuffleReader = new TestShuffleReader(); + final int kNumRecords = 10; + + for (int i = 0; i < kNumRecords; ++i) { + byte[] position = fabricatePosition(i, null); + byte[] keyByte = CoderUtils.encodeToByteArray(BigEndianIntegerCoder.of(), i); + positionsList.add(position); + ShuffleEntry entry = new ShuffleEntry(position, keyByte, null, keyByte); + shuffleReader.addEntry(entry); + } + + try (Source.SourceIterator>>> sourceIterator = + shuffleSource.iterator(shuffleReader)) { + Integer i = 0; + while (sourceIterator.hasNext()) { + Assert.assertTrue(sourceIterator.hasNext()); + ApproximateProgress progress = sourceProgressToCloudProgress(sourceIterator.getProgress()); + Assert.assertNotNull(progress.getPosition().getShufflePosition()); + + // Compare returned position with the expected position. + Assert.assertEquals(ByteArrayShufflePosition.of(positionsList.get(i)).encodeBase64(), + progress.getPosition().getShufflePosition()); + + WindowedValue>> elem = sourceIterator.next(); + Assert.assertEquals(i, elem.getValue().getKey()); + i++; + } + Assert.assertFalse(sourceIterator.hasNext()); + + ApproximateProgress finalProgress = + sourceProgressToCloudProgress(sourceIterator.getProgress()); + Assert.assertEquals(1.0, + (float) finalProgress.getPercentComplete(), 0.000000001); + Assert.assertEquals(Duration.ZERO, fromCloudDuration(finalProgress.getRemainingTime())); + } + } + + private ApproximateProgress createApproximateProgress( + com.google.api.services.dataflow.model.Position position) { + return new ApproximateProgress().setPosition(position); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactoryTest.java new file mode 100644 index 0000000000000..64cf4f5520217 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactoryTest.java @@ -0,0 +1,110 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.runners.worker.InMemorySourceTest.encodedElements; +import static com.google.cloud.dataflow.sdk.util.Structs.addLong; +import static com.google.cloud.dataflow.sdk.util.Structs.addStringList; + +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Tests for InMemorySourceFactory. + */ +@RunWith(JUnit4.class) +public class InMemorySourceFactoryTest { + static com.google.api.services.dataflow.model.Source createInMemoryCloudSource( + List elements, + Long start, + Long end, + Coder coder) + throws Exception { + List encodedElements = encodedElements(elements, coder); + + CloudObject spec = CloudObject.forClassName("InMemorySource"); + addStringList(spec, PropertyNames.ELEMENTS, encodedElements); + + if (start != null) { + addLong(spec, PropertyNames.START_INDEX, start); + } + if (end != null) { + addLong(spec, PropertyNames.END_INDEX, end); + } + + com.google.api.services.dataflow.model.Source cloudSource = + new com.google.api.services.dataflow.model.Source(); + cloudSource.setSpec(spec); + cloudSource.setCodec(coder.asCloudObject()); + + return cloudSource; + } + + void runTestCreateInMemorySource(List elements, + Long start, + Long end, + int expectedStart, + int expectedEnd, + Coder coder) + throws Exception { + com.google.api.services.dataflow.model.Source cloudSource = + createInMemoryCloudSource(elements, start, end, coder); + + Source source = SourceFactory.create(PipelineOptionsFactory.create(), cloudSource, + new BatchModeExecutionContext()); + Assert.assertThat(source, new IsInstanceOf(InMemorySource.class)); + InMemorySource inMemorySource = (InMemorySource) source; + Assert.assertEquals(encodedElements(elements, coder), + inMemorySource.encodedElements); + Assert.assertEquals(expectedStart, inMemorySource.startIndex); + Assert.assertEquals(expectedEnd, inMemorySource.endIndex); + Assert.assertEquals(coder, inMemorySource.coder); + } + + @Test + public void testCreatePlainInMemorySource() throws Exception { + runTestCreateInMemorySource( + Arrays.asList("hi", "there", "bob"), + null, null, + 0, 3, + StringUtf8Coder.of()); + } + + @Test + public void testCreateRichInMemorySource() throws Exception { + runTestCreateInMemorySource( + Arrays.asList(33, 44, 55, 66, 77, 88), + 1L, 3L, + 1, 3, + BigEndianIntegerCoder.of()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceTest.java new file mode 100644 index 0000000000000..d7574c517b4e8 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceTest.java @@ -0,0 +1,236 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray; +import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString; + +import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.Position; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Tests for InMemorySource. + */ +@RunWith(JUnit4.class) +public class InMemorySourceTest { + static List encodedElements(List elements, Coder coder) + throws Exception { + List encodedElements = new ArrayList<>(); + for (T element : elements) { + byte[] encodedElement = encodeToByteArray(coder, element); + String encodedElementString = byteArrayToJsonString(encodedElement); + encodedElements.add(encodedElementString); + } + return encodedElements; + } + + void runTestReadInMemorySource(List elements, + Long startIndex, + Long endIndex, + List expectedElements, + List expectedSizes, + Coder coder) + throws Exception { + InMemorySource inMemorySource = new InMemorySource<>( + encodedElements(elements, coder), startIndex, endIndex, coder); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(inMemorySource); + List actualElements = new ArrayList<>(); + try (Source.SourceIterator iterator = inMemorySource.iterator()) { + for (long i = inMemorySource.startIndex; iterator.hasNext(); i++) { + Assert.assertEquals( + new ApproximateProgress().setPosition(makeIndexPosition(i)), + sourceProgressToCloudProgress(iterator.getProgress())); + actualElements.add(iterator.next()); + } + } + Assert.assertEquals(expectedElements, actualElements); + Assert.assertEquals(expectedSizes, observer.getActualSizes()); + } + + @Test + public void testReadAllElements() throws Exception { + runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), + null, + null, + Arrays.asList(33, 44, 55, 66, 77, 88), + Arrays.asList(4, 4, 4, 4, 4, 4), + BigEndianIntegerCoder.of()); + } + + @Test + public void testReadElementsFromStart() throws Exception { + runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), + 2L, + null, + Arrays.asList(55, 66, 77, 88), + Arrays.asList(4, 4, 4, 4), + BigEndianIntegerCoder.of()); + } + + @Test + public void testReadElementsToEnd() throws Exception { + runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), + null, + 3L, + Arrays.asList(33, 44, 55), + Arrays.asList(4, 4, 4), + BigEndianIntegerCoder.of()); + } + + @Test + public void testReadElementsFromStartToEnd() throws Exception { + runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), + 2L, + 5L, + Arrays.asList(55, 66, 77), + Arrays.asList(4, 4, 4), + BigEndianIntegerCoder.of()); + } + + @Test + public void testReadElementsOffEnd() throws Exception { + runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), + null, + 30L, + Arrays.asList(33, 44, 55, 66, 77, 88), + Arrays.asList(4, 4, 4, 4, 4, 4), + BigEndianIntegerCoder.of()); + } + + @Test + public void testReadElementsFromStartPastEnd() throws Exception { + runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), + 20L, + null, + Arrays.asList(), + Arrays.asList(), + BigEndianIntegerCoder.of()); + } + + @Test + public void testReadElementsFromStartToEndEmptyRange() throws Exception { + runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), + 2L, + 2L, + Arrays.asList(), + Arrays.asList(), + BigEndianIntegerCoder.of()); + } + + @Test + public void testReadNoElements() throws Exception { + runTestReadInMemorySource(Arrays.asList(), + null, + null, + Arrays.asList(), + Arrays.asList(), + BigEndianIntegerCoder.of()); + } + + @Test + public void testReadNoElementsFromStartToEndEmptyRange() throws Exception { + runTestReadInMemorySource(Arrays.asList(), + 0L, + 0L, + Arrays.asList(), + Arrays.asList(), + BigEndianIntegerCoder.of()); + } + + @Test + public void testUpdatePosition() throws Exception { + List elements = Arrays.asList(33, 44, 55, 66, 77, 88); + final long start = 1L; + final long stop = 3L; + final long end = 4L; + + Coder coder = BigEndianIntegerCoder.of(); + InMemorySource inMemorySource = new InMemorySource<>( + encodedElements(elements, coder), start, end, coder); + + // Illegal proposed stop position. + try (Source.SourceIterator iterator = inMemorySource.iterator()) { + Assert.assertNull(iterator.updateStopPosition( + cloudProgressToSourceProgress(new ApproximateProgress()))); + Assert.assertNull(iterator.updateStopPosition( + cloudProgressToSourceProgress( + new ApproximateProgress().setPosition(makeIndexPosition(null))))); + } + + // Successful update. + try (InMemorySource.InMemorySourceIterator iterator = + (InMemorySource.InMemorySourceIterator) inMemorySource.iterator()) { + Assert.assertEquals( + makeIndexPosition(stop), + sourcePositionToCloudPosition( + iterator.updateStopPosition( + cloudProgressToSourceProgress( + new ApproximateProgress().setPosition(makeIndexPosition(stop)))))); + Assert.assertEquals(stop, iterator.endPosition); + Assert.assertEquals(44, iterator.next().intValue()); + Assert.assertEquals(55, iterator.next().intValue()); + Assert.assertFalse(iterator.hasNext()); + } + + // Proposed stop position is before the current position, no update. + try (InMemorySource.InMemorySourceIterator iterator = + (InMemorySource.InMemorySourceIterator) inMemorySource.iterator()) { + Assert.assertEquals(44, iterator.next().intValue()); + Assert.assertEquals(55, iterator.next().intValue()); + Assert.assertNull(iterator.updateStopPosition( + cloudProgressToSourceProgress( + new ApproximateProgress().setPosition(makeIndexPosition(stop))))); + Assert.assertEquals((int) end, iterator.endPosition); + Assert.assertTrue(iterator.hasNext()); + } + + // Proposed stop position is after the current stop (end) position, no update. + try (InMemorySource.InMemorySourceIterator iterator = + (InMemorySource.InMemorySourceIterator) inMemorySource.iterator()) { + Assert.assertNull( + iterator.updateStopPosition( + cloudProgressToSourceProgress( + new ApproximateProgress().setPosition(makeIndexPosition(end + 1))))); + Assert.assertEquals((int) end, iterator.endPosition); + } + } + + private Position makeIndexPosition(Long index) { + Position position = new Position(); + if (index != null) { + position.setRecordIndex(index); + } + return position; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java new file mode 100644 index 0000000000000..fae22797ef89c --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java @@ -0,0 +1,567 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThat; + +import com.google.api.services.dataflow.model.FlattenInstruction; +import com.google.api.services.dataflow.model.InstructionInput; +import com.google.api.services.dataflow.model.InstructionOutput; +import com.google.api.services.dataflow.model.MapTask; +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.WriteInstruction; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.worker.SinkFactoryTest.TestSink; +import com.google.cloud.dataflow.sdk.runners.worker.SinkFactoryTest.TestSinkFactory; +import com.google.cloud.dataflow.sdk.runners.worker.SourceFactoryTest.TestSource; +import com.google.cloud.dataflow.sdk.runners.worker.SourceFactoryTest.TestSourceFactory; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.util.StringUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils.TestOperation; +import com.google.cloud.dataflow.sdk.util.common.worker.FlattenOperation; +import com.google.cloud.dataflow.sdk.util.common.worker.MapTaskExecutor; +import com.google.cloud.dataflow.sdk.util.common.worker.Operation; +import com.google.cloud.dataflow.sdk.util.common.worker.ParDoOperation; +import com.google.cloud.dataflow.sdk.util.common.worker.PartialGroupByKeyOperation; +import com.google.cloud.dataflow.sdk.util.common.worker.ReadOperation; +import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; +import com.google.cloud.dataflow.sdk.util.common.worker.WriteOperation; + +import org.hamcrest.CoreMatchers; +import org.hamcrest.core.IsInstanceOf; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * Tests for MapTaskExecutorFactory. + */ +@RunWith(JUnit4.class) +public class MapTaskExecutorFactoryTest { + @Test + public void testCreateMapTaskExecutor() throws Exception { + List instructions = + Arrays.asList( + createReadInstruction("Read"), + createParDoInstruction(0, 0, "DoFn1"), + createParDoInstruction(0, 0, "DoFn2"), + createFlattenInstruction(1, 0, 2, 0, "Flatten"), + createWriteInstruction(3, 0, "Write")); + + MapTask mapTask = new MapTask(); + mapTask.setStageName("test"); + mapTask.setInstructions(instructions); + + CounterSet counterSet = null; + try (MapTaskExecutor executor = + MapTaskExecutorFactory.create( + PipelineOptionsFactory.create(), + mapTask, + new BatchModeExecutionContext())) { + + @SuppressWarnings("unchecked") + List operations = (List) executor.operations; + assertThat( + operations, + CoreMatchers.hasItems( + new IsInstanceOf(ReadOperation.class), + new IsInstanceOf(ParDoOperation.class), + new IsInstanceOf(ParDoOperation.class), + new IsInstanceOf(FlattenOperation.class), + new IsInstanceOf(WriteOperation.class))); + counterSet = executor.getOutputCounters(); + } + + assertEquals( + new CounterSet( + Counter.longs("read_output_name-ElementCount", SUM) + .resetToValue(0L), + Counter.longs("read_output_name-MeanByteCount", MEAN) + .resetToValue(0, 0L), + Counter.longs("Read-ByteCount", SUM).resetToValue(0L), + Counter.longs("test-Read-start-msecs", SUM) + .resetToValue(0L), + Counter.longs("test-Read-read-msecs", SUM) + .resetToValue(0L), + Counter.longs("test-Read-process-msecs", SUM) + .resetToValue(0L), + Counter.longs("test-Read-finish-msecs", SUM) + .resetToValue(0L), + Counter.longs("DoFn1_output-ElementCount", SUM) + .resetToValue(0L), + Counter.longs("DoFn1_output-MeanByteCount", MEAN) + .resetToValue(0, 0L), + Counter.longs("test-DoFn1-start-msecs", SUM).resetToValue(0L), + Counter.longs("test-DoFn1-process-msecs", SUM).resetToValue(0L), + Counter.longs("test-DoFn1-finish-msecs", SUM).resetToValue(0L), + Counter.longs("DoFn2_output-ElementCount", SUM) + .resetToValue(0L), + Counter.longs("DoFn2_output-MeanByteCount", MEAN) + .resetToValue(0, 0L), + Counter.longs("test-DoFn2-start-msecs", SUM).resetToValue(0L), + Counter.longs("test-DoFn2-process-msecs", SUM).resetToValue(0L), + Counter.longs("test-DoFn2-finish-msecs", SUM).resetToValue(0L), + Counter.longs("flatten_output_name-ElementCount", SUM) + .resetToValue(0L), + Counter.longs("flatten_output_name-MeanByteCount", MEAN) + .resetToValue(0, 0L), + Counter.longs("test-Flatten-start-msecs", SUM).resetToValue(0L), + Counter.longs("test-Flatten-process-msecs", SUM).resetToValue(0L), + Counter.longs("test-Flatten-finish-msecs", SUM).resetToValue(0L), + Counter.longs("Write-ByteCount", SUM) + .resetToValue(0L), + Counter.longs("test-Write-start-msecs", SUM).resetToValue(0L), + Counter.longs("test-Write-process-msecs", SUM).resetToValue(0L), + Counter.longs("test-Write-finish-msecs", SUM).resetToValue(0L), + Counter.longs("test-other-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-other-msecs")).getAggregate(false))), + counterSet); + } + + @Test + public void testExecutionContextPlumbing() throws Exception { + List instructions = + Arrays.asList( + createReadInstruction("Read"), + createParDoInstruction(0, 0, "DoFn1"), + createParDoInstruction(1, 0, "DoFn2"), + createWriteInstruction(2, 0, "Write")); + + MapTask mapTask = new MapTask(); + mapTask.setInstructions(instructions); + + BatchModeExecutionContext context = new BatchModeExecutionContext(); + + try (MapTaskExecutor executor = + MapTaskExecutorFactory.create( + PipelineOptionsFactory.create(), mapTask, context)) { + executor.execute(); + } + + List stepNames = new ArrayList<>(); + for (ExecutionContext.StepContext stepContext + : context.getAllStepContexts()) { + stepNames.add(stepContext.getStepName()); + } + assertThat(stepNames, CoreMatchers.hasItems("DoFn1", "DoFn2")); + } + + static ParallelInstruction createReadInstruction(String name) { + CloudObject spec = CloudObject.forClass(TestSourceFactory.class); + + com.google.api.services.dataflow.model.Source cloudSource = + new com.google.api.services.dataflow.model.Source(); + cloudSource.setSpec(spec); + cloudSource.setCodec(CloudObject.forClass(StringUtf8Coder.class)); + + ReadInstruction readInstruction = new ReadInstruction(); + readInstruction.setSource(cloudSource); + + InstructionOutput output = new InstructionOutput(); + output.setName("read_output_name"); + output.setCodec(CloudObject.forClass(StringUtf8Coder.class)); + + ParallelInstruction instruction = new ParallelInstruction(); + instruction.setSystemName(name); + instruction.setRead(readInstruction); + instruction.setOutputs(Arrays.asList(output)); + + return instruction; + } + + @Test + public void testCreateReadOperation() throws Exception { + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler(counterPrefix, + counterSet.getAddCounterMutator()); + Operation operation = MapTaskExecutorFactory.createOperation( + PipelineOptionsFactory.create(), + createReadInstruction("Read"), + new BatchModeExecutionContext(), + Collections.emptyList(), + counterPrefix, + counterSet.getAddCounterMutator(), + stateSampler); + assertThat(operation, new IsInstanceOf(ReadOperation.class)); + ReadOperation readOperation = (ReadOperation) operation; + + assertEquals(readOperation.receivers.length, 1); + assertEquals(readOperation.receivers[0].getReceiverCount(), 0); + assertEquals(readOperation.initializationState, + Operation.InitializationState.UNSTARTED); + assertThat(readOperation.source, new IsInstanceOf(TestSource.class)); + + assertEquals( + new CounterSet( + Counter.longs("test-Read-start-msecs", SUM) + .resetToValue(0L), + Counter.longs("read_output_name-MeanByteCount", MEAN) + .resetToValue(0, 0L), + Counter.longs("Read-ByteCount", SUM).resetToValue(0L), + Counter.longs("test-Read-finish-msecs", SUM) + .resetToValue(0L), + Counter.longs("test-Read-read-msecs", SUM), + Counter.longs("test-Read-process-msecs", SUM), + Counter.longs("read_output_name-ElementCount", SUM).resetToValue(0L)), + counterSet); + } + + static ParallelInstruction createWriteInstruction( + int producerIndex, + int producerOutputNum, + String systemName) { + InstructionInput cloudInput = new InstructionInput(); + cloudInput.setProducerInstructionIndex(producerIndex); + cloudInput.setOutputNum(producerOutputNum); + + CloudObject spec = CloudObject.forClass(TestSinkFactory.class); + + com.google.api.services.dataflow.model.Sink cloudSink = + new com.google.api.services.dataflow.model.Sink(); + cloudSink.setSpec(spec); + cloudSink.setCodec(CloudObject.forClass(StringUtf8Coder.class)); + + WriteInstruction writeInstruction = new WriteInstruction(); + writeInstruction.setInput(cloudInput); + writeInstruction.setSink(cloudSink); + + ParallelInstruction instruction = new ParallelInstruction(); + instruction.setWrite(writeInstruction); + instruction.setSystemName(systemName); + + return instruction; + } + + @Test + public void testCreateWriteOperation() throws Exception { + List priorOperations = Arrays.asList(new Operation[]{ + new TestOperation(3), + new TestOperation(5), + new TestOperation(1) }); + + int producerIndex = 1; + int producerOutputNum = 2; + + ParallelInstruction instruction = + createWriteInstruction(producerIndex, producerOutputNum, "WriteOperation"); + + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler(counterPrefix, + counterSet.getAddCounterMutator()); + Operation operation = MapTaskExecutorFactory.createOperation( + PipelineOptionsFactory.create(), + instruction, + new BatchModeExecutionContext(), + priorOperations, + counterPrefix, + counterSet.getAddCounterMutator(), + stateSampler); + assertThat(operation, new IsInstanceOf(WriteOperation.class)); + WriteOperation writeOperation = (WriteOperation) operation; + + assertEquals(writeOperation.receivers.length, 0); + assertEquals(writeOperation.initializationState, + Operation.InitializationState.UNSTARTED); + assertThat(writeOperation.sink, + new IsInstanceOf(TestSink.class)); + + assertSame( + writeOperation, + priorOperations.get(producerIndex).receivers[producerOutputNum] + .getOnlyReceiver()); + + assertEquals( + new CounterSet( + Counter.longs("WriteOperation-ByteCount", SUM) + .resetToValue(0L), + Counter.longs("test-WriteOperation-start-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-WriteOperation-start-msecs")).getAggregate(false)), + Counter.longs("test-WriteOperation-process-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-WriteOperation-process-msecs")).getAggregate(false)), + Counter.longs("test-WriteOperation-finish-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-WriteOperation-finish-msecs")).getAggregate(false))), + counterSet); + } + + static class TestDoFn extends DoFn { + @Override + public void processElement(ProcessContext c) { } + } + + static ParallelInstruction createParDoInstruction( + int producerIndex, + int producerOutputNum, + String systemName) { + InstructionInput cloudInput = new InstructionInput(); + cloudInput.setProducerInstructionIndex(producerIndex); + cloudInput.setOutputNum(producerOutputNum); + + TestDoFn fn = new TestDoFn(); + + String serializedFn = + StringUtils.byteArrayToJsonString( + SerializableUtils.serializeToByteArray(fn)); + + CloudObject cloudUserFn = CloudObject.forClassName("DoFn"); + addString(cloudUserFn, PropertyNames.SERIALIZED_FN, serializedFn); + + ParDoInstruction parDoInstruction = new ParDoInstruction(); + parDoInstruction.setInput(cloudInput); + parDoInstruction.setNumOutputs(1); + parDoInstruction.setUserFn(cloudUserFn); + + InstructionOutput output = new InstructionOutput(); + output.setName(systemName + "_output"); + output.setCodec(CloudObject.forClass(StringUtf8Coder.class)); + + ParallelInstruction instruction = new ParallelInstruction(); + instruction.setParDo(parDoInstruction); + instruction.setOutputs(Arrays.asList(output)); + instruction.setSystemName(systemName); + return instruction; + } + + @Test + public void testCreateParDoOperation() throws Exception { + List priorOperations = Arrays.asList(new Operation[]{ + new TestOperation(3), + new TestOperation(5), + new TestOperation(1) }); + + int producerIndex = 1; + int producerOutputNum = 2; + + ParallelInstruction instruction = + createParDoInstruction(producerIndex, producerOutputNum, "DoFn"); + + BatchModeExecutionContext context = new BatchModeExecutionContext(); + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler(counterPrefix, + counterSet.getAddCounterMutator()); + Operation operation = MapTaskExecutorFactory.createOperation( + PipelineOptionsFactory.create(), + instruction, + context, + priorOperations, + counterPrefix, + counterSet.getAddCounterMutator(), stateSampler); + assertThat(operation, new IsInstanceOf(ParDoOperation.class)); + ParDoOperation parDoOperation = (ParDoOperation) operation; + + assertEquals(parDoOperation.receivers.length, 1); + assertEquals(parDoOperation.receivers[0].getReceiverCount(), 0); + assertEquals(parDoOperation.initializationState, + Operation.InitializationState.UNSTARTED); + assertThat(parDoOperation.fn, + new IsInstanceOf(NormalParDoFn.class)); + NormalParDoFn normalParDoFn = (NormalParDoFn) parDoOperation.fn; + + assertThat(normalParDoFn.fn, + new IsInstanceOf(TestDoFn.class)); + + assertSame( + parDoOperation, + priorOperations.get(producerIndex).receivers[producerOutputNum] + .getOnlyReceiver()); + + assertEquals(context, normalParDoFn.executionContext); + } + + static ParallelInstruction createPartialGroupByKeyInstruction( + int producerIndex, + int producerOutputNum) { + InstructionInput cloudInput = new InstructionInput(); + cloudInput.setProducerInstructionIndex(producerIndex); + cloudInput.setOutputNum(producerOutputNum); + + PartialGroupByKeyInstruction pgbkInstruction = + new PartialGroupByKeyInstruction(); + pgbkInstruction.setInput(cloudInput); + pgbkInstruction.setInputElementCodec( + makeCloudEncoding(FullWindowedValueCoder.class.getName(), + makeCloudEncoding("KvCoder", + makeCloudEncoding("StringUtf8Coder"), + makeCloudEncoding("BigEndianIntegerCoder")), + IntervalWindow.getCoder().asCloudObject())); + + InstructionOutput output = new InstructionOutput(); + output.setName("pgbk_output_name"); + output.setCodec(makeCloudEncoding( + "KvCoder", + makeCloudEncoding("StringUtf8Coder"), + makeCloudEncoding( + "IterableCoder", + makeCloudEncoding("BigEndianIntegerCoder")))); + + ParallelInstruction instruction = new ParallelInstruction(); + instruction.setPartialGroupByKey(pgbkInstruction); + instruction.setOutputs(Arrays.asList(output)); + + return instruction; + } + + @Test + public void testCreatePartialGroupByKeyOperation() throws Exception { + List priorOperations = Arrays.asList(new Operation[]{ + new TestOperation(3), + new TestOperation(5), + new TestOperation(1) }); + + int producerIndex = 1; + int producerOutputNum = 2; + + ParallelInstruction instruction = + createPartialGroupByKeyInstruction(producerIndex, producerOutputNum); + + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler(counterPrefix, + counterSet.getAddCounterMutator()); + Operation operation = MapTaskExecutorFactory.createOperation( + PipelineOptionsFactory.create(), + instruction, + new BatchModeExecutionContext(), + priorOperations, + counterPrefix, + counterSet.getAddCounterMutator(), + stateSampler); + assertThat(operation, instanceOf(PartialGroupByKeyOperation.class)); + PartialGroupByKeyOperation pgbkOperation = + (PartialGroupByKeyOperation) operation; + + assertEquals(pgbkOperation.receivers.length, 1); + assertEquals(pgbkOperation.receivers[0].getReceiverCount(), 0); + assertEquals(pgbkOperation.initializationState, + Operation.InitializationState.UNSTARTED); + + assertSame( + pgbkOperation, + priorOperations.get(producerIndex).receivers[producerOutputNum] + .getOnlyReceiver()); + } + + static ParallelInstruction createFlattenInstruction( + int producerIndex1, + int producerOutputNum1, + int producerIndex2, + int producerOutputNum2, + String systemName) { + List cloudInputs = new ArrayList<>(); + + InstructionInput cloudInput1 = new InstructionInput(); + cloudInput1.setProducerInstructionIndex(producerIndex1); + cloudInput1.setOutputNum(producerOutputNum1); + cloudInputs.add(cloudInput1); + + InstructionInput cloudInput2 = new InstructionInput(); + cloudInput2.setProducerInstructionIndex(producerIndex2); + cloudInput2.setOutputNum(producerOutputNum2); + cloudInputs.add(cloudInput2); + + FlattenInstruction flattenInstruction = new FlattenInstruction(); + flattenInstruction.setInputs(cloudInputs); + + InstructionOutput output = new InstructionOutput(); + output.setName("flatten_output_name"); + output.setCodec(makeCloudEncoding(StringUtf8Coder.class.getName())); + + ParallelInstruction instruction = new ParallelInstruction(); + instruction.setFlatten(flattenInstruction); + instruction.setOutputs(Arrays.asList(output)); + instruction.setSystemName(systemName); + + return instruction; + } + + @Test + public void testCreateFlattenOperation() throws Exception { + List priorOperations = Arrays.asList(new Operation[]{ + new TestOperation(3), + new TestOperation(5), + new TestOperation(1) }); + + int producerIndex1 = 1; + int producerOutputNum1 = 2; + int producerIndex2 = 0; + int producerOutputNum2 = 1; + + ParallelInstruction instruction = + createFlattenInstruction(producerIndex1, producerOutputNum1, + producerIndex2, producerOutputNum2, "Flatten"); + + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler(counterPrefix, + counterSet.getAddCounterMutator()); + Operation operation = MapTaskExecutorFactory.createOperation( + PipelineOptionsFactory.create(), + instruction, + new BatchModeExecutionContext(), + priorOperations, + counterPrefix, + counterSet.getAddCounterMutator(), + stateSampler); + assertThat(operation, new IsInstanceOf(FlattenOperation.class)); + FlattenOperation flattenOperation = (FlattenOperation) operation; + + assertEquals(flattenOperation.receivers.length, 1); + assertEquals(flattenOperation.receivers[0].getReceiverCount(), 0); + assertEquals(flattenOperation.initializationState, + Operation.InitializationState.UNSTARTED); + + assertSame( + flattenOperation, + priorOperations.get(producerIndex1).receivers[producerOutputNum1] + .getOnlyReceiver()); + assertSame( + flattenOperation, + priorOperations.get(producerIndex2).receivers[producerOutputNum2] + .getOnlyReceiver()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java new file mode 100644 index 0000000000000..f94ab8339f9d1 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java @@ -0,0 +1,331 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.collection.IsIterableContainingInOrder.contains; +import static org.hamcrest.core.AnyOf.anyOf; +import static org.hamcrest.core.IsEqual.equalTo; +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.PTuple; +import com.google.cloud.dataflow.sdk.util.UserCodeException; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.worker.Receiver; +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * Tests for NormalParDoFn. + */ +@RunWith(JUnit4.class) +public class NormalParDoFnTest { + static class TestDoFn extends DoFn { + enum State { UNSTARTED, STARTED, PROCESSING, FINISHED } + State state = State.UNSTARTED; + + List sideOutputTupleTags; + + public TestDoFn(List sideOutputTags) { + sideOutputTupleTags = new ArrayList<>(); + for (String sideOutputTag : sideOutputTags) { + sideOutputTupleTags.add(new TupleTag(sideOutputTag)); + } + } + + @Override + public void startBundle(Context c) { + assertEquals(State.UNSTARTED, state); + state = State.STARTED; + outputToAll(c, "started"); + } + + @Override + public void processElement(ProcessContext c) { + assertThat(state, anyOf(equalTo(State.STARTED), + equalTo(State.PROCESSING))); + state = State.PROCESSING; + outputToAll(c, "processing: " + c.element()); + } + + @Override + public void finishBundle(Context c) { + assertThat(state, anyOf(equalTo(State.STARTED), + equalTo(State.PROCESSING))); + state = State.FINISHED; + outputToAll(c, "finished"); + } + + private void outputToAll(Context c, String value) { + c.output(value); + for (TupleTag sideOutputTupleTag : sideOutputTupleTags) { + c.sideOutput(sideOutputTupleTag, + sideOutputTupleTag.getId() + ": " + value); + } + } + } + + static class TestErrorDoFn extends DoFn { + + // Used to test nested stack traces. + private void nestedFunctionBeta(String s) { + throw new RuntimeException(s); + } + + private void nestedFunctionAlpha(String s) { + nestedFunctionBeta(s); + } + + @Override + public void startBundle(Context c) { + nestedFunctionAlpha("test error in initialize"); + } + + @Override + public void processElement(ProcessContext c) { + nestedFunctionBeta("test error in process"); + } + + @Override + public void finishBundle(Context c) { + throw new RuntimeException("test error in finalize"); + } + } + + static class TestReceiver implements Receiver { + List receivedElems = new ArrayList<>(); + + @Override + public void process(Object outputElem) { + receivedElems.add(outputElem); + } + } + + @Test + public void testNormalParDoFn() throws Exception { + List sideOutputTags = Arrays.asList("tag1", "tag2", "tag3"); + + TestDoFn fn = new TestDoFn(sideOutputTags); + TestReceiver receiver = new TestReceiver(); + TestReceiver receiver1 = new TestReceiver(); + TestReceiver receiver2 = new TestReceiver(); + TestReceiver receiver3 = new TestReceiver(); + + PTuple sideInputValues = PTuple.empty(); + + List outputTags = new ArrayList<>(); + outputTags.add("output"); + outputTags.addAll(sideOutputTags); + NormalParDoFn normalParDoFn = + new NormalParDoFn(PipelineOptionsFactory.create(), + fn, sideInputValues, outputTags, "doFn", + new BatchModeExecutionContext(), + (new CounterSet()).getAddCounterMutator()); + + normalParDoFn.startBundle(receiver, receiver1, receiver2, receiver3); + + normalParDoFn.processElement(WindowedValue.valueInGlobalWindow(3)); + normalParDoFn.processElement(WindowedValue.valueInGlobalWindow(42)); + normalParDoFn.processElement(WindowedValue.valueInGlobalWindow(666)); + + normalParDoFn.finishBundle(); + + Object[] expectedReceivedElems = { + WindowedValue.valueInGlobalWindow("started"), + WindowedValue.valueInGlobalWindow("processing: 3"), + WindowedValue.valueInGlobalWindow("processing: 42"), + WindowedValue.valueInGlobalWindow("processing: 666"), + WindowedValue.valueInGlobalWindow("finished"), + }; + assertArrayEquals(expectedReceivedElems, receiver.receivedElems.toArray()); + + Object[] expectedReceivedElems1 = { + WindowedValue.valueInGlobalWindow("tag1: started"), + WindowedValue.valueInGlobalWindow("tag1: processing: 3"), + WindowedValue.valueInGlobalWindow("tag1: processing: 42"), + WindowedValue.valueInGlobalWindow("tag1: processing: 666"), + WindowedValue.valueInGlobalWindow("tag1: finished"), + }; + assertArrayEquals(expectedReceivedElems1, receiver1.receivedElems.toArray()); + + Object[] expectedReceivedElems2 = { + WindowedValue.valueInGlobalWindow("tag2: started"), + WindowedValue.valueInGlobalWindow("tag2: processing: 3"), + WindowedValue.valueInGlobalWindow("tag2: processing: 42"), + WindowedValue.valueInGlobalWindow("tag2: processing: 666"), + WindowedValue.valueInGlobalWindow("tag2: finished"), + }; + assertArrayEquals(expectedReceivedElems2, receiver2.receivedElems.toArray()); + + Object[] expectedReceivedElems3 = { + WindowedValue.valueInGlobalWindow("tag3: started"), + WindowedValue.valueInGlobalWindow("tag3: processing: 3"), + WindowedValue.valueInGlobalWindow("tag3: processing: 42"), + WindowedValue.valueInGlobalWindow("tag3: processing: 666"), + WindowedValue.valueInGlobalWindow("tag3: finished"), + }; + assertArrayEquals(expectedReceivedElems3, receiver3.receivedElems.toArray()); + } + + @Test + public void testUnexpectedNumberOfReceivers() throws Exception { + TestDoFn fn = new TestDoFn(Collections.emptyList()); + TestReceiver receiver = new TestReceiver(); + + PTuple sideInputValues = PTuple.empty(); + List outputTags = Arrays.asList("output"); + NormalParDoFn normalParDoFn = + new NormalParDoFn(PipelineOptionsFactory.create(), + fn, sideInputValues, outputTags, "doFn", + new BatchModeExecutionContext(), + (new CounterSet()).getAddCounterMutator()); + + try { + normalParDoFn.startBundle(); + fail("should have failed"); + } catch (Throwable exn) { + assertThat(exn.toString(), + containsString("unexpected number of receivers")); + } + try { + normalParDoFn.startBundle(receiver, receiver); + fail("should have failed"); + } catch (Throwable exn) { + assertThat(exn.toString(), + containsString("unexpected number of receivers")); + } + } + + private List stackTraceFrameStrings(Throwable t) { + List stack = new ArrayList<>(); + for (StackTraceElement frame : t.getStackTrace()) { + // Make sure that the frame has the expected name. + stack.add(frame.toString()); + } + return stack; + } + + @Test + public void testErrorPropagation() throws Exception { + TestErrorDoFn fn = new TestErrorDoFn(); + TestReceiver receiver = new TestReceiver(); + + PTuple sideInputValues = PTuple.empty(); + List outputTags = Arrays.asList("output"); + NormalParDoFn normalParDoFn = + new NormalParDoFn(PipelineOptionsFactory.create(), + fn, sideInputValues, outputTags, "doFn", + new BatchModeExecutionContext(), + (new CounterSet()).getAddCounterMutator()); + + try { + normalParDoFn.startBundle(receiver); + fail("should have failed"); + } catch (Exception exn) { + // Because we're calling this from inside the SDK and not from a + // user's program (e.g. through Pipeline.run), the error should + // be thrown as a UserCodeException. The cause of the + // UserCodeError shouldn't contain any of the stack from within + // the SDK, since we don't want to overwhelm users with stack + // frames outside of their control. + assertThat(exn, instanceOf(UserCodeException.class)); + // Stack trace of the cause should contain three frames: + // TestErrorDoFn.nestedFunctionBeta + // TestErrorDoFn.nestedFunctionAlpha + // TestErrorDoFn.startBundle + assertThat(stackTraceFrameStrings(exn.getCause()), contains( + containsString("TestErrorDoFn.nestedFunctionBeta"), + containsString("TestErrorDoFn.nestedFunctionAlpha"), + containsString("TestErrorDoFn.startBundle"))); + assertThat(exn.toString(), + containsString("test error in initialize")); + } + + try { + normalParDoFn.processElement(WindowedValue.valueInGlobalWindow(3)); + fail("should have failed"); + } catch (Exception exn) { + // Exception should be a UserCodeException since we're calling + // from inside the SDK. + assertThat(exn, instanceOf(UserCodeException.class)); + // Stack trace of the cause should contain two frames: + // TestErrorDoFn.nestedFunctionBeta + // TestErrorDoFn.processElement + assertThat(stackTraceFrameStrings(exn.getCause()), contains( + containsString("TestErrorDoFn.nestedFunctionBeta"), + containsString("TestErrorDoFn.processElement"))); + assertThat(exn.toString(), containsString("test error in process")); + } + + try { + normalParDoFn.finishBundle(); + fail("should have failed"); + } catch (Exception exn) { + // Exception should be a UserCodeException since we're calling + // from inside the SDK. + assertThat(exn, instanceOf(UserCodeException.class)); + // Stack trace should only contain a single frame: + // TestErrorDoFn.finishBundle + assertThat(stackTraceFrameStrings(exn.getCause()), contains( + containsString("TestErrorDoFn.finishBundle"))); + assertThat(exn.toString(), containsString("test error in finalize")); + } + } + + @Test + public void testUndeclaredSideOutputs() throws Exception { + TestDoFn fn = new TestDoFn(Arrays.asList("declared", "undecl1", "undecl2", "undecl3")); + CounterSet counters = new CounterSet(); + NormalParDoFn normalParDoFn = + new NormalParDoFn(PipelineOptionsFactory.create(), fn, PTuple.empty(), + Arrays.asList("output", "declared"), "doFn", + new BatchModeExecutionContext(), + counters.getAddCounterMutator()); + + normalParDoFn.startBundle(new TestReceiver(), new TestReceiver()); + normalParDoFn.processElement(WindowedValue.valueInGlobalWindow(5)); + normalParDoFn.finishBundle(); + + assertEquals( + new CounterSet( + Counter.longs("implicit-undecl1-ElementCount", SUM) + .resetToValue(3L), + Counter.longs("implicit-undecl2-ElementCount", SUM) + .resetToValue(3L), + Counter.longs("implicit-undecl3-ElementCount", SUM) + .resetToValue(3L)), + counters); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/OrderedCodeTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/OrderedCodeTest.java new file mode 100644 index 0000000000000..6f467ba1173af --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/OrderedCodeTest.java @@ -0,0 +1,504 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import com.google.common.io.BaseEncoding; +import com.google.common.primitives.Bytes; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for OrderedCode. + */ +@RunWith(JUnit4.class) +public class OrderedCodeTest { + @Test + public void testWriteInfinity() { + OrderedCode orderedCode = new OrderedCode(); + try { + orderedCode.readInfinity(); + fail("Expected IllegalArgumentException."); + } catch (IllegalArgumentException e) { + // expected + } + orderedCode.writeInfinity(); + assertTrue(orderedCode.readInfinity()); + try { + orderedCode.readInfinity(); + fail("Expected IllegalArgumentException."); + } catch (IllegalArgumentException e) { + // expected + } + } + + @Test + public void testWriteBytes() { + byte[] first = { 'a', 'b', 'c'}; + byte[] second = { 'd', 'e', 'f'}; + byte[] last = { 'x', 'y', 'z'}; + OrderedCode orderedCode = new OrderedCode(); + orderedCode.writeBytes(first); + byte[] firstEncoded = orderedCode.getEncodedBytes(); + assertArrayEquals(orderedCode.readBytes(), first); + + orderedCode.writeBytes(first); + orderedCode.writeBytes(second); + orderedCode.writeBytes(last); + byte[] allEncoded = orderedCode.getEncodedBytes(); + assertArrayEquals(orderedCode.readBytes(), first); + assertArrayEquals(orderedCode.readBytes(), second); + assertArrayEquals(orderedCode.readBytes(), last); + + orderedCode = new OrderedCode(firstEncoded); + orderedCode.writeBytes(second); + orderedCode.writeBytes(last); + assertArrayEquals(orderedCode.getEncodedBytes(), allEncoded); + assertArrayEquals(orderedCode.readBytes(), first); + assertArrayEquals(orderedCode.readBytes(), second); + assertArrayEquals(orderedCode.readBytes(), last); + + orderedCode = new OrderedCode(allEncoded); + assertArrayEquals(orderedCode.readBytes(), first); + assertArrayEquals(orderedCode.readBytes(), second); + assertArrayEquals(orderedCode.readBytes(), last); + } + + @Test + public void testWriteNumIncreasing() { + OrderedCode orderedCode = new OrderedCode(); + orderedCode.writeNumIncreasing(0); + orderedCode.writeNumIncreasing(1); + orderedCode.writeNumIncreasing(Long.MIN_VALUE); + orderedCode.writeNumIncreasing(Long.MAX_VALUE); + assertEquals(orderedCode.readNumIncreasing(), 0); + assertEquals(orderedCode.readNumIncreasing(), 1); + assertEquals(orderedCode.readNumIncreasing(), Long.MIN_VALUE); + assertEquals(orderedCode.readNumIncreasing(), Long.MAX_VALUE); + } + + /** + * Assert that encoding the specified long via + * {@link OrderedCode#writeSignedNumIncreasing(long)} results in the bytes + * represented by the specified string of hex digits. + * E.g. assertSignedNumIncreasingEncodingEquals("3fbf", -65) asserts that + * -65 is encoded as { (byte) 0x3f, (byte) 0xbf }. + */ + private static void assertSignedNumIncreasingEncodingEquals( + String expectedHexEncoding, long num) { + OrderedCode orderedCode = new OrderedCode(); + orderedCode.writeSignedNumIncreasing(num); + assertEquals( + "Unexpected encoding for " + num, + expectedHexEncoding, + BaseEncoding.base16().lowerCase().encode(orderedCode.getEncodedBytes())); + } + + /** + * Assert that encoding various long values via + * {@link OrderedCode#writeSignedNumIncreasing(long)} produces the expected + * bytes. Expected byte sequences were generated via the c++ (authoritative) + * implementation of OrderedCode::WriteSignedNumIncreasing. + */ + @Test + public void testSignedNumIncreasing_write() { + assertSignedNumIncreasingEncodingEquals( + "003f8000000000000000", Long.MIN_VALUE); + assertSignedNumIncreasingEncodingEquals( + "003f8000000000000001", Long.MIN_VALUE + 1); + assertSignedNumIncreasingEncodingEquals( + "077fffffff", Integer.MIN_VALUE - 1L); + assertSignedNumIncreasingEncodingEquals("0780000000", Integer.MIN_VALUE); + assertSignedNumIncreasingEncodingEquals( + "0780000001", Integer.MIN_VALUE + 1); + assertSignedNumIncreasingEncodingEquals("3fbf", -65); + assertSignedNumIncreasingEncodingEquals("40", -64); + assertSignedNumIncreasingEncodingEquals("41", -63); + assertSignedNumIncreasingEncodingEquals("7d", -3); + assertSignedNumIncreasingEncodingEquals("7e", -2); + assertSignedNumIncreasingEncodingEquals("7f", -1); + assertSignedNumIncreasingEncodingEquals("80", 0); + assertSignedNumIncreasingEncodingEquals("81", 1); + assertSignedNumIncreasingEncodingEquals("82", 2); + assertSignedNumIncreasingEncodingEquals("83", 3); + assertSignedNumIncreasingEncodingEquals("bf", 63); + assertSignedNumIncreasingEncodingEquals("c040", 64); + assertSignedNumIncreasingEncodingEquals("c041", 65); + assertSignedNumIncreasingEncodingEquals( + "f87ffffffe", Integer.MAX_VALUE - 1); + assertSignedNumIncreasingEncodingEquals("f87fffffff", Integer.MAX_VALUE); + assertSignedNumIncreasingEncodingEquals( + "f880000000", Integer.MAX_VALUE + 1L); + assertSignedNumIncreasingEncodingEquals( + "ffc07ffffffffffffffe", Long.MAX_VALUE - 1); + assertSignedNumIncreasingEncodingEquals( + "ffc07fffffffffffffff", Long.MAX_VALUE); + } + + /** + * Convert a string of hex digits (e.g. "3fbf") to a byte[] + * (e.g. { (byte) 0x3f, (byte) 0xbf }). + */ + private static byte[] bytesFromHexString(String hexDigits) { + return BaseEncoding.base16().lowerCase().decode(hexDigits); + } + + /** + * Assert that decoding (via {@link OrderedCode#readSignedNumIncreasing()}) + * the bytes represented by the specified string of hex digits results in the + * expected long value. + * E.g. assertDecodedSignedNumIncreasingEquals(-65, "3fbf") asserts that the + * byte array { (byte) 0x3f, (byte) 0xbf } is decoded as -65. + */ + private static void assertDecodedSignedNumIncreasingEquals( + long expectedNum, String encodedHexString) { + OrderedCode orderedCode = + new OrderedCode(bytesFromHexString(encodedHexString)); + assertEquals( + "Unexpected value when decoding 0x" + encodedHexString, + expectedNum, + orderedCode.readSignedNumIncreasing()); + assertFalse( + "Unexpected encoded bytes remain after decoding 0x" + encodedHexString, + orderedCode.hasRemainingEncodedBytes()); + } + + /** + * Assert that decoding various sequences of bytes via + * {@link OrderedCode#readSignedNumIncreasing()} produces the expected long + * value. + * Input byte sequences were generated via the c++ (authoritative) + * implementation of OrderedCode::WriteSignedNumIncreasing. + */ + @Test + public void testSignedNumIncreasing_read() { + assertDecodedSignedNumIncreasingEquals( + Long.MIN_VALUE, "003f8000000000000000"); + assertDecodedSignedNumIncreasingEquals( + Long.MIN_VALUE + 1, "003f8000000000000001"); + assertDecodedSignedNumIncreasingEquals( + Integer.MIN_VALUE - 1L, "077fffffff"); + assertDecodedSignedNumIncreasingEquals(Integer.MIN_VALUE, "0780000000"); + assertDecodedSignedNumIncreasingEquals(Integer.MIN_VALUE + 1, "0780000001"); + assertDecodedSignedNumIncreasingEquals(-65, "3fbf"); + assertDecodedSignedNumIncreasingEquals(-64, "40"); + assertDecodedSignedNumIncreasingEquals(-63, "41"); + assertDecodedSignedNumIncreasingEquals(-3, "7d"); + assertDecodedSignedNumIncreasingEquals(-2, "7e"); + assertDecodedSignedNumIncreasingEquals(-1, "7f"); + assertDecodedSignedNumIncreasingEquals(0, "80"); + assertDecodedSignedNumIncreasingEquals(1, "81"); + assertDecodedSignedNumIncreasingEquals(2, "82"); + assertDecodedSignedNumIncreasingEquals(3, "83"); + assertDecodedSignedNumIncreasingEquals(63, "bf"); + assertDecodedSignedNumIncreasingEquals(64, "c040"); + assertDecodedSignedNumIncreasingEquals(65, "c041"); + assertDecodedSignedNumIncreasingEquals(Integer.MAX_VALUE - 1, "f87ffffffe"); + assertDecodedSignedNumIncreasingEquals(Integer.MAX_VALUE, "f87fffffff"); + assertDecodedSignedNumIncreasingEquals( + Integer.MAX_VALUE + 1L, "f880000000"); + assertDecodedSignedNumIncreasingEquals( + Long.MAX_VALUE - 1, "ffc07ffffffffffffffe"); + assertDecodedSignedNumIncreasingEquals( + Long.MAX_VALUE, "ffc07fffffffffffffff"); + } + + /** + * Assert that encoding (via + * {@link OrderedCode#writeSignedNumIncreasing(long)}) the specified long + * value and then decoding (via {@link OrderedCode#readSignedNumIncreasing()}) + * results in the original value. + */ + private static void assertSignedNumIncreasingWriteAndReadIsLossless( + long num) { + OrderedCode orderedCode = new OrderedCode(); + orderedCode.writeSignedNumIncreasing(num); + assertEquals( + "Unexpected result when decoding writeSignedNumIncreasing(" + num + ")", + num, + orderedCode.readSignedNumIncreasing()); + assertFalse("Unexpected remaining encoded bytes after decoding " + num, + orderedCode.hasRemainingEncodedBytes()); + } + + /** + * Assert that for various long values, encoding (via + * {@link OrderedCode#writeSignedNumIncreasing(long)}) and then decoding (via + * {@link OrderedCode#readSignedNumIncreasing()}) results in the original + * value. + */ + @Test + public void testSignedNumIncreasing_writeAndRead() { + assertSignedNumIncreasingWriteAndReadIsLossless(Long.MIN_VALUE); + assertSignedNumIncreasingWriteAndReadIsLossless(Long.MIN_VALUE + 1); + assertSignedNumIncreasingWriteAndReadIsLossless(Integer.MIN_VALUE - 1L); + assertSignedNumIncreasingWriteAndReadIsLossless(Integer.MIN_VALUE); + assertSignedNumIncreasingWriteAndReadIsLossless(Integer.MIN_VALUE + 1); + assertSignedNumIncreasingWriteAndReadIsLossless(-65); + assertSignedNumIncreasingWriteAndReadIsLossless(-64); + assertSignedNumIncreasingWriteAndReadIsLossless(-63); + assertSignedNumIncreasingWriteAndReadIsLossless(-3); + assertSignedNumIncreasingWriteAndReadIsLossless(-2); + assertSignedNumIncreasingWriteAndReadIsLossless(-1); + assertSignedNumIncreasingWriteAndReadIsLossless(0); + assertSignedNumIncreasingWriteAndReadIsLossless(1); + assertSignedNumIncreasingWriteAndReadIsLossless(2); + assertSignedNumIncreasingWriteAndReadIsLossless(3); + assertSignedNumIncreasingWriteAndReadIsLossless(63); + assertSignedNumIncreasingWriteAndReadIsLossless(64); + assertSignedNumIncreasingWriteAndReadIsLossless(65); + assertSignedNumIncreasingWriteAndReadIsLossless(Integer.MAX_VALUE - 1); + assertSignedNumIncreasingWriteAndReadIsLossless(Integer.MAX_VALUE); + assertSignedNumIncreasingWriteAndReadIsLossless(Integer.MAX_VALUE + 1L); + assertSignedNumIncreasingWriteAndReadIsLossless(Long.MAX_VALUE - 1); + assertSignedNumIncreasingWriteAndReadIsLossless(Long.MAX_VALUE); + } + + @Test + public void testLog2Floor_Positive() { + OrderedCode orderedCode = new OrderedCode(); + assertEquals(0, orderedCode.log2Floor(1)); + assertEquals(1, orderedCode.log2Floor(2)); + assertEquals(1, orderedCode.log2Floor(3)); + assertEquals(2, orderedCode.log2Floor(4)); + assertEquals(5, orderedCode.log2Floor(63)); + assertEquals(6, orderedCode.log2Floor(64)); + assertEquals(62, orderedCode.log2Floor(Long.MAX_VALUE)); + } + + /** + * OrderedCode.log2Floor(long) is defined to return -1 given an input of zero. + */ + @Test + public void testLog2Floor_zero() { + OrderedCode orderedCode = new OrderedCode(); + assertEquals(-1, orderedCode.log2Floor(0)); + } + + @Test + public void testLog2Floor_negative() { + OrderedCode orderedCode = new OrderedCode(); + try { + orderedCode.log2Floor(-1); + fail("Expected an IllegalArgumentException."); + } catch (IllegalArgumentException expected) { + // Expected! + } + } + + @Test + public void testGetSignedEncodingLength() { + OrderedCode orderedCode = new OrderedCode(); + assertEquals(10, orderedCode.getSignedEncodingLength(Long.MIN_VALUE)); + assertEquals(10, orderedCode.getSignedEncodingLength(~(1L << 62))); + assertEquals(9, orderedCode.getSignedEncodingLength(~(1L << 62) + 1)); + assertEquals(3, orderedCode.getSignedEncodingLength(-8193)); + assertEquals(2, orderedCode.getSignedEncodingLength(-8192)); + assertEquals(2, orderedCode.getSignedEncodingLength(-65)); + assertEquals(1, orderedCode.getSignedEncodingLength(-64)); + assertEquals(1, orderedCode.getSignedEncodingLength(-2)); + assertEquals(1, orderedCode.getSignedEncodingLength(-1)); + assertEquals(1, orderedCode.getSignedEncodingLength(0)); + assertEquals(1, orderedCode.getSignedEncodingLength(1)); + assertEquals(1, orderedCode.getSignedEncodingLength(63)); + assertEquals(2, orderedCode.getSignedEncodingLength(64)); + assertEquals(2, orderedCode.getSignedEncodingLength(8191)); + assertEquals(3, orderedCode.getSignedEncodingLength(8192)); + assertEquals(9, orderedCode.getSignedEncodingLength((1L << 62)) - 1); + assertEquals(10, orderedCode.getSignedEncodingLength(1L << 62)); + assertEquals(10, orderedCode.getSignedEncodingLength(Long.MAX_VALUE)); + } + + @Test + public void testWriteTrailingBytes() { + byte[] escapeChars = new byte[] { OrderedCode.ESCAPE1, + OrderedCode.NULL_CHARACTER, OrderedCode.SEPARATOR, OrderedCode.ESCAPE2, + OrderedCode.INFINITY, OrderedCode.FF_CHARACTER}; + byte[] anotherArray = new byte[] { 'a', 'b', 'c', 'd', 'e' }; + + OrderedCode orderedCode = new OrderedCode(); + orderedCode.writeTrailingBytes(escapeChars); + assertArrayEquals(orderedCode.getEncodedBytes(), escapeChars); + assertArrayEquals(orderedCode.readTrailingBytes(), escapeChars); + try { + orderedCode.readInfinity(); + fail("Expected IllegalArgumentException."); + } catch (IllegalArgumentException e) { + // expected + } + + orderedCode = new OrderedCode(); + orderedCode.writeTrailingBytes(anotherArray); + assertArrayEquals(orderedCode.getEncodedBytes(), anotherArray); + assertArrayEquals(orderedCode.readTrailingBytes(), anotherArray); + } + + @Test + public void testMixedWrite() { + byte[] first = { 'a', 'b', 'c'}; + byte[] second = { 'd', 'e', 'f'}; + byte[] last = { 'x', 'y', 'z'}; + byte[] escapeChars = new byte[] { OrderedCode.ESCAPE1, + OrderedCode.NULL_CHARACTER, OrderedCode.SEPARATOR, OrderedCode.ESCAPE2, + OrderedCode.INFINITY, OrderedCode.FF_CHARACTER}; + + OrderedCode orderedCode = new OrderedCode(); + orderedCode.writeBytes(first); + orderedCode.writeBytes(second); + orderedCode.writeBytes(last); + orderedCode.writeInfinity(); + orderedCode.writeNumIncreasing(0); + orderedCode.writeNumIncreasing(1); + orderedCode.writeNumIncreasing(Long.MIN_VALUE); + orderedCode.writeNumIncreasing(Long.MAX_VALUE); + orderedCode.writeSignedNumIncreasing(0); + orderedCode.writeSignedNumIncreasing(1); + orderedCode.writeSignedNumIncreasing(Long.MIN_VALUE); + orderedCode.writeSignedNumIncreasing(Long.MAX_VALUE); + orderedCode.writeTrailingBytes(escapeChars); + byte[] allEncoded = orderedCode.getEncodedBytes(); + assertArrayEquals(orderedCode.readBytes(), first); + assertArrayEquals(orderedCode.readBytes(), second); + assertFalse(orderedCode.readInfinity()); + assertArrayEquals(orderedCode.readBytes(), last); + assertTrue(orderedCode.readInfinity()); + assertEquals(orderedCode.readNumIncreasing(), 0); + assertEquals(orderedCode.readNumIncreasing(), 1); + assertFalse(orderedCode.readInfinity()); + assertEquals(orderedCode.readNumIncreasing(), Long.MIN_VALUE); + assertEquals(orderedCode.readNumIncreasing(), Long.MAX_VALUE); + assertEquals(orderedCode.readSignedNumIncreasing(), 0); + assertEquals(orderedCode.readSignedNumIncreasing(), 1); + assertFalse(orderedCode.readInfinity()); + assertEquals(orderedCode.readSignedNumIncreasing(), Long.MIN_VALUE); + assertEquals(orderedCode.readSignedNumIncreasing(), Long.MAX_VALUE); + assertArrayEquals(orderedCode.getEncodedBytes(), escapeChars); + assertArrayEquals(orderedCode.readTrailingBytes(), escapeChars); + + orderedCode = new OrderedCode(allEncoded); + assertArrayEquals(orderedCode.readBytes(), first); + assertArrayEquals(orderedCode.readBytes(), second); + assertFalse(orderedCode.readInfinity()); + assertArrayEquals(orderedCode.readBytes(), last); + assertTrue(orderedCode.readInfinity()); + assertEquals(orderedCode.readNumIncreasing(), 0); + assertEquals(orderedCode.readNumIncreasing(), 1); + assertFalse(orderedCode.readInfinity()); + assertEquals(orderedCode.readNumIncreasing(), Long.MIN_VALUE); + assertEquals(orderedCode.readNumIncreasing(), Long.MAX_VALUE); + assertEquals(orderedCode.readSignedNumIncreasing(), 0); + assertEquals(orderedCode.readSignedNumIncreasing(), 1); + assertFalse(orderedCode.readInfinity()); + assertEquals(orderedCode.readSignedNumIncreasing(), Long.MIN_VALUE); + assertEquals(orderedCode.readSignedNumIncreasing(), Long.MAX_VALUE); + assertArrayEquals(orderedCode.getEncodedBytes(), escapeChars); + assertArrayEquals(orderedCode.readTrailingBytes(), escapeChars); + } + + @Test + public void testEdgeCases() { + byte[] ffChar = {OrderedCode.FF_CHARACTER}; + byte[] nullChar = {OrderedCode.NULL_CHARACTER}; + + byte[] separatorEncoded = {OrderedCode.ESCAPE1, OrderedCode.SEPARATOR}; + byte[] ffCharEncoded = {OrderedCode.ESCAPE1, OrderedCode.NULL_CHARACTER}; + byte[] nullCharEncoded = {OrderedCode.ESCAPE2, OrderedCode.FF_CHARACTER}; + byte[] infinityEncoded = {OrderedCode.ESCAPE2, OrderedCode.INFINITY}; + + OrderedCode orderedCode = new OrderedCode(); + orderedCode.writeBytes(ffChar); + orderedCode.writeBytes(nullChar); + orderedCode.writeInfinity(); + assertArrayEquals(orderedCode.getEncodedBytes(), + Bytes.concat(ffCharEncoded, separatorEncoded, + nullCharEncoded, separatorEncoded, + infinityEncoded)); + assertArrayEquals(orderedCode.readBytes(), ffChar); + assertArrayEquals(orderedCode.readBytes(), nullChar); + assertTrue(orderedCode.readInfinity()); + + orderedCode = new OrderedCode( + Bytes.concat(ffCharEncoded, separatorEncoded)); + assertArrayEquals(orderedCode.readBytes(), ffChar); + + orderedCode = new OrderedCode( + Bytes.concat(nullCharEncoded, separatorEncoded)); + assertArrayEquals(orderedCode.readBytes(), nullChar); + + byte[] invalidEncodingForRead = {OrderedCode.ESCAPE2, OrderedCode.ESCAPE2, + OrderedCode.ESCAPE1, OrderedCode.SEPARATOR}; + orderedCode = new OrderedCode(invalidEncodingForRead); + try { + orderedCode.readBytes(); + fail("Should have failed."); + } catch (Exception e) { + // Expected + } + assertTrue(orderedCode.hasRemainingEncodedBytes()); + } + + @Test + public void testHasRemainingEncodedBytes() { + byte[] bytes = { 'a', 'b', 'c'}; + long number = 12345; + + // Empty + OrderedCode orderedCode = new OrderedCode(); + assertFalse(orderedCode.hasRemainingEncodedBytes()); + + // First and only field of each type. + orderedCode.writeBytes(bytes); + assertTrue(orderedCode.hasRemainingEncodedBytes()); + assertArrayEquals(orderedCode.readBytes(), bytes); + assertFalse(orderedCode.hasRemainingEncodedBytes()); + + orderedCode.writeNumIncreasing(number); + assertTrue(orderedCode.hasRemainingEncodedBytes()); + assertEquals(orderedCode.readNumIncreasing(), number); + assertFalse(orderedCode.hasRemainingEncodedBytes()); + + orderedCode.writeSignedNumIncreasing(number); + assertTrue(orderedCode.hasRemainingEncodedBytes()); + assertEquals(orderedCode.readSignedNumIncreasing(), number); + assertFalse(orderedCode.hasRemainingEncodedBytes()); + + orderedCode.writeInfinity(); + assertTrue(orderedCode.hasRemainingEncodedBytes()); + assertTrue(orderedCode.readInfinity()); + assertFalse(orderedCode.hasRemainingEncodedBytes()); + + orderedCode.writeTrailingBytes(bytes); + assertTrue(orderedCode.hasRemainingEncodedBytes()); + assertArrayEquals(orderedCode.readTrailingBytes(), bytes); + assertFalse(orderedCode.hasRemainingEncodedBytes()); + + // Two fields of same type. + orderedCode.writeBytes(bytes); + orderedCode.writeBytes(bytes); + assertTrue(orderedCode.hasRemainingEncodedBytes()); + assertArrayEquals(orderedCode.readBytes(), bytes); + assertArrayEquals(orderedCode.readBytes(), bytes); + assertFalse(orderedCode.hasRemainingEncodedBytes()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java new file mode 100644 index 0000000000000..05a3864d9bd4d --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java @@ -0,0 +1,125 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.api.services.dataflow.model.MultiOutputInfo; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.util.StringUtils; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.worker.ParDoFn; +import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; + +import org.hamcrest.CoreMatchers; +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Tests for ParDoFnFactory. + */ +@RunWith(JUnit4.class) +public class ParDoFnFactoryTest { + static class TestDoFn extends DoFn { + final String stringState; + final long longState; + + TestDoFn(String stringState, long longState) { + this.stringState = stringState; + this.longState = longState; + } + + @Override + public void processElement(ProcessContext c) { + throw new RuntimeException("not expecting to call this"); + } + } + + @Test + public void testCreateNormalParDoFn() throws Exception { + String stringState = "some state"; + long longState = 42L; + + TestDoFn fn = new TestDoFn(stringState, longState); + + String serializedFn = + StringUtils.byteArrayToJsonString( + SerializableUtils.serializeToByteArray(fn)); + + CloudObject cloudUserFn = CloudObject.forClassName("DoFn"); + addString(cloudUserFn, "serialized_fn", serializedFn); + + String tag = "output"; + MultiOutputInfo multiOutputInfo = new MultiOutputInfo(); + multiOutputInfo.setTag(tag); + List multiOutputInfos = + Arrays.asList(multiOutputInfo); + + BatchModeExecutionContext context = new BatchModeExecutionContext(); + CounterSet counters = new CounterSet(); + StateSampler stateSampler = new StateSampler( + "test", counters.getAddCounterMutator()); + ParDoFn parDoFn = ParDoFnFactory.create( + PipelineOptionsFactory.create(), + cloudUserFn, "name", null, multiOutputInfos, 1, + context, counters.getAddCounterMutator(), stateSampler); + + Assert.assertThat(parDoFn, new IsInstanceOf(NormalParDoFn.class)); + NormalParDoFn normalParDoFn = (NormalParDoFn) parDoFn; + + DoFn actualDoFn = normalParDoFn.fn; + Assert.assertThat(actualDoFn, new IsInstanceOf(TestDoFn.class)); + TestDoFn actualTestDoFn = (TestDoFn) actualDoFn; + + Assert.assertEquals(stringState, actualTestDoFn.stringState); + Assert.assertEquals(longState, actualTestDoFn.longState); + + Assert.assertEquals(context, normalParDoFn.executionContext); + } + + @Test + public void testCreateUnknownParDoFn() throws Exception { + CloudObject cloudUserFn = CloudObject.forClassName("UnknownKindOfDoFn"); + try { + CounterSet counters = new CounterSet(); + StateSampler stateSampler = new StateSampler( + "test", counters.getAddCounterMutator()); + ParDoFnFactory.create(PipelineOptionsFactory.create(), + cloudUserFn, "name", null, null, 1, + new BatchModeExecutionContext(), + counters.getAddCounterMutator(), + stateSampler); + Assert.fail("should have thrown an exception"); + } catch (Exception exn) { + Assert.assertThat(exn.toString(), + CoreMatchers.containsString( + "unable to create a ParDoFn")); + } + } + + // TODO: Test side inputs. +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceTest.java new file mode 100644 index 0000000000000..be8c972c5944a --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceTest.java @@ -0,0 +1,137 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.collect.Lists; + +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * Tests for PartitioningShuffleSource. + */ +@RunWith(JUnit4.class) +public class PartitioningShuffleSourceTest { + static final List>> NO_KVS = Collections.emptyList(); + + static final Instant timestamp = new Instant(123000); + static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); + + static final List>> KVS = Arrays.asList( + WindowedValue.of(KV.of(1, "in 1a"), timestamp, Lists.newArrayList(window)), + WindowedValue.of(KV.of(1, "in 1b"), timestamp, Lists.newArrayList(window)), + WindowedValue.of(KV.of(2, "in 2a"), timestamp, Lists.newArrayList(window)), + WindowedValue.of(KV.of(2, "in 2b"), timestamp, Lists.newArrayList(window)), + WindowedValue.of(KV.of(3, "in 3"), timestamp, Lists.newArrayList(window)), + WindowedValue.of(KV.of(4, "in 4a"), timestamp, Lists.newArrayList(window)), + WindowedValue.of(KV.of(4, "in 4b"), timestamp, Lists.newArrayList(window)), + WindowedValue.of(KV.of(4, "in 4c"), timestamp, Lists.newArrayList(window)), + WindowedValue.of(KV.of(4, "in 4d"), timestamp, Lists.newArrayList(window)), + WindowedValue.of(KV.of(5, "in 5"), timestamp, Lists.newArrayList(window))); + + void runTestReadShuffleSource(List>> expected) + throws Exception { + Coder>> elemCoder = WindowedValue.getFullCoder( + KvCoder.of(BigEndianIntegerCoder.of(), StringUtf8Coder.of()), + IntervalWindow.getCoder()); + + // Write to shuffle with PARTITION_KEYS ShuffleSink. + ShuffleSink> shuffleSink = new ShuffleSink<>( + PipelineOptionsFactory.create(), + null, ShuffleSink.ShuffleKind.PARTITION_KEYS, + elemCoder); + + TestShuffleWriter shuffleWriter = new TestShuffleWriter(); + + List actualSizes = new ArrayList<>(); + try (Sink.SinkWriter>> shuffleSinkWriter = + shuffleSink.writer(shuffleWriter)) { + for (WindowedValue> value : expected) { + actualSizes.add(shuffleSinkWriter.add(value)); + } + } + List records = shuffleWriter.getRecords(); + Assert.assertEquals(expected.size(), records.size()); + Assert.assertEquals(shuffleWriter.getSizes(), actualSizes); + + // Read from shuffle with PartitioningShuffleSource. + PartitioningShuffleSource shuffleSource = + new PartitioningShuffleSource<>( + PipelineOptionsFactory.create(), + null, null, null, + elemCoder); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(shuffleSource); + + TestShuffleReader shuffleReader = new TestShuffleReader(); + List expectedSizes = new ArrayList<>(); + for (ShuffleEntry record : records) { + expectedSizes.add(record.length()); + shuffleReader.addEntry(record); + } + + List>> actual = new ArrayList<>(); + try (Source.SourceIterator>> iter = + shuffleSource.iterator(shuffleReader)) { + while (iter.hasNext()) { + Assert.assertTrue(iter.hasNext()); + actual.add(iter.next()); + } + Assert.assertFalse(iter.hasNext()); + try { + iter.next(); + Assert.fail("should have failed"); + } catch (NoSuchElementException exn) { + // As expected. + } + } + + Assert.assertEquals(expected, actual); + Assert.assertEquals(expectedSizes, observer.getActualSizes()); + } + + @Test + public void testReadEmptyShuffleSource() throws Exception { + runTestReadShuffleSource(NO_KVS); + } + + @Test + public void testReadNonEmptyShuffleSource() throws Exception { + runTestReadShuffleSource(KVS); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkFactoryTest.java new file mode 100644 index 0000000000000..4b8901af34b57 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkFactoryTest.java @@ -0,0 +1,187 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Base64.encodeBase64String; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; + +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for ShuffleSinkFactory. + */ +@RunWith(JUnit4.class) +public class ShuffleSinkFactoryTest { + ShuffleSink runTestCreateShuffleSinkHelper(byte[] shuffleWriterConfig, + String shuffleKind, + CloudObject encoding, + FullWindowedValueCoder coder) + throws Exception { + CloudObject spec = CloudObject.forClassName("ShuffleSink"); + addString(spec, "shuffle_writer_config", encodeBase64String(shuffleWriterConfig)); + addString(spec, "shuffle_kind", shuffleKind); + + com.google.api.services.dataflow.model.Sink cloudSink = + new com.google.api.services.dataflow.model.Sink(); + cloudSink.setSpec(spec); + cloudSink.setCodec(encoding); + + Sink sink = SinkFactory.create(PipelineOptionsFactory.create(), + cloudSink, + new BatchModeExecutionContext()); + Assert.assertThat(sink, new IsInstanceOf(ShuffleSink.class)); + ShuffleSink shuffleSink = (ShuffleSink) sink; + Assert.assertArrayEquals(shuffleWriterConfig, + shuffleSink.shuffleWriterConfig); + Assert.assertEquals(coder, shuffleSink.windowedElemCoder); + return shuffleSink; + } + + void runTestCreateUngroupingShuffleSink(byte[] shuffleWriterConfig, + CloudObject encoding, + FullWindowedValueCoder coder) + throws Exception { + ShuffleSink shuffleSink = runTestCreateShuffleSinkHelper( + shuffleWriterConfig, "ungrouped", encoding, coder); + Assert.assertEquals(ShuffleSink.ShuffleKind.UNGROUPED, + shuffleSink.shuffleKind); + Assert.assertFalse(shuffleSink.shardByKey); + Assert.assertFalse(shuffleSink.groupValues); + Assert.assertFalse(shuffleSink.sortValues); + Assert.assertNull(shuffleSink.keyCoder); + Assert.assertNull(shuffleSink.valueCoder); + Assert.assertNull(shuffleSink.sortKeyCoder); + Assert.assertNull(shuffleSink.sortValueCoder); + } + + void runTestCreatePartitioningShuffleSink(byte[] shuffleWriterConfig, + Coder keyCoder, + Coder valueCoder) + throws Exception { + FullWindowedValueCoder coder = (FullWindowedValueCoder) WindowedValue.getFullCoder( + KvCoder.of(keyCoder, valueCoder), IntervalWindow.getCoder()); + ShuffleSink shuffleSink = runTestCreateShuffleSinkHelper( + shuffleWriterConfig, "partition_keys", coder.asCloudObject(), coder); + Assert.assertEquals(ShuffleSink.ShuffleKind.PARTITION_KEYS, + shuffleSink.shuffleKind); + Assert.assertTrue(shuffleSink.shardByKey); + Assert.assertFalse(shuffleSink.groupValues); + Assert.assertFalse(shuffleSink.sortValues); + Assert.assertEquals(keyCoder, shuffleSink.keyCoder); + Assert.assertEquals(valueCoder, shuffleSink.valueCoder); + Assert.assertEquals(FullWindowedValueCoder.of(valueCoder, + IntervalWindow.getCoder()), + shuffleSink.windowedValueCoder); + Assert.assertNull(shuffleSink.sortKeyCoder); + Assert.assertNull(shuffleSink.sortValueCoder); + } + + void runTestCreateGroupingShuffleSink(byte[] shuffleWriterConfig, + Coder keyCoder, + Coder valueCoder) + throws Exception { + FullWindowedValueCoder coder = (FullWindowedValueCoder) WindowedValue.getFullCoder( + KvCoder.of(keyCoder, valueCoder), IntervalWindow.getCoder()); + ShuffleSink shuffleSink = runTestCreateShuffleSinkHelper( + shuffleWriterConfig, "group_keys", coder.asCloudObject(), coder); + Assert.assertEquals(ShuffleSink.ShuffleKind.GROUP_KEYS, + shuffleSink.shuffleKind); + Assert.assertTrue(shuffleSink.shardByKey); + Assert.assertTrue(shuffleSink.groupValues); + Assert.assertFalse(shuffleSink.sortValues); + Assert.assertEquals(keyCoder, shuffleSink.keyCoder); + Assert.assertEquals(valueCoder, shuffleSink.valueCoder); + Assert.assertNull(shuffleSink.windowedValueCoder); + Assert.assertNull(shuffleSink.sortKeyCoder); + Assert.assertNull(shuffleSink.sortValueCoder); + } + + void runTestCreateGroupingSortingShuffleSink(byte[] shuffleWriterConfig, + Coder keyCoder, + Coder sortKeyCoder, + Coder sortValueCoder) + throws Exception { + FullWindowedValueCoder coder = (FullWindowedValueCoder) WindowedValue.getFullCoder( + KvCoder.of(keyCoder, KvCoder.of(sortKeyCoder, sortValueCoder)), + IntervalWindow.getCoder()); + ShuffleSink shuffleSink = runTestCreateShuffleSinkHelper( + shuffleWriterConfig, "group_keys_and_sort_values", coder.asCloudObject(), coder); + Assert.assertEquals(ShuffleSink.ShuffleKind.GROUP_KEYS_AND_SORT_VALUES, + shuffleSink.shuffleKind); + Assert.assertTrue(shuffleSink.shardByKey); + Assert.assertTrue(shuffleSink.groupValues); + Assert.assertTrue(shuffleSink.sortValues); + Assert.assertEquals(keyCoder, shuffleSink.keyCoder); + Assert.assertEquals(KvCoder.of(sortKeyCoder, sortValueCoder), + shuffleSink.valueCoder); + Assert.assertEquals(sortKeyCoder, shuffleSink.sortKeyCoder); + Assert.assertEquals(sortValueCoder, shuffleSink.sortValueCoder); + Assert.assertNull(shuffleSink.windowedValueCoder); + } + + @Test + public void testCreateUngroupingShuffleSink() throws Exception { + FullWindowedValueCoder coder = (FullWindowedValueCoder) WindowedValue.getFullCoder( + StringUtf8Coder.of(), IntervalWindow.getCoder()); + runTestCreateUngroupingShuffleSink( + new byte[]{(byte) 0xE1}, + coder.asCloudObject(), + coder); + } + + @Test + public void testCreatePartitionShuffleSink() throws Exception { + runTestCreatePartitioningShuffleSink( + new byte[]{(byte) 0xE2}, + BigEndianIntegerCoder.of(), + StringUtf8Coder.of()); + } + + @Test + public void testCreateGroupingShuffleSink() throws Exception { + runTestCreateGroupingShuffleSink( + new byte[]{(byte) 0xE2}, + BigEndianIntegerCoder.of(), + WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder())); + } + + @Test + public void testCreateGroupingSortingShuffleSink() throws Exception { + runTestCreateGroupingSortingShuffleSink( + new byte[]{(byte) 0xE3}, + BigEndianIntegerCoder.of(), + StringUtf8Coder.of(), + VoidCoder.of()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java new file mode 100644 index 0000000000000..3e390b8966afa --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java @@ -0,0 +1,236 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.TestUtils; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.InstantCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink.SinkWriter; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.collect.Lists; + +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * Tests for ShuffleSink. + */ +@RunWith(JUnit4.class) +public class ShuffleSinkTest { + static final List> NO_KVS = Collections.emptyList(); + + static final List> KVS = Arrays.asList( + KV.of(1, "in 1a"), + KV.of(1, "in 1b"), + KV.of(2, "in 2a"), + KV.of(2, "in 2b"), + KV.of(3, "in 3"), + KV.of(4, "in 4a"), + KV.of(4, "in 4b"), + KV.of(4, "in 4c"), + KV.of(4, "in 4d"), + KV.of(5, "in 5")); + + static final List>> NO_SORTING_KVS = + Collections.emptyList(); + + static final List>> SORTING_KVS = + Arrays.asList( + KV.of(1, KV.of("in 1a", 3)), + KV.of(1, KV.of("in 1b", 9)), + KV.of(2, KV.of("in 2a", 2)), + KV.of(2, KV.of("in 2b", 77)), + KV.of(3, KV.of("in 3", 33)), + KV.of(4, KV.of("in 4a", -123)), + KV.of(4, KV.of("in 4b", 0)), + KV.of(4, KV.of("in 4c", -1)), + KV.of(4, KV.of("in 4d", 1)), + KV.of(5, KV.of("in 5", 666))); + + static final Instant timestamp = new Instant(123000); + static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); + + void runTestWriteUngroupingShuffleSink(List expected) + throws Exception { + Coder> windowedValueCoder = + WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), new GlobalWindow().windowCoder()); + ShuffleSink shuffleSink = new ShuffleSink<>( + PipelineOptionsFactory.create(), + null, ShuffleSink.ShuffleKind.UNGROUPED, + windowedValueCoder); + + TestShuffleWriter shuffleWriter = new TestShuffleWriter(); + List actualSizes = new ArrayList<>(); + try (Sink.SinkWriter> shuffleSinkWriter = + shuffleSink.writer(shuffleWriter)) { + for (Integer value : expected) { + actualSizes.add(shuffleSinkWriter.add(WindowedValue.valueInGlobalWindow(value))); + } + } + + List records = shuffleWriter.getRecords(); + + List actual = new ArrayList<>(); + for (ShuffleEntry record : records) { + // Ignore the key. + byte[] valueBytes = record.getValue(); + WindowedValue value = CoderUtils.decodeFromByteArray(windowedValueCoder, valueBytes); + Assert.assertEquals(Lists.newArrayList(GlobalWindow.Window.INSTANCE), value.getWindows()); + actual.add(value.getValue()); + } + + Assert.assertEquals(expected, actual); + Assert.assertEquals(shuffleWriter.getSizes(), actualSizes); + } + + void runTestWriteGroupingShuffleSink( + List> expected) + throws Exception { + ShuffleSink> shuffleSink = new ShuffleSink<>( + PipelineOptionsFactory.create(), + null, ShuffleSink.ShuffleKind.GROUP_KEYS, + WindowedValue.getFullCoder( + KvCoder.of(BigEndianIntegerCoder.of(), StringUtf8Coder.of()), + IntervalWindow.getCoder())); + + TestShuffleWriter shuffleWriter = new TestShuffleWriter(); + List actualSizes = new ArrayList<>(); + try (SinkWriter>> shuffleSinkWriter = + shuffleSink.writer(shuffleWriter)) { + for (KV kv : expected) { + actualSizes.add(shuffleSinkWriter.add( + WindowedValue.of(KV.of(kv.getKey(), kv.getValue()), + timestamp, + Lists.newArrayList(window)))); + } + } + + List records = shuffleWriter.getRecords(); + + List> actual = new ArrayList<>(); + for (ShuffleEntry record : records) { + byte[] keyBytes = record.getKey(); + byte[] valueBytes = record.getValue(); + Assert.assertEquals(timestamp, + CoderUtils.decodeFromByteArray(InstantCoder.of(), record.getSecondaryKey())); + + Integer key = + CoderUtils.decodeFromByteArray(BigEndianIntegerCoder.of(), + keyBytes); + String valueElem = CoderUtils.decodeFromByteArray(StringUtf8Coder.of(), valueBytes); + + actual.add(KV.of(key, valueElem)); + } + + Assert.assertEquals(expected, actual); + Assert.assertEquals(shuffleWriter.getSizes(), actualSizes); + } + + void runTestWriteGroupingSortingShuffleSink( + List>> expected) + throws Exception { + ShuffleSink>> shuffleSink = + new ShuffleSink<>( + PipelineOptionsFactory.create(), + null, + ShuffleSink.ShuffleKind.GROUP_KEYS_AND_SORT_VALUES, + WindowedValue.getFullCoder( + KvCoder.of(BigEndianIntegerCoder.of(), + KvCoder.of(StringUtf8Coder.of(), + BigEndianIntegerCoder.of())), + new GlobalWindow().windowCoder())); + + TestShuffleWriter shuffleWriter = new TestShuffleWriter(); + List actualSizes = new ArrayList<>(); + try (Sink.SinkWriter>>> shuffleSinkWriter = + shuffleSink.writer(shuffleWriter)) { + for (KV> kv : expected) { + actualSizes.add(shuffleSinkWriter.add(WindowedValue.valueInGlobalWindow(kv))); + } + } + + List records = shuffleWriter.getRecords(); + + List>> actual = new ArrayList<>(); + for (ShuffleEntry record : records) { + byte[] keyBytes = record.getKey(); + byte[] valueBytes = record.getValue(); + byte[] sortKeyBytes = record.getSecondaryKey(); + + Integer key = + CoderUtils.decodeFromByteArray(BigEndianIntegerCoder.of(), + keyBytes); + String sortKey = + CoderUtils.decodeFromByteArray(StringUtf8Coder.of(), + sortKeyBytes); + Integer sortValue = CoderUtils.decodeFromByteArray(BigEndianIntegerCoder.of(), valueBytes); + + actual.add(KV.of(key, KV.of(sortKey, sortValue))); + } + + Assert.assertEquals(expected, actual); + Assert.assertEquals(shuffleWriter.getSizes(), actualSizes); + } + + @Test + public void testWriteEmptyUngroupingShuffleSink() throws Exception { + runTestWriteUngroupingShuffleSink(TestUtils.NO_INTS); + } + + @Test + public void testWriteNonEmptyUngroupingShuffleSink() throws Exception { + runTestWriteUngroupingShuffleSink(TestUtils.INTS); + } + + @Test + public void testWriteEmptyGroupingShuffleSink() throws Exception { + runTestWriteGroupingShuffleSink(NO_KVS); + } + + @Test + public void testWriteNonEmptyGroupingShuffleSink() throws Exception { + runTestWriteGroupingShuffleSink(KVS); + } + + @Test + public void testWriteEmptyGroupingSortingShuffleSink() throws Exception { + runTestWriteGroupingSortingShuffleSink(NO_SORTING_KVS); + } + + @Test + public void testWriteNonEmptyGroupingSortingShuffleSink() throws Exception { + runTestWriteGroupingSortingShuffleSink(SORTING_KVS); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSourceFactoryTest.java new file mode 100644 index 0000000000000..75fc7479687e7 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSourceFactoryTest.java @@ -0,0 +1,230 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Base64.encodeBase64String; +import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import javax.annotation.Nullable; + +/** + * Tests for UngroupedShuffleSourceFactory, GroupingShuffleSourceFactory, + * and PartitioningShuffleSourceFactory. + */ +@RunWith(JUnit4.class) +public class ShuffleSourceFactoryTest { + + T runTestCreateShuffleSource(byte[] shuffleReaderConfig, + @Nullable String start, + @Nullable String end, + CloudObject encoding, + BatchModeExecutionContext context, + Class shuffleSourceClass) + throws Exception { + CloudObject spec = CloudObject.forClassName(shuffleSourceClass.getSimpleName()); + addString(spec, "shuffle_reader_config", encodeBase64String(shuffleReaderConfig)); + if (start != null) { + addString(spec, "start_shuffle_position", start); + } + if (end != null) { + addString(spec, "end_shuffle_position", end); + } + + com.google.api.services.dataflow.model.Source cloudSource = + new com.google.api.services.dataflow.model.Source(); + cloudSource.setSpec(spec); + cloudSource.setCodec(encoding); + + Source source = SourceFactory.create( + PipelineOptionsFactory.create(), cloudSource, context); + Assert.assertThat(source, new IsInstanceOf(shuffleSourceClass)); + T shuffleSource = (T) source; + return shuffleSource; + } + + void runTestCreateUngroupedShuffleSource(byte[] shuffleReaderConfig, + @Nullable String start, + @Nullable String end, + CloudObject encoding, + Coder coder) throws Exception { + UngroupedShuffleSource shuffleSource = + runTestCreateShuffleSource(shuffleReaderConfig, + start, + end, + encoding, + new BatchModeExecutionContext(), + UngroupedShuffleSource.class); + Assert.assertArrayEquals(shuffleReaderConfig, + shuffleSource.shuffleReaderConfig); + Assert.assertEquals(start, shuffleSource.startShufflePosition); + Assert.assertEquals(end, shuffleSource.stopShufflePosition); + + Assert.assertEquals(coder, shuffleSource.coder); + } + + void runTestCreateGroupingShuffleSource(byte[] shuffleReaderConfig, + @Nullable String start, + @Nullable String end, + CloudObject encoding, + Coder keyCoder, + Coder valueCoder) throws Exception { + BatchModeExecutionContext context = new BatchModeExecutionContext(); + GroupingShuffleSource shuffleSource = + runTestCreateShuffleSource(shuffleReaderConfig, + start, + end, + encoding, + context, + GroupingShuffleSource.class); + Assert.assertArrayEquals(shuffleReaderConfig, + shuffleSource.shuffleReaderConfig); + Assert.assertEquals(start, shuffleSource.startShufflePosition); + Assert.assertEquals(end, shuffleSource.stopShufflePosition); + + Assert.assertEquals(keyCoder, shuffleSource.keyCoder); + Assert.assertEquals(valueCoder, shuffleSource.valueCoder); + Assert.assertEquals(context, shuffleSource.executionContext); + } + + void runTestCreatePartitioningShuffleSource(byte[] shuffleReaderConfig, + @Nullable String start, + @Nullable String end, + CloudObject encoding, + Coder keyCoder, + Coder windowedValueCoder) throws Exception { + PartitioningShuffleSource shuffleSource = + runTestCreateShuffleSource(shuffleReaderConfig, + start, + end, + encoding, + new BatchModeExecutionContext(), + PartitioningShuffleSource.class); + Assert.assertArrayEquals(shuffleReaderConfig, + shuffleSource.shuffleReaderConfig); + Assert.assertEquals(start, shuffleSource.startShufflePosition); + Assert.assertEquals(end, shuffleSource.stopShufflePosition); + + Assert.assertEquals(keyCoder, shuffleSource.keyCoder); + Assert.assertEquals(windowedValueCoder, shuffleSource.windowedValueCoder); + } + + @Test + public void testCreatePlainUngroupedShuffleSource() throws Exception { + runTestCreateUngroupedShuffleSource( + new byte[]{(byte) 0xE1}, null, null, + makeCloudEncoding("StringUtf8Coder"), + StringUtf8Coder.of()); + } + + @Test + public void testCreateRichUngroupedShuffleSource() throws Exception { + runTestCreateUngroupedShuffleSource( + new byte[]{(byte) 0xE2}, "aaa", "zzz", + makeCloudEncoding("BigEndianIntegerCoder"), + BigEndianIntegerCoder.of()); + } + + @Test + public void testCreatePlainGroupingShuffleSource() throws Exception { + runTestCreateGroupingShuffleSource( + new byte[]{(byte) 0xE1}, null, null, + makeCloudEncoding( + FullWindowedValueCoder.class.getName(), + makeCloudEncoding( + "KvCoder", + makeCloudEncoding("BigEndianIntegerCoder"), + makeCloudEncoding( + "IterableCoder", + makeCloudEncoding("StringUtf8Coder"))), + IntervalWindow.getCoder().asCloudObject()), + BigEndianIntegerCoder.of(), + StringUtf8Coder.of()); + } + + @Test + public void testCreateRichGroupingShuffleSource() throws Exception { + runTestCreateGroupingShuffleSource( + new byte[]{(byte) 0xE2}, "aaa", "zzz", + makeCloudEncoding( + FullWindowedValueCoder.class.getName(), + makeCloudEncoding( + "KvCoder", + makeCloudEncoding("BigEndianIntegerCoder"), + makeCloudEncoding( + "IterableCoder", + makeCloudEncoding( + "KvCoder", + makeCloudEncoding("StringUtf8Coder"), + makeCloudEncoding("VoidCoder")))), + IntervalWindow.getCoder().asCloudObject()), + BigEndianIntegerCoder.of(), + KvCoder.of(StringUtf8Coder.of(), VoidCoder.of())); + } + + @Test + public void testCreatePlainPartitioningShuffleSource() throws Exception { + runTestCreatePartitioningShuffleSource( + new byte[]{(byte) 0xE1}, null, null, + makeCloudEncoding( + FullWindowedValueCoder.class.getName(), + makeCloudEncoding( + "KvCoder", + makeCloudEncoding("BigEndianIntegerCoder"), + makeCloudEncoding("StringUtf8Coder")), + IntervalWindow.getCoder().asCloudObject()), + BigEndianIntegerCoder.of(), + FullWindowedValueCoder.of(StringUtf8Coder.of(), IntervalWindow.getCoder())); + } + + @Test + public void testCreateRichPartitioningShuffleSource() throws Exception { + runTestCreatePartitioningShuffleSource( + new byte[]{(byte) 0xE2}, "aaa", "zzz", + makeCloudEncoding( + FullWindowedValueCoder.class.getName(), + makeCloudEncoding( + "KvCoder", + makeCloudEncoding("BigEndianIntegerCoder"), + makeCloudEncoding( + "KvCoder", + makeCloudEncoding("StringUtf8Coder"), + makeCloudEncoding("VoidCoder"))), + IntervalWindow.getCoder().asCloudObject()), + BigEndianIntegerCoder.of(), + FullWindowedValueCoder.of(KvCoder.of(StringUtf8Coder.of(), VoidCoder.of()), + IntervalWindow.getCoder())); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtilsTest.java new file mode 100644 index 0000000000000..ea879335ec02a --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtilsTest.java @@ -0,0 +1,145 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static org.hamcrest.Matchers.emptyIterable; +import static org.hamcrest.collection.IsIterableContainingInOrder.contains; +import static org.hamcrest.core.Is.is; +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import com.google.api.services.dataflow.model.SideInputInfo; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Tests for SideInputUtils. + */ +@RunWith(JUnit4.class) +public class SideInputUtilsTest { + SideInputInfo createSingletonSideInputInfo( + com.google.api.services.dataflow.model.Source sideInputSource) { + SideInputInfo sideInputInfo = new SideInputInfo(); + sideInputInfo.setSources(Arrays.asList(sideInputSource)); + sideInputInfo.setKind(CloudObject.forClassName("singleton")); + return sideInputInfo; + } + + SideInputInfo createCollectionSideInputInfo( + com.google.api.services.dataflow.model.Source... sideInputSources) { + SideInputInfo sideInputInfo = new SideInputInfo(); + sideInputInfo.setSources(Arrays.asList(sideInputSources)); + sideInputInfo.setKind(CloudObject.forClassName("collection")); + return sideInputInfo; + } + + com.google.api.services.dataflow.model.Source createSideInputSource(Integer... ints) + throws Exception { + return InMemorySourceFactoryTest.createInMemoryCloudSource( + Arrays.asList(ints), + null, null, + BigEndianIntegerCoder.of()); + } + + void assertThatContains(Object actual, Object... expected) { + assertThat(actual, instanceOf(Iterable.class)); + Iterable iter = (Iterable) actual; + if (expected.length == 0) { + assertThat(iter, is(emptyIterable())); + } else { + assertThat(iter, contains(expected)); + } + } + + @Test + public void testReadSingletonSideInput() throws Exception { + SideInputInfo sideInputInfo = + createSingletonSideInputInfo(createSideInputSource(42)); + + assertEquals(42, + SideInputUtils.readSideInput(PipelineOptionsFactory.create(), + sideInputInfo, + new BatchModeExecutionContext())); + } + + @Test + public void testReadEmptyCollectionSideInput() throws Exception { + SideInputInfo sideInputInfo = + createCollectionSideInputInfo(createSideInputSource()); + + assertThatContains( + SideInputUtils.readSideInput(PipelineOptionsFactory.create(), + sideInputInfo, + new BatchModeExecutionContext())); + } + + @Test + public void testReadCollectionSideInput() throws Exception { + SideInputInfo sideInputInfo = + createCollectionSideInputInfo(createSideInputSource(3, 4, 5, 6)); + + assertThatContains( + SideInputUtils.readSideInput(PipelineOptionsFactory.create(), + sideInputInfo, + new BatchModeExecutionContext()), + 3, 4, 5, 6); + } + + @Test + public void testReadCollectionShardedSideInput() throws Exception { + SideInputInfo sideInputInfo = + createCollectionSideInputInfo( + createSideInputSource(3), + createSideInputSource(), + createSideInputSource(4, 5), + createSideInputSource(6), + createSideInputSource()); + + assertThatContains( + SideInputUtils.readSideInput(PipelineOptionsFactory.create(), + sideInputInfo, + new BatchModeExecutionContext()), + 3, 4, 5, 6); + } + + @Test + public void testReadSingletonSideInputValue() throws Exception { + CloudObject sideInputKind = CloudObject.forClassName("singleton"); + Object elem = "hi"; + List elems = Arrays.asList(elem); + assertEquals(elem, + SideInputUtils.readSideInputValue(sideInputKind, elems)); + } + + @Test + public void testReadCollectionSideInputValue() throws Exception { + CloudObject sideInputKind = CloudObject.forClassName("collection"); + List elems = Arrays.asList("hi", "there", "bob"); + assertEquals(elems, + SideInputUtils.readSideInputValue(sideInputKind, elems)); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactoryTest.java new file mode 100644 index 0000000000000..66e72545cb717 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactoryTest.java @@ -0,0 +1,119 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; + +import org.hamcrest.CoreMatchers; +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for SinkFactory. + */ +@RunWith(JUnit4.class) +public class SinkFactoryTest { + static class TestSinkFactory { + public static TestSink create(PipelineOptions options, + CloudObject o, + Coder coder, + ExecutionContext executionContext) { + return new TestSink(); + } + } + + static class TestSink extends Sink { + @Override + public SinkWriter writer() { + return new TestSinkWriter(); + } + + /** A sink writer that drops its input values, for testing. */ + class TestSinkWriter implements SinkWriter { + @Override + public long add(Integer outputElem) { + return 4; + } + + @Override + public void close() { + } + } + } + + @Test + public void testCreatePredefinedSink() throws Exception { + CloudObject spec = CloudObject.forClassName("TextSink"); + addString(spec, "filename", "/path/to/file.txt"); + + com.google.api.services.dataflow.model.Sink cloudSink = + new com.google.api.services.dataflow.model.Sink(); + cloudSink.setSpec(spec); + cloudSink.setCodec(makeCloudEncoding("StringUtf8Coder")); + + Sink sink = SinkFactory.create(PipelineOptionsFactory.create(), + cloudSink, + new BatchModeExecutionContext()); + Assert.assertThat(sink, new IsInstanceOf(TextSink.class)); + } + + @Test + public void testCreateUserDefinedSink() throws Exception { + CloudObject spec = CloudObject.forClass(TestSinkFactory.class); + + com.google.api.services.dataflow.model.Sink cloudSink = + new com.google.api.services.dataflow.model.Sink(); + cloudSink.setSpec(spec); + cloudSink.setCodec(makeCloudEncoding("BigEndianIntegerCoder")); + + Sink sink = SinkFactory.create(PipelineOptionsFactory.create(), + cloudSink, + new BatchModeExecutionContext()); + Assert.assertThat(sink, new IsInstanceOf(TestSink.class)); + } + + @Test + public void testCreateUnknownSink() throws Exception { + CloudObject spec = CloudObject.forClassName("UnknownSink"); + com.google.api.services.dataflow.model.Sink cloudSink = + new com.google.api.services.dataflow.model.Sink(); + cloudSink.setSpec(spec); + cloudSink.setCodec(makeCloudEncoding("StringUtf8Coder")); + try { + SinkFactory.create(PipelineOptionsFactory.create(), + cloudSink, + new BatchModeExecutionContext()); + Assert.fail("should have thrown an exception"); + } catch (Exception exn) { + Assert.assertThat(exn.toString(), + CoreMatchers.containsString( + "unable to create a sink")); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactoryTest.java new file mode 100644 index 0000000000000..4b4665b55869d --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactoryTest.java @@ -0,0 +1,124 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.hamcrest.CoreMatchers; +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.NoSuchElementException; + +/** + * Tests for SourceFactory. + */ +@RunWith(JUnit4.class) +public class SourceFactoryTest { + static class TestSourceFactory { + public static TestSource create(PipelineOptions options, + CloudObject o, + Coder coder, + ExecutionContext executionContext) { + return new TestSource(); + } + } + + static class TestSource extends Source { + @Override + public SourceIterator iterator() { + return new TestSourceIterator(); + } + + /** A source iterator that produces no values, for testing. */ + class TestSourceIterator extends AbstractSourceIterator { + @Override + public boolean hasNext() { return false; } + + @Override + public Integer next() { + throw new NoSuchElementException(); + } + + @Override + public void close() { + } + } + } + + @Test + public void testCreatePredefinedSource() throws Exception { + CloudObject spec = CloudObject.forClassName("TextSource"); + addString(spec, "filename", "/path/to/file.txt"); + + com.google.api.services.dataflow.model.Source cloudSource = + new com.google.api.services.dataflow.model.Source(); + cloudSource.setSpec(spec); + cloudSource.setCodec(makeCloudEncoding("StringUtf8Coder")); + + Source source = SourceFactory.create(PipelineOptionsFactory.create(), + cloudSource, + new BatchModeExecutionContext()); + Assert.assertThat(source, new IsInstanceOf(TextSource.class)); + } + + @Test + public void testCreateUserDefinedSource() throws Exception { + CloudObject spec = CloudObject.forClass(TestSourceFactory.class); + + com.google.api.services.dataflow.model.Source cloudSource = + new com.google.api.services.dataflow.model.Source(); + cloudSource.setSpec(spec); + cloudSource.setCodec(makeCloudEncoding("BigEndianIntegerCoder")); + + Source source = SourceFactory.create(PipelineOptionsFactory.create(), + cloudSource, + new BatchModeExecutionContext()); + Assert.assertThat(source, new IsInstanceOf(TestSource.class)); + } + + @Test + public void testCreateUnknownSource() throws Exception { + CloudObject spec = CloudObject.forClassName("UnknownSource"); + com.google.api.services.dataflow.model.Source cloudSource = + new com.google.api.services.dataflow.model.Source(); + cloudSource.setSpec(spec); + cloudSource.setCodec(makeCloudEncoding("StringUtf8Coder")); + try { + SourceFactory.create(PipelineOptionsFactory.create(), + cloudSource, + new BatchModeExecutionContext()); + Assert.fail("should have thrown an exception"); + } catch (Exception exn) { + Assert.assertThat(exn.toString(), + CoreMatchers.containsString( + "unable to create a source")); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TestShuffleReader.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TestShuffleReader.java new file mode 100644 index 0000000000000..4d5e85881be91 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TestShuffleReader.java @@ -0,0 +1,177 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.util.common.Reiterator; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntryReader; +import com.google.cloud.dataflow.sdk.util.common.worker.ShufflePosition; +// TODO: Decide how we want to handle this Guava dependency. +import com.google.common.primitives.UnsignedBytes; + +import org.junit.Assert; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.NavigableMap; +import java.util.NoSuchElementException; +import java.util.TreeMap; + +/** + * A fake implementation of a ShuffleEntryReader, for testing. + */ +public class TestShuffleReader implements ShuffleEntryReader { + static final Comparator SHUFFLE_KEY_COMPARATOR = + UnsignedBytes.lexicographicalComparator(); + final NavigableMap> records; + + public TestShuffleReader(NavigableMap> records) { + this.records = records; + } + + public TestShuffleReader() { + this(new TreeMap>(SHUFFLE_KEY_COMPARATOR)); + } + + public void addEntry(String key, String value) { + addEntry(key.getBytes(), value.getBytes()); + } + + public void addEntry(byte[] key, byte[] value) { + addEntry(new ShuffleEntry(key, null, value)); + } + + public void addEntry(ShuffleEntry entry) { + List values = records.get(entry.getKey()); + if (values == null) { + values = new ArrayList<>(); + records.put(entry.getKey(), values); + } + values.add(entry); + } + + public Iterator read() { + return read((byte[]) null, (byte[]) null); + } + + @Override + public Reiterator read(ShufflePosition startPosition, + ShufflePosition endPosition) { + return read(ByteArrayShufflePosition.getPosition(startPosition), + ByteArrayShufflePosition.getPosition(endPosition)); + } + + public Reiterator read(String startKey, String endKey) { + return read(startKey == null ? null : startKey.getBytes(), + endKey == null ? null : endKey.getBytes()); + } + + public Reiteratorread(byte[] startKey, byte[] endKey) { + return new ShuffleReaderIterator(startKey, endKey); + } + + class ShuffleReaderIterator implements Reiterator { + final Iterator>> recordsIter; + final byte[] startKey; + final byte[] endKey; + byte[] currentKey; + Map.Entry> currentRecord; + ListIterator currentValuesIter; + + public ShuffleReaderIterator(byte[] startKey, byte[] endKey) { + this.recordsIter = records.entrySet().iterator(); + this.startKey = startKey; + this.endKey = endKey; + advanceKey(); + } + + private ShuffleReaderIterator(ShuffleReaderIterator it) { + if (it.currentKey != null) { + this.recordsIter = + records.tailMap(it.currentKey, false).entrySet().iterator(); + } else { + this.recordsIter = null; + } + this.startKey = it.startKey; + this.endKey = it.endKey; + this.currentKey = it.currentKey; + this.currentRecord = it.currentRecord; + if (it.currentValuesIter != null) { + this.currentValuesIter = + it.currentRecord.getValue().listIterator( + it.currentValuesIter.nextIndex()); + } else { + this.currentValuesIter = null; + } + } + + @Override + public boolean hasNext() { + return currentKey != null; + } + + @Override + public ShuffleEntry next() { + if (currentKey == null) { + throw new NoSuchElementException(); + } + ShuffleEntry resultValue = currentValuesIter.next(); + Assert.assertTrue(Arrays.equals(currentKey, resultValue.getKey())); + if (!currentValuesIter.hasNext()) { + advanceKey(); + } + return resultValue; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public Reiterator copy() { + return new ShuffleReaderIterator(this); + } + + private void advanceKey() { + while (recordsIter.hasNext()) { + currentRecord = recordsIter.next(); + currentKey = currentRecord.getKey(); + if (startKey != null && + SHUFFLE_KEY_COMPARATOR.compare(currentKey, startKey) < 0) { + // This key is before the start of the range. Keep looking. + continue; + } + if (endKey != null && + SHUFFLE_KEY_COMPARATOR.compare(currentKey, endKey) >= 0) { + // This key is at or after the end of the range. Stop looking. + break; + } + // In range. + currentValuesIter = currentRecord.getValue().listIterator(); + return; + } + currentKey = null; + currentValuesIter = null; + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TestShuffleReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TestShuffleReaderTest.java new file mode 100644 index 0000000000000..87935a7bb3d86 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TestShuffleReaderTest.java @@ -0,0 +1,139 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.AbstractMap.SimpleEntry; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; + +/** + * Tests of TestShuffleReader. + */ +@RunWith(JUnit4.class) +public class TestShuffleReaderTest { + static final String START_KEY = "ddd"; + static final String END_KEY = "mmm"; + + static final List> NO_ENTRIES = + Collections.emptyList(); + + static final List> IN_RANGE_ENTRIES = + Arrays.>asList( + new SimpleEntry<>("ddd", "in 1"), + new SimpleEntry<>("ddd", "in 1"), + new SimpleEntry<>("ddd", "in 1"), + new SimpleEntry<>("dddd", "in 2"), + new SimpleEntry<>("dddd", "in 2"), + new SimpleEntry<>("de", "in 3"), + new SimpleEntry<>("ee", "in 4"), + new SimpleEntry<>("ee", "in 4"), + new SimpleEntry<>("ee", "in 4"), + new SimpleEntry<>("ee", "in 4"), + new SimpleEntry<>("mm", "in 5")); + static final List> BEFORE_RANGE_ENTRIES = + Arrays.>asList( + new SimpleEntry<>("", "out 1"), + new SimpleEntry<>("dd", "out 2")); + static final List> AFTER_RANGE_ENTRIES = + Arrays.>asList( + new SimpleEntry<>("mmm", "out 3"), + new SimpleEntry<>("mmm", "out 3"), + new SimpleEntry<>("mmmm", "out 4"), + new SimpleEntry<>("mn", "out 5"), + new SimpleEntry<>("zzz", "out 6")); + static final List> OUT_OF_RANGE_ENTRIES = + new ArrayList<>(); + static { + OUT_OF_RANGE_ENTRIES.addAll(BEFORE_RANGE_ENTRIES); + OUT_OF_RANGE_ENTRIES.addAll(AFTER_RANGE_ENTRIES); + } + static final List> ALL_ENTRIES = new ArrayList<>(); + static { + ALL_ENTRIES.addAll(BEFORE_RANGE_ENTRIES); + ALL_ENTRIES.addAll(IN_RANGE_ENTRIES); + ALL_ENTRIES.addAll(AFTER_RANGE_ENTRIES); + } + + void runTest(List> expected, + List> outOfRange, + String startKey, + String endKey) { + TestShuffleReader shuffleReader = new TestShuffleReader(); + List> expectedCopy = new ArrayList<>(expected); + expectedCopy.addAll(outOfRange); + Collections.shuffle(expectedCopy); + for (Map.Entry entry : expectedCopy) { + shuffleReader.addEntry(entry.getKey(), entry.getValue()); + } + Iterator iter = shuffleReader.read(startKey, endKey); + List> actual = new ArrayList<>(); + while (iter.hasNext()) { + ShuffleEntry entry = iter.next(); + actual.add(new SimpleEntry<>(new String(entry.getKey()), + new String(entry.getValue()))); + } + try { + iter.next(); + Assert.fail("should have failed"); + } catch (NoSuchElementException exn) { + // Success. + } + Assert.assertEquals(expected, actual); + } + + @Test + public void testEmpty() { + runTest(NO_ENTRIES, NO_ENTRIES, null, null); + } + + @Test + public void testEmptyWithRange() { + runTest(NO_ENTRIES, NO_ENTRIES, START_KEY, END_KEY); + } + + @Test + public void testNonEmpty() { + runTest(ALL_ENTRIES, NO_ENTRIES, null, null); + } + + @Test + public void testNonEmptyWithAllInRange() { + runTest(IN_RANGE_ENTRIES, NO_ENTRIES, START_KEY, END_KEY); + } + + @Test + public void testNonEmptyWithSomeOutOfRange() { + runTest(IN_RANGE_ENTRIES, OUT_OF_RANGE_ENTRIES, START_KEY, END_KEY); + } + + @Test + public void testNonEmptyWithAllOutOfRange() { + runTest(NO_ENTRIES, OUT_OF_RANGE_ENTRIES, START_KEY, END_KEY); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TestShuffleWriter.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TestShuffleWriter.java new file mode 100644 index 0000000000000..4fde0bbcdbaa1 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TestShuffleWriter.java @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; + +import java.util.ArrayList; +import java.util.List; + +/** + * A fake implementation of a ShuffleEntryWriter, for testing. + */ +public class TestShuffleWriter implements ShuffleEntryWriter { + final List records = new ArrayList<>(); + final List sizes = new ArrayList<>(); + boolean closed = false; + + public TestShuffleWriter() { } + + @Override + public long put(ShuffleEntry entry) { + if (closed) { + throw new AssertionError("shuffle writer already closed"); + } + records.add(entry); + + long size = entry.length(); + sizes.add(size); + return size; + } + + @Override + public void close() { + if (closed) { + throw new AssertionError("shuffle writer already closed"); + } + closed = true; + } + + /** Returns the key/value records that were written to this ShuffleWriter. */ + public List getRecords() { + if (!closed) { + throw new AssertionError("shuffle writer not closed"); + } + return records; + } + + /** Returns the sizes in bytes of the records that were written to this ShuffleWriter. */ + public List getSizes() { + if (!closed) { + throw new AssertionError("shuffle writer not closed"); + } + return sizes; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkFactoryTest.java new file mode 100644 index 0000000000000..9f9e63090a6e5 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkFactoryTest.java @@ -0,0 +1,98 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; +import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.TextualIntegerCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; + +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import javax.annotation.Nullable; + +/** + * Tests for TextSinkFactory. + */ +@RunWith(JUnit4.class) +public class TextSinkFactoryTest { + void runTestCreateTextSink(String filename, + @Nullable Boolean appendTrailingNewlines, + @Nullable String header, + @Nullable String footer, + CloudObject encoding, + Coder coder) + throws Exception { + CloudObject spec = CloudObject.forClassName("TextSink"); + addString(spec, PropertyNames.FILENAME, filename); + if (appendTrailingNewlines != null) { + addBoolean(spec, PropertyNames.APPEND_TRAILING_NEWLINES, appendTrailingNewlines); + } + if (header != null) { + addString(spec, PropertyNames.HEADER, header); + } + if (footer != null) { + addString(spec, PropertyNames.FOOTER, footer); + } + + com.google.api.services.dataflow.model.Sink cloudSink = + new com.google.api.services.dataflow.model.Sink(); + cloudSink.setSpec(spec); + cloudSink.setCodec(encoding); + + Sink sink = SinkFactory.create(PipelineOptionsFactory.create(), + cloudSink, + new BatchModeExecutionContext()); + Assert.assertThat(sink, new IsInstanceOf(TextSink.class)); + TextSink textSink = (TextSink) sink; + Assert.assertEquals(filename, textSink.namePrefix); + Assert.assertEquals( + appendTrailingNewlines == null ? true : appendTrailingNewlines, + textSink.appendTrailingNewlines); + Assert.assertEquals(header, textSink.header); + Assert.assertEquals(footer, textSink.footer); + Assert.assertEquals(coder, textSink.coder); + } + + @Test + public void testCreatePlainTextSink() throws Exception { + runTestCreateTextSink( + "/path/to/file.txt", null, null, null, + makeCloudEncoding("StringUtf8Coder"), + StringUtf8Coder.of()); + } + + @Test + public void testCreateRichTextSink() throws Exception { + runTestCreateTextSink( + "gs://bucket/path/to/file2.txt", false, "$$$", "***", + makeCloudEncoding("TextualIntegerCoder"), + TextualIntegerCoder.of()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkTest.java new file mode 100644 index 0000000000000..d1b8b436a2510 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkTest.java @@ -0,0 +1,144 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.TestUtils; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.TextualIntegerCoder; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * Tests for TextSink. + */ +@RunWith(JUnit4.class) +public class TextSinkTest { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + void runTestWriteFile(List elems, + @Nullable String header, + @Nullable String footer, + Coder coder) throws Exception { + File tmpFile = tmpFolder.newFile("file.txt"); + TextSink> textSink = TextSink.createForTest( + tmpFile.getPath(), true, header, footer, coder); + List expected = new ArrayList<>(); + List actualSizes = new ArrayList<>(); + if (header != null) { + expected.add(header); + } + try (Sink.SinkWriter> writer = textSink.writer()) { + for (T elem : elems) { + actualSizes.add((int) writer.add(WindowedValue.valueInGlobalWindow(elem))); + byte[] encodedElem = CoderUtils.encodeToByteArray(coder, elem); + String line = new String(encodedElem); + expected.add(line); + } + } + if (footer != null) { + expected.add(footer); + } + + BufferedReader reader = new BufferedReader(new FileReader(tmpFile)); + List actual = new ArrayList<>(); + List expectedSizes = new ArrayList<>(); + for (;;) { + String line = reader.readLine(); + if (line == null) { + break; + } + actual.add(line); + expectedSizes.add(line.length() + TextSink.NEWLINE.length); + } + if (header != null) { + expectedSizes.remove(0); + } + if (footer != null) { + expectedSizes.remove(expectedSizes.size() - 1); + } + + Assert.assertEquals(expected, actual); + Assert.assertEquals(expectedSizes, actualSizes); + } + + @Test + public void testWriteEmptyFile() throws Exception { + runTestWriteFile(Collections.emptyList(), null, null, + StringUtf8Coder.of()); + } + + @Test + public void testWriteEmptyFileWithHeaderAndFooter() throws Exception { + runTestWriteFile(Collections.emptyList(), "the head", "the foot", + StringUtf8Coder.of()); + } + + @Test + public void testWriteNonEmptyFile() throws Exception { + List lines = Arrays.asList( + "", + " hi there ", + "bob", + "", + " ", + "--zowie!--", + ""); + runTestWriteFile(lines, null, null, StringUtf8Coder.of()); + } + + @Test + public void testWriteNonEmptyFileWithHeaderAndFooter() throws Exception { + List lines = Arrays.asList( + "", + " hi there ", + "bob", + "", + " ", + "--zowie!--", + ""); + runTestWriteFile(lines, "the head", "the foot", StringUtf8Coder.of()); + } + + @Test + public void testWriteNonEmptyNonStringFile() throws Exception { + runTestWriteFile(TestUtils.INTS, null, null, TextualIntegerCoder.of()); + } + + // TODO: sharded filenames + // TODO: not appending newlines + // TODO: writing to GCS +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactoryTest.java new file mode 100644 index 0000000000000..2fa50b567e781 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactoryTest.java @@ -0,0 +1,98 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; +import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean; +import static com.google.cloud.dataflow.sdk.util.Structs.addLong; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.TextualIntegerCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import javax.annotation.Nullable; + +/** + * Tests for TextSourceFactory. + */ +@RunWith(JUnit4.class) +public class TextSourceFactoryTest { + void runTestCreateTextSource(String filename, + @Nullable Boolean stripTrailingNewlines, + @Nullable Long start, + @Nullable Long end, + CloudObject encoding, + Coder coder) + throws Exception { + CloudObject spec = CloudObject.forClassName("TextSource"); + addString(spec, "filename", filename); + if (stripTrailingNewlines != null) { + addBoolean(spec, "strip_trailing_newlines", stripTrailingNewlines); + } + if (start != null) { + addLong(spec, "start_offset", start); + } + if (end != null) { + addLong(spec, "end_offset", end); + } + + com.google.api.services.dataflow.model.Source cloudSource = + new com.google.api.services.dataflow.model.Source(); + cloudSource.setSpec(spec); + cloudSource.setCodec(encoding); + + Source source = SourceFactory.create(PipelineOptionsFactory.create(), + cloudSource, + new BatchModeExecutionContext()); + Assert.assertThat(source, new IsInstanceOf(TextSource.class)); + TextSource textSource = (TextSource) source; + Assert.assertEquals(filename, textSource.filename); + Assert.assertEquals( + stripTrailingNewlines == null ? true : stripTrailingNewlines, + textSource.stripTrailingNewlines); + Assert.assertEquals(start, textSource.startPosition); + Assert.assertEquals(end, textSource.endPosition); + Assert.assertEquals(coder, textSource.coder); + } + + @Test + public void testCreatePlainTextSource() throws Exception { + runTestCreateTextSource( + "/path/to/file.txt", null, null, null, + makeCloudEncoding("StringUtf8Coder"), + StringUtf8Coder.of()); + } + + @Test + public void testCreateRichTextSource() throws Exception { + runTestCreateTextSource( + "gs://bucket/path/to/file2.txt", false, 200L, 500L, + makeCloudEncoding("TextualIntegerCoder"), + TextualIntegerCoder.of()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceTest.java new file mode 100644 index 0000000000000..8aee7aaf00529 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceTest.java @@ -0,0 +1,581 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static org.hamcrest.Matchers.greaterThan; + +import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.cloud.dataflow.sdk.TestUtils; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.TextualIntegerCoder; +import com.google.cloud.dataflow.sdk.runners.worker.TextSource.TextFileIterator; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; + +/** + * Tests for TextSource. + */ +@RunWith(JUnit4.class) +public class TextSourceTest { + private static final String[] fileContent = {"First line\n", + "Second line\r\n", + "Third line"}; + private static final long TOTAL_BYTES_COUNT; + + static { + long sumLen = 0L; + for (String s : fileContent) { + sumLen += s.length(); + } + TOTAL_BYTES_COUNT = sumLen; + } + + @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); + + private File initTestFile() throws IOException { + File tmpFile = tmpFolder.newFile(); + FileOutputStream output = new FileOutputStream(tmpFile); + for (String s : fileContent) { + output.write(s.getBytes()); + } + output.close(); + + return tmpFile; + } + + @Test + public void testReadEmptyFile() throws Exception { + TextSource textSource = new TextSource<>( + "/dev/null", true, null, null, StringUtf8Coder.of()); + try (Source.SourceIterator iterator = textSource.iterator()) { + Assert.assertFalse(iterator.hasNext()); + } + } + + @Test + public void testStrippedNewlines() throws Exception { + testNewlineHandling("\r", true); + testNewlineHandling("\r\n", true); + testNewlineHandling("\n", true); + } + + @Test + public void testStrippedNewlinesAtEndOfReadBuffer() throws Exception { + boolean stripNewLines = true; + StringBuilder payload = new StringBuilder(); + for (int i = 0; i < TextSource.BUF_SIZE - 2; ++i) { + payload.append('a'); + } + String[] lines = {payload.toString(), payload.toString()}; + testStringPayload(lines , "\r", stripNewLines); + testStringPayload(lines , "\r\n", stripNewLines); + testStringPayload(lines , "\n", stripNewLines); + } + + @Test + public void testUnstrippedNewlines() throws Exception { + testNewlineHandling("\r", false); + testNewlineHandling("\r\n", false); + testNewlineHandling("\n", false); + } + + @Test + public void testUnstrippedNewlinesAtEndOfReadBuffer() throws Exception { + boolean stripNewLines = false; + StringBuilder payload = new StringBuilder(); + for (int i = 0; i < TextSource.BUF_SIZE - 2; ++i) { + payload.append('a'); + } + String[] lines = {payload.toString(), payload.toString()}; + testStringPayload(lines , "\r", stripNewLines); + testStringPayload(lines , "\r\n", stripNewLines); + testStringPayload(lines , "\n", stripNewLines); + } + + @Test + public void testStartPosition() throws Exception { + File tmpFile = initTestFile(); + + { + TextSource textSource = new TextSource<>( + tmpFile.getPath(), false, 11L, null, StringUtf8Coder.of()); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(textSource); + + try (Source.SourceIterator iterator = textSource.iterator()) { + Assert.assertEquals("Second line\r\n", iterator.next()); + Assert.assertEquals("Third line", iterator.next()); + Assert.assertFalse(iterator.hasNext()); + // The first '1' in the array represents the reading of '\n' between first and + // second line, to confirm that we are reading from the beginning of a record. + Assert.assertEquals(Arrays.asList(1, 13, 10), observer.getActualSizes()); + } + } + + { + TextSource textSource = new TextSource<>( + tmpFile.getPath(), false, 20L, null, StringUtf8Coder.of()); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(textSource); + + try (Source.SourceIterator iterator = textSource.iterator()) { + Assert.assertEquals("Third line", iterator.next()); + Assert.assertFalse(iterator.hasNext()); + // The first '5' in the array represents the reading of a portion of the second + // line, which had to be read to find the beginning of the third line. + Assert.assertEquals(Arrays.asList(5, 10), observer.getActualSizes()); + } + } + + { + TextSource textSource = new TextSource<>( + tmpFile.getPath(), true, 0L, 20L, StringUtf8Coder.of()); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(textSource); + + try (Source.SourceIterator iterator = textSource.iterator()) { + Assert.assertEquals("First line", iterator.next()); + Assert.assertEquals("Second line", iterator.next()); + Assert.assertFalse(iterator.hasNext()); + Assert.assertEquals(Arrays.asList(11, 13), observer.getActualSizes()); + } + } + + { + TextSource textSource = new TextSource<>( + tmpFile.getPath(), true, 1L, 20L, StringUtf8Coder.of()); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(textSource); + + try (Source.SourceIterator iterator = textSource.iterator()) { + Assert.assertEquals("Second line", iterator.next()); + Assert.assertFalse(iterator.hasNext()); + // The first '11' in the array represents the reading of the entire first + // line, which had to be read to find the beginning of the second line. + Assert.assertEquals(Arrays.asList(11, 13), observer.getActualSizes()); + } + } + } + + @Test + public void testUtf8Handling() throws Exception { + File tmpFile = tmpFolder.newFile(); + FileOutputStream output = new FileOutputStream(tmpFile); + // first line: €\n + // second line: ¢\n + output.write(new byte[]{(byte) 0xE2, (byte) 0x82, (byte) 0xAC, '\n', + (byte) 0xC2, (byte) 0xA2, '\n'}); + output.close(); + + { + // 3L is after the first line if counting codepoints, but within + // the first line if counting chars. So correct behavior is to return + // just one line, since offsets are in chars, not codepoints. + TextSource textSource = new TextSource<>( + tmpFile.getPath(), true, 0L, 3L, StringUtf8Coder.of()); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(textSource); + + try (Source.SourceIterator iterator = textSource.iterator()) { + Assert.assertArrayEquals("€".getBytes("UTF-8"), + iterator.next().getBytes("UTF-8")); + Assert.assertFalse(iterator.hasNext()); + Assert.assertEquals(Arrays.asList(4), observer.getActualSizes()); + } + } + + { + // Starting location is mid-way into a codepoint. + // Ensures we don't fail when skipping over an incomplete codepoint. + TextSource textSource = new TextSource<>( + tmpFile.getPath(), true, 2L, null, StringUtf8Coder.of()); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(textSource); + + try (Source.SourceIterator iterator = textSource.iterator()) { + Assert.assertArrayEquals("¢".getBytes("UTF-8"), + iterator.next().getBytes("UTF-8")); + Assert.assertFalse(iterator.hasNext()); + // The first '3' in the array represents the reading of a portion of the first + // line, which had to be read to find the beginning of the second line. + Assert.assertEquals(Arrays.asList(3, 3), observer.getActualSizes()); + } + } + } + + private void testNewlineHandling(String separator, boolean stripNewlines) + throws Exception { + File tmpFile = tmpFolder.newFile(); + PrintStream writer = + new PrintStream( + new FileOutputStream(tmpFile)); + List expected = Arrays.asList( + "", + " hi there ", + "bob", + "", + " ", + "--zowie!--", + ""); + List expectedSizes = new ArrayList<>(); + for (String line : expected) { + writer.print(line); + writer.print(separator); + expectedSizes.add(line.length() + separator.length()); + } + writer.close(); + + TextSource textSource = new TextSource<>( + tmpFile.getPath(), stripNewlines, null, null, StringUtf8Coder.of()); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(textSource); + + List actual = new ArrayList<>(); + try (Source.SourceIterator iterator = textSource.iterator()) { + while (iterator.hasNext()) { + actual.add(iterator.next()); + } + } + + if (stripNewlines) { + Assert.assertEquals(expected, actual); + } else { + List unstripped = new LinkedList<>(); + for (String s : expected) { + unstripped.add(s + separator); + } + Assert.assertEquals(unstripped, actual); + } + + Assert.assertEquals(expectedSizes, observer.getActualSizes()); + } + + private void testStringPayload( + String[] lines, String separator, boolean stripNewlines) + throws Exception { + File tmpFile = tmpFolder.newFile(); + List expected = new ArrayList<>(); + PrintStream writer = + new PrintStream( + new FileOutputStream(tmpFile)); + for (String line : lines) { + writer.print(line); + writer.print(separator); + expected.add(stripNewlines ? line : line + separator); + } + writer.close(); + + TextSource textSource = new TextSource<>( + tmpFile.getPath(), stripNewlines, null, null, StringUtf8Coder.of()); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(textSource); + + List actual = new ArrayList<>(); + try (Source.SourceIterator iterator = textSource.iterator()) { + while (iterator.hasNext()) { + actual.add(iterator.next()); + } + } + Assert.assertEquals(expected, actual); + } + + @Test + public void testCloneIteratorWithEndPositionAndFinalBytesInBuffer() + throws Exception { + String line = "a\n"; + boolean stripNewlines = false; + File tmpFile = tmpFolder.newFile(); + List expected = new ArrayList<>(); + PrintStream writer = new PrintStream(new FileOutputStream(tmpFile)); + // Write 5x the size of the buffer and 10 extra trailing bytes + for (long bytesWritten = 0; + bytesWritten < TextSource.BUF_SIZE * 3 + 10; ) { + writer.print(line); + expected.add(line); + bytesWritten += line.length(); + } + writer.close(); + Long fileSize = tmpFile.length(); + + TextSource textSource = new TextSource<>( + tmpFile.getPath(), stripNewlines, + null, fileSize, StringUtf8Coder.of()); + + List actual = new ArrayList<>(); + Source.SourceIterator iterator = textSource.iterator(); + while (iterator.hasNext()) { + actual.add(iterator.next()); + iterator = iterator.copy(); + } + Assert.assertEquals(expected, actual); + } + + @Test + public void testNonStringCoders() throws Exception { + File tmpFile = tmpFolder.newFile(); + PrintStream writer = + new PrintStream( + new FileOutputStream(tmpFile)); + List expected = TestUtils.INTS; + List expectedSizes = new ArrayList<>(); + for (Integer elem : expected) { + byte[] encodedElem = + CoderUtils.encodeToByteArray(TextualIntegerCoder.of(), elem); + writer.print(elem); + writer.print("\n"); + expectedSizes.add(1 + encodedElem.length); + } + writer.close(); + + TextSource textSource = new TextSource<>( + tmpFile.getPath(), true, null, null, TextualIntegerCoder.of()); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(textSource); + + List actual = new ArrayList<>(); + try (Source.SourceIterator iterator = textSource.iterator()) { + while (iterator.hasNext()) { + actual.add(iterator.next()); + } + } + + Assert.assertEquals(expected, actual); + Assert.assertEquals(expectedSizes, observer.getActualSizes()); + } + + @Test + public void testGetApproximatePosition() throws Exception { + File tmpFile = initTestFile(); + TextSource textSource = new TextSource<>( + tmpFile.getPath(), false, 0L, null, StringUtf8Coder.of()); + + try (Source.SourceIterator iterator = textSource.iterator()) { + ApproximateProgress progress = + sourceProgressToCloudProgress(iterator.getProgress()); + Assert.assertEquals(0L, + progress.getPosition().getByteOffset().longValue()); + iterator.next(); + progress = sourceProgressToCloudProgress(iterator.getProgress()); + Assert.assertEquals(11L, + progress.getPosition().getByteOffset().longValue()); + iterator.next(); + progress = sourceProgressToCloudProgress(iterator.getProgress()); + Assert.assertEquals(24L, + progress.getPosition().getByteOffset().longValue()); + iterator.next(); + progress = sourceProgressToCloudProgress(iterator.getProgress()); + Assert.assertEquals(34L, + progress.getPosition().getByteOffset().longValue()); + Assert.assertFalse(iterator.hasNext()); + } + } + + @Test + public void testUpdateStopPosition() throws Exception { + final long end = 10L; // in the first line + final long stop = 14L; // in the middle of the second line + File tmpFile = initTestFile(); + + com.google.api.services.dataflow.model.Position proposedStopPosition = + new com.google.api.services.dataflow.model.Position(); + + // Illegal proposed stop position, no update. + { + TextSource textSource = new TextSource<>( + tmpFile.getPath(), false, null, null, + StringUtf8Coder.of()); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(textSource); + + try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + Assert.assertNull(iterator.updateStopPosition( + cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + } + } + + proposedStopPosition.setByteOffset(stop); + + // Successful update. + { + TextSource textSource = new TextSource<>( + tmpFile.getPath(), false, null, null, + StringUtf8Coder.of()); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(textSource); + + try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + Assert.assertNull(iterator.getEndOffset()); + Assert.assertEquals( + stop, + sourcePositionToCloudPosition( + iterator.updateStopPosition( + cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))) + .getByteOffset().longValue()); + Assert.assertEquals(stop, iterator.getEndOffset().longValue()); + Assert.assertEquals(fileContent[0], iterator.next()); + Assert.assertEquals(fileContent[1], iterator.next()); + Assert.assertFalse(iterator.hasNext()); + Assert.assertEquals(Arrays.asList(fileContent[0].length(), + fileContent[1].length()), + observer.getActualSizes()); + } + } + + // Proposed stop position is before the current position, no update. + { + TextSource textSource = new TextSource<>( + tmpFile.getPath(), false, null, null, + StringUtf8Coder.of()); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(textSource); + + try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + Assert.assertEquals(fileContent[0], iterator.next()); + Assert.assertEquals(fileContent[1], iterator.next()); + Assert.assertThat(sourceProgressToCloudProgress(iterator.getProgress()) + .getPosition().getByteOffset(), + greaterThan(stop)); + Assert.assertNull(iterator.updateStopPosition( + cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + Assert.assertNull(iterator.getEndOffset()); + Assert.assertTrue(iterator.hasNext()); + Assert.assertEquals(fileContent[2], iterator.next()); + Assert.assertEquals(Arrays.asList(fileContent[0].length(), + fileContent[1].length(), + fileContent[2].length()), + observer.getActualSizes()); + } + } + + // Proposed stop position is after the current stop (end) position, no update. + { + TextSource textSource = new TextSource<>( + tmpFile.getPath(), false, null, end, StringUtf8Coder.of()); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(textSource); + + try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + Assert.assertEquals(fileContent[0], iterator.next()); + Assert.assertNull(iterator.updateStopPosition( + cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + Assert.assertEquals(end, iterator.getEndOffset().longValue()); + Assert.assertFalse(iterator.hasNext()); + Assert.assertEquals(Arrays.asList(fileContent[0].length()), + observer.getActualSizes()); + } + } + } + + @Test + public void testUpdateStopPositionExhaustive() throws Exception { + File tmpFile = initTestFile(); + + // Checks for every possible position in the file, that either we fail to + // "updateStop" at it, or we succeed and then reading both halves together + // yields the original file with no missed records or duplicates. + for (long start = 0; start < TOTAL_BYTES_COUNT - 1; start++) { + for (long end = start + 1; end < TOTAL_BYTES_COUNT; end++) { + for (long stop = start; stop <= end; stop++) { + stopPositionTestInternal(start, end, + stop, tmpFile); + } + } + } + + // Test with null start/end positions. + for (long stop = 0L; stop < TOTAL_BYTES_COUNT; stop++) { + stopPositionTestInternal(null, null, stop, tmpFile); + } + } + + private void stopPositionTestInternal(Long startOffset, + Long endOffset, + Long stopOffset, + File tmpFile) throws Exception { + String readWithoutSplit; + String readWithSplit1, readWithSplit2; + StringBuilder accumulatedRead = new StringBuilder(); + + // Read from source without split attempts. + TextSource textSource = new TextSource<>( + tmpFile.getPath(), false, startOffset, endOffset, + StringUtf8Coder.of()); + + try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + while (iterator.hasNext()) { + accumulatedRead.append((String) iterator.next()); + } + readWithoutSplit = accumulatedRead.toString(); + } + + // Read the first half of the split. + textSource = new TextSource<>( + tmpFile.getPath(), false, startOffset, stopOffset, + StringUtf8Coder.of()); + accumulatedRead = new StringBuilder(); + + try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + while (iterator.hasNext()) { + accumulatedRead.append((String) iterator.next()); + } + readWithSplit1 = accumulatedRead.toString(); + } + + // Read the second half of the split. + textSource = new TextSource<>( + tmpFile.getPath(), false, stopOffset, endOffset, + StringUtf8Coder.of()); + accumulatedRead = new StringBuilder(); + + try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + while (iterator.hasNext()) { + accumulatedRead.append((String) iterator.next()); + } + readWithSplit2 = accumulatedRead.toString(); + } + + Assert.assertEquals(readWithoutSplit, readWithSplit1 + readWithSplit2); + } + + private ApproximateProgress createApproximateProgress( + com.google.api.services.dataflow.model.Position position) { + return new ApproximateProgress().setPosition(position); + } + + // TODO: sharded filenames + // TODO: reading from GCS +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java new file mode 100644 index 0000000000000..3a360d8d24add --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java @@ -0,0 +1,112 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.TestUtils; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.common.collect.Lists; + +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * Tests for UngroupedShuffleSource. + */ +@RunWith(JUnit4.class) +public class UngroupedShuffleSourceTest { + static final Instant timestamp = new Instant(123000); + static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); + + byte[] asShuffleKey(long seqNum) throws Exception { + return CoderUtils.encodeToByteArray(BigEndianLongCoder.of(), seqNum); + } + + byte[] asShuffleValue(Integer value) throws Exception { + return CoderUtils.encodeToByteArray( + WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), IntervalWindow.getCoder()), + WindowedValue.of(value, timestamp, Lists.newArrayList(window))); + } + + void runTestReadShuffleSource(List expected) throws Exception { + UngroupedShuffleSource> shuffleSource = + new UngroupedShuffleSource<>( + PipelineOptionsFactory.create(), + null, null, null, + WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), IntervalWindow.getCoder())); + ExecutorTestUtils.TestSourceObserver observer = + new ExecutorTestUtils.TestSourceObserver(shuffleSource); + + TestShuffleReader shuffleReader = new TestShuffleReader(); + List expectedSizes = new ArrayList<>(); + long seqNum = 0; + for (Integer value : expected) { + byte[] shuffleKey = asShuffleKey(seqNum++); + byte[] shuffleValue = asShuffleValue(value); + shuffleReader.addEntry(shuffleKey, shuffleValue); + + ShuffleEntry record = new ShuffleEntry(shuffleKey, null, shuffleValue); + expectedSizes.add(record.length()); + } + + List actual = new ArrayList<>(); + try (Source.SourceIterator> iter = + shuffleSource.iterator(shuffleReader)) { + while (iter.hasNext()) { + Assert.assertTrue(iter.hasNext()); + Assert.assertTrue(iter.hasNext()); + WindowedValue elem = iter.next(); + actual.add(elem.getValue()); + } + Assert.assertFalse(iter.hasNext()); + Assert.assertFalse(iter.hasNext()); + try { + iter.next(); + Assert.fail("should have failed"); + } catch (NoSuchElementException exn) { + // As expected. + } + } + + Assert.assertEquals(expected, actual); + Assert.assertEquals(expectedSizes, observer.getActualSizes()); + } + + @Test + public void testReadEmptyShuffleSource() throws Exception { + runTestReadShuffleSource(TestUtils.NO_INTS); + } + + @Test + public void testReadNonEmptyShuffleSource() throws Exception { + runTestReadShuffleSource(TestUtils.INTS); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java new file mode 100644 index 0000000000000..065092aeaab14 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java @@ -0,0 +1,134 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker.logging; + +import static org.junit.Assert.assertEquals; + +import com.google.cloud.dataflow.sdk.testing.RestoreMappedDiagnosticContext; +import com.google.common.collect.ImmutableMap; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestRule; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.MDC; + +import java.util.logging.Level; +import java.util.logging.LogRecord; + +/** Unit tests for {@link DataflowWorkerLoggingFormatter}. */ +@RunWith(JUnit4.class) +public class DataflowWorkerLoggingFormatterTest { + @Rule public TestRule restoreMDC = new RestoreMappedDiagnosticContext(); + + @Test + public void testWithUnsetValuesInMDC() { + assertEquals( + "1970-01-01T00:00:00.001Z INFO unknown unknown unknown 2 LoggerName " + + "test.message\n", + new DataflowWorkerLoggingFormatter().format( + createLogRecord("test.message", null))); + } + + @Test + public void testWithMessage() { + MDC.setContextMap(ImmutableMap.of( + "dataflow.jobId", "testJobId", + "dataflow.workerId", "testWorkerId", + "dataflow.workId", "testWorkId")); + assertEquals( + "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName " + + "test.message\n", + new DataflowWorkerLoggingFormatter().format( + createLogRecord("test.message", null))); + } + + @Test + public void testWithMessageAndException() { + MDC.setContextMap(ImmutableMap.of( + "dataflow.jobId", "testJobId", + "dataflow.workerId", "testWorkerId", + "dataflow.workId", "testWorkId")); + assertEquals( + "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName " + + "test.message\n" + + "java.lang.Throwable: exception.test.message\n" + + "\tat declaringClass1.method1(file1.java:1)\n" + + "\tat declaringClass2.method2(file2.java:1)\n" + + "\tat declaringClass3.method3(file3.java:1)\n", + new DataflowWorkerLoggingFormatter().format( + createLogRecord("test.message", createThrowable()))); + } + + @Test + public void testWithException() { + MDC.setContextMap(ImmutableMap.of( + "dataflow.jobId", "testJobId", + "dataflow.workerId", "testWorkerId", + "dataflow.workId", "testWorkId")); + assertEquals( + "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName null\n" + + "java.lang.Throwable: exception.test.message\n" + + "\tat declaringClass1.method1(file1.java:1)\n" + + "\tat declaringClass2.method2(file2.java:1)\n" + + "\tat declaringClass3.method3(file3.java:1)\n", + new DataflowWorkerLoggingFormatter().format( + createLogRecord(null, createThrowable()))); + } + + @Test + public void testWithoutExceptionOrMessage() { + MDC.setContextMap(ImmutableMap.of( + "dataflow.jobId", "testJobId", + "dataflow.workerId", "testWorkerId", + "dataflow.workId", "testWorkId")); + assertEquals( + "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName null\n", + new DataflowWorkerLoggingFormatter().format( + createLogRecord(null, null))); + } + + /** + * @return A throwable with a fixed stack trace. + */ + private Throwable createThrowable() { + Throwable throwable = new Throwable("exception.test.message"); + throwable.setStackTrace(new StackTraceElement[]{ + new StackTraceElement("declaringClass1", "method1", "file1.java", 1), + new StackTraceElement("declaringClass2", "method2", "file2.java", 1), + new StackTraceElement("declaringClass3", "method3", "file3.java", 1), + }); + return throwable; + } + + /** + * Creates and returns a LogRecord with a given message and throwable. + * + * @param message The message to place in the {@link LogRecord} + * @param throwable The throwable to place in the {@link LogRecord} + * @return A {@link LogRecord} with the given message and throwable. + */ + private LogRecord createLogRecord(String message, Throwable throwable) { + LogRecord logRecord = new LogRecord(Level.INFO, message); + logRecord.setLoggerName("LoggerName"); + logRecord.setMillis(1L); + logRecord.setThreadID(2); + logRecord.setThrown(throwable); + return logRecord; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java new file mode 100644 index 0000000000000..71e51f430d88d --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java @@ -0,0 +1,109 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker.logging; + +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestRule; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.util.List; +import java.util.logging.ConsoleHandler; +import java.util.logging.FileHandler; +import java.util.logging.Handler; +import java.util.logging.Level; +import java.util.logging.LogManager; +import java.util.logging.Logger; + +/** Unit tests for {@link DataflowWorkerLoggingInitializer}. */ +@RunWith(JUnit4.class) +public class DataflowWorkerLoggingInitializerTest { + @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); + + @Mock LogManager mockLogManager; + @Mock Logger mockRootLogger; + @Mock Handler mockHandler; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + when(mockLogManager.getLogger("")).thenReturn(mockRootLogger); + when(mockRootLogger.getHandlers()).thenReturn(new Handler[]{ mockHandler }); + } + + @Test + public void testWithDefaults() { + ArgumentCaptor argument = ArgumentCaptor.forClass(Handler.class); + + new DataflowWorkerLoggingInitializer().initialize(mockLogManager); + verify(mockLogManager).getLogger(""); + verify(mockLogManager).reset(); + verify(mockRootLogger).getHandlers(); + verify(mockRootLogger).removeHandler(mockHandler); + verify(mockRootLogger).setLevel(Level.INFO); + verify(mockRootLogger, times(2)).addHandler(argument.capture()); + verifyNoMoreInteractions(mockLogManager, mockRootLogger); + + List handlers = argument.getAllValues(); + assertTrue(isConsoleHandler(handlers.get(0), Level.INFO)); + assertTrue(isFileHandler(handlers.get(1), Level.INFO)); + } + + @Test + public void testWithOverrides() { + ArgumentCaptor argument = ArgumentCaptor.forClass(Handler.class); + System.setProperty("dataflow.worker.logging.level", "WARNING"); + + new DataflowWorkerLoggingInitializer().initialize(mockLogManager); + verify(mockLogManager).getLogger(""); + verify(mockLogManager).reset(); + verify(mockRootLogger).getHandlers(); + verify(mockRootLogger).removeHandler(mockHandler); + verify(mockRootLogger).setLevel(Level.WARNING); + verify(mockRootLogger, times(2)).addHandler(argument.capture()); + verifyNoMoreInteractions(mockLogManager, mockRootLogger); + + List handlers = argument.getAllValues(); + assertTrue(isConsoleHandler(handlers.get(0), Level.WARNING)); + assertTrue(isFileHandler(handlers.get(1), Level.WARNING)); + } + + private boolean isConsoleHandler(Handler handler, Level level) { + return handler instanceof ConsoleHandler + && level.equals(handler.getLevel()) + && handler.getFormatter() instanceof DataflowWorkerLoggingFormatter; + } + + private boolean isFileHandler(Handler handler, Level level) { + return handler instanceof FileHandler + && level.equals(handler.getLevel()) + && handler.getFormatter() instanceof DataflowWorkerLoggingFormatter; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogs.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogs.java new file mode 100644 index 0000000000000..3f4e33d63268a --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogs.java @@ -0,0 +1,240 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import static org.junit.Assert.fail; + +import com.google.common.collect.Lists; + +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; +import org.junit.rules.ExternalResource; +import org.junit.rules.TestRule; + +import java.util.Collection; +import java.util.logging.Handler; +import java.util.logging.Level; +import java.util.logging.LogRecord; +import java.util.logging.Logger; + +/** + * This {@link TestRule} enables the ability to capture JUL logging events during test execution and + * assert expectations that they contain certain messages (with or without {@link Throwable}) at + * certain log levels. For logs generated via the SLF4J logging frontend, the JUL backend must be + * used. + */ +public class ExpectedLogs extends ExternalResource { + /** + * Returns a {@link TestRule} which captures logs for the given class. + * + * @param klass The class to capture logs for. + * @return A {@link ExpectedLogs} test rule. + */ + public static ExpectedLogs none(Class klass) { + return new ExpectedLogs(klass); + } + + /** + * Expect a logging event at the trace level with the given message. + * + * @param substring The message to match against. + */ + public void expectTrace(String substring) { + expect(Level.FINEST, substring); + } + + /** + * Expect a logging event at the trace level with the given message and throwable. + * + * @param substring The message to match against. + * @param t The throwable to match against. + */ + public void expectTrace(String substring, Throwable t) { + expect(Level.FINEST, substring, t); + } + + /** + * Expect a logging event at the debug level with the given message. + * + * @param substring The message to match against. + */ + public void expectDebug(String substring) { + expect(Level.FINE, substring); + } + + /** + * Expect a logging event at the debug level with the given message and throwable. + * + * @param message The message to match against. + * @param t The throwable to match against. + */ + public void expectDebug(String message, Throwable t) { + expect(Level.FINE, message, t); + } + + /** + * Expect a logging event at the info level with the given message. + * @param substring The message to match against. + */ + public void expectInfo(String substring) { + expect(Level.INFO, substring); + } + + /** + * Expect a logging event at the info level with the given message and throwable. + * + * @param message The message to match against. + * @param t The throwable to match against. + */ + public void expectInfo(String message, Throwable t) { + expect(Level.INFO, message, t); + } + + /** + * Expect a logging event at the warn level with the given message. + * + * @param substring The message to match against. + */ + public void expectWarn(String substring) { + expect(Level.WARNING, substring); + } + + /** + * Expect a logging event at the warn level with the given message and throwable. + * + * @param substring The message to match against. + * @param t The throwable to match against. + */ + public void expectWarn(String substring, Throwable t) { + expect(Level.WARNING, substring, t); + } + + /** + * Expect a logging event at the error level with the given message. + * + * @param substring The message to match against. + */ + public void expectError(String substring) { + expect(Level.SEVERE, substring); + } + + /** + * Expect a logging event at the error level with the given message and throwable. + * + * @param substring The message to match against. + * @param t The throwable to match against. + */ + public void expectError(String substring, Throwable t) { + expect(Level.SEVERE, substring, t); + } + + private void expect(final Level level, final String substring) { + expectations.add(new TypeSafeMatcher() { + @Override + public void describeTo(Description description) { + description.appendText(String.format( + "Expected log message of level [%s] containing message [%s]", level, substring)); + } + + @Override + protected boolean matchesSafely(LogRecord item) { + return level.equals(item.getLevel()) + && item.getMessage().contains(substring); + } + }); + } + + private void expect(final Level level, final String substring, final Throwable throwable) { + expectations.add(new TypeSafeMatcher() { + @Override + public void describeTo(Description description) { + description.appendText(String.format( + "Expected log message of level [%s] containg message [%s] with exception [%s] " + + "containing message [%s]", + level, substring, throwable.getClass(), throwable.getMessage())); + } + + @Override + protected boolean matchesSafely(LogRecord item) { + return level.equals(item.getLevel()) + && item.getMessage().contains(substring) + && item.getThrown().getClass().equals(throwable.getClass()) + && item.getThrown().getMessage().contains(throwable.getMessage()); + } + }); + } + + @Override + protected void before() throws Throwable { + previousLevel = log.getLevel(); + log.setLevel(Level.ALL); + log.addHandler(logSaver); + } + + @Override + protected void after() { + log.removeHandler(logSaver); + log.setLevel(previousLevel); + Collection> missingExpecations = Lists.newArrayList(); + FOUND: for (Matcher expectation : expectations) { + for (LogRecord log : logSaver.getLogs()) { + if (expectation.matches(log)) { + continue FOUND; + } + } + missingExpecations.add(expectation); + } + + if (!missingExpecations.isEmpty()) { + fail(String.format("Missed logging expectations: %s", missingExpecations)); + } + } + + private final Logger log; + private final LogSaver logSaver; + private final Collection> expectations; + private Level previousLevel; + + private ExpectedLogs(Class klass) { + log = Logger.getLogger(klass.getName()); + logSaver = new LogSaver(); + expectations = Lists.newArrayList(); + } + + /** + * A JUL logging {@link Handler} that records all logging events which are passed to it. + */ + private static class LogSaver extends Handler { + Collection logRecords = Lists.newArrayList(); + + public Collection getLogs() { + return logRecords; + } + + @Override + public void publish(LogRecord record) { + logRecords.add(record); + } + + @Override + public void flush() {} + + @Override + public void close() throws SecurityException {} + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java new file mode 100644 index 0000000000000..4d9cd0e76639a --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java @@ -0,0 +1,102 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Random; + +/** Tests for {@link FastNanoClockAndSleeper}. */ +@RunWith(JUnit4.class) +public class ExpectedLogsTest { + private static final Logger LOG = LoggerFactory.getLogger(ExpectedLogsTest.class); + + private ExpectedLogs expectedLogs; + + @Before + public void setUp() { + expectedLogs = ExpectedLogs.none(ExpectedLogsTest.class); + } + + @Test + public void testWhenNoExpectations() throws Throwable { + expectedLogs.before(); + LOG.error(generateRandomString()); + expectedLogs.after(); + } + + @Test + public void testWhenExpectationIsMatchedFully() throws Throwable { + String expected = generateRandomString(); + expectedLogs.before(); + expectedLogs.expectError(expected); + LOG.error(expected); + expectedLogs.after(); + } + + + @Test + public void testWhenExpectationIsMatchedPartially() throws Throwable { + String expected = generateRandomString(); + expectedLogs.before(); + expectedLogs.expectError(expected); + LOG.error("Extra stuff around expected " + expected + " blah"); + expectedLogs.after(); + } + + @Test + public void testWhenExpectationIsMatchedWithExceptionBeingLogged() throws Throwable { + String expected = generateRandomString(); + expectedLogs.before(); + expectedLogs.expectError(expected); + LOG.error(expected, new IOException()); + expectedLogs.after(); + } + + @Test(expected = AssertionError.class) + public void testWhenExpectationIsNotMatched() throws Throwable { + String expected = generateRandomString(); + expectedLogs.before(); + expectedLogs.expectError(expected); + expectedLogs.after(); + } + + @Test + public void testLogCaptureOccursAtLowestLogLevel() throws Throwable { + String expected = generateRandomString(); + expectedLogs.before(); + expectedLogs.expectTrace(expected); + LOG.trace(expected); + expectedLogs.after(); + } + + // Generates random strings of 10 characters. + private static String generateRandomString() { + Random random = new Random(); + StringBuilder builder = new StringBuilder(); + for (int i = 0; i < 10; i++) { + builder.append('a' + (char) random.nextInt(26)); + } + return builder.toString(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/FastNanoClockAndSleeper.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/FastNanoClockAndSleeper.java new file mode 100644 index 0000000000000..e9fa9839e7374 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/FastNanoClockAndSleeper.java @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import com.google.api.client.util.NanoClock; +import com.google.api.client.util.Sleeper; + +import org.junit.rules.ExternalResource; +import org.junit.rules.TestRule; + +/** + * This object quickly moves time forward based upon how much it has been asked to sleep, + * without actually sleeping, to simulate the backoff. + */ +public class FastNanoClockAndSleeper extends ExternalResource + implements NanoClock, Sleeper, TestRule { + private long fastNanoTime; + + @Override + public long nanoTime() { + return fastNanoTime; + } + + @Override + protected void before() throws Throwable { + fastNanoTime = NanoClock.SYSTEM.nanoTime(); + } + + @Override + public void sleep(long millis) throws InterruptedException { + fastNanoTime += millis * 1000000L; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/FastNanoClockAndSleeperTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/FastNanoClockAndSleeperTest.java new file mode 100644 index 0000000000000..3c9275f54a23d --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/FastNanoClockAndSleeperTest.java @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.concurrent.TimeUnit; + +/** Tests for {@link FastNanoClockAndSleeper}. */ +@RunWith(JUnit4.class) +public class FastNanoClockAndSleeperTest { + @Rule public FastNanoClockAndSleeper fastNanoClockAndSleeper = new FastNanoClockAndSleeper(); + + @Test + public void testClockAndSleeper() throws Exception { + long sleepTimeMs = TimeUnit.SECONDS.toMillis(30); + long sleepTimeNano = TimeUnit.MILLISECONDS.toNanos(sleepTimeMs); + long fakeTimeNano = fastNanoClockAndSleeper.nanoTime(); + long startTimeNano = System.nanoTime(); + fastNanoClockAndSleeper.sleep(sleepTimeMs); + long maxTimeNano = startTimeNano + TimeUnit.SECONDS.toNanos(1); + // Verify that actual time didn't progress as much as was requested + assertTrue(System.nanoTime() < maxTimeNano); + // Verify that the fake time did go up by the amount requested + assertEquals(fakeTimeNano + sleepTimeNano, fastNanoClockAndSleeper.nanoTime()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ResetDateTimeProvider.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ResetDateTimeProvider.java new file mode 100644 index 0000000000000..675d7ac113611 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ResetDateTimeProvider.java @@ -0,0 +1,41 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import org.joda.time.DateTimeUtils; +import org.joda.time.format.ISODateTimeFormat; +import org.junit.rules.ExternalResource; +import org.junit.rules.TestRule; + +/** + * This {@link TestRule} resets the date time provider in Joda to the system date + * time provider after tests. + */ +public class ResetDateTimeProvider extends ExternalResource { + public void setDateTimeFixed(String iso8601) { + setDateTimeFixed(ISODateTimeFormat.dateTime().parseMillis(iso8601)); + } + + public void setDateTimeFixed(long millis) { + DateTimeUtils.setCurrentMillisFixed(millis); + } + + @Override + protected void after() { + DateTimeUtils.setCurrentMillisSystem(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ResetDateTimeProviderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ResetDateTimeProviderTest.java new file mode 100644 index 0000000000000..5aa96835676c9 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ResetDateTimeProviderTest.java @@ -0,0 +1,55 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +import org.joda.time.DateTimeUtils; +import org.joda.time.format.ISODateTimeFormat; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link ResetDateTimeProvider}. */ +@RunWith(JUnit4.class) +public class ResetDateTimeProviderTest { + private static final String TEST_TIME = "2014-12-08T19:07:06.698Z"; + private static final long TEST_TIME_MS = + ISODateTimeFormat.dateTime().parseMillis(TEST_TIME); + + @Rule public ResetDateTimeProvider resetDateTimeProviderRule = new ResetDateTimeProvider(); + + /* + * Since these tests can run out of order, both test A and B change the provider + * and verify that the provider was reset. + */ + @Test + public void testResetA() { + assertNotEquals(TEST_TIME_MS, DateTimeUtils.currentTimeMillis()); + resetDateTimeProviderRule.setDateTimeFixed(TEST_TIME); + assertEquals(TEST_TIME_MS, DateTimeUtils.currentTimeMillis()); + } + + @Test + public void testResetB() { + assertNotEquals(TEST_TIME_MS, DateTimeUtils.currentTimeMillis()); + resetDateTimeProviderRule.setDateTimeFixed(TEST_TIME); + assertEquals(TEST_TIME_MS, DateTimeUtils.currentTimeMillis()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContext.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContext.java new file mode 100644 index 0000000000000..f0bdb9e217044 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContext.java @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableMap; + +import org.junit.rules.ExternalResource; +import org.slf4j.MDC; + +import java.util.Map; + +/** + * Saves and restores the current MDC for tests. + */ +public class RestoreMappedDiagnosticContext extends ExternalResource { + private Map previousValue; + + public RestoreMappedDiagnosticContext() { + } + + @Override + protected void before() throws Throwable { + previousValue = MoreObjects.firstNonNull( + MDC.getCopyOfContextMap(), + ImmutableMap.of()); + } + + @Override + protected void after() { + MDC.setContextMap(previousValue); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContextTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContextTest.java new file mode 100644 index 0000000000000..c88f275f4bf65 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContextTest.java @@ -0,0 +1,51 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestRule; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.MDC; + +/** Tests for {@link RestoreMappedDiagnosticContext}. */ +@RunWith(JUnit4.class) +public class RestoreMappedDiagnosticContextTest { + @Rule public TestRule restoreMappedDiagnosticContext = new RestoreMappedDiagnosticContext(); + + /* + * Since these tests can run out of order, both test A and B verify that they + * could insert their property and that the other does not exist. + */ + @Test + public void testThatMDCIsClearedA() { + MDC.put("TestA", "TestA"); + assertNotNull(MDC.get("TestA")); + assertNull(MDC.get("TestB")); + } + + @Test + public void testThatMDCIsClearedB() { + MDC.put("TestB", "TestB"); + assertNotNull(MDC.get("TestB")); + assertNull(MDC.get("TestA")); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreSystemProperties.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreSystemProperties.java new file mode 100644 index 0000000000000..ef4f3427b8891 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreSystemProperties.java @@ -0,0 +1,51 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import com.google.common.base.Throwables; + +import org.junit.rules.ExternalResource; +import org.junit.rules.TestRule; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +/** + * Saves and restores the current system properties for tests. + */ +public class RestoreSystemProperties extends ExternalResource implements TestRule { + private byte[] originalProperties; + + @Override + protected void before() throws Throwable { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + System.getProperties().store(baos, ""); + baos.close(); + originalProperties = baos.toByteArray(); + } + + @Override + protected void after() { + try (ByteArrayInputStream bais = new ByteArrayInputStream(originalProperties)) { + System.getProperties().clear(); + System.getProperties().load(bais); + } catch (IOException e) { + throw Throwables.propagate(e); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreSystemPropertiesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreSystemPropertiesTest.java new file mode 100644 index 0000000000000..8a4bb488922ef --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreSystemPropertiesTest.java @@ -0,0 +1,50 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestRule; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link RestoreSystemProperties}. */ +@RunWith(JUnit4.class) +public class RestoreSystemPropertiesTest { + @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); + + /* + * Since these tests can run out of order, both test A and B verify that they + * could insert their property and that the other does not exist. + */ + @Test + public void testThatPropertyIsClearedA() { + System.getProperties().put("TestA", "TestA"); + assertNotNull(System.getProperty("TestA")); + assertNull(System.getProperty("TestB")); + } + + @Test + public void testThatPropertyIsClearedB() { + System.getProperties().put("TestB", "TestB"); + assertNotNull(System.getProperty("TestB")); + assertNull(System.getProperty("TestA")); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java new file mode 100644 index 0000000000000..da4f66ec07758 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java @@ -0,0 +1,76 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner; +import com.google.common.collect.ImmutableMap; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestRule; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link TestPipeline}. */ +@RunWith(JUnit4.class) +public class TestPipelineTest { + @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); + + @Test + public void testCreationUsingDefaults() { + assertNotNull(TestPipeline.create()); + } + + @Test + public void testCreationOfPipelineOptions() throws Exception { + ObjectMapper mapper = new ObjectMapper(); + String stringOptions = mapper.writeValueAsString( + ImmutableMap.of("options", + ImmutableMap.builder() + .put("runner", DataflowPipelineRunner.class.getName()) + .put("project", "testProject") + .put("apiRootUrl", "testApiRootUrl") + .put("dataflowEndpoint", "testDataflowEndpoint") + .put("tempLocation", "testTempLocation") + .put("serviceAccountName", "testServiceAccountName") + .put("serviceAccountKeyfile", "testServiceAccountKeyfile") + .put("zone", "testZone") + .put("numWorkers", "1") + .put("diskSizeGb", "2") + .put("shuffleDiskSizeGb", "3") + .build())); + System.getProperties().put("dataflowOptions", stringOptions); + TestDataflowPipelineOptions options = TestPipeline.getPipelineOptions(); + assertEquals(DataflowPipelineRunner.class, options.getRunner()); + assertEquals("TestPipelineTest", options.getAppName()); + assertEquals("testCreationOfPipelineOptions", options.getJobName()); + assertEquals("testProject", options.getProject()); + assertEquals("testApiRootUrl", options.getApiRootUrl()); + assertEquals("testDataflowEndpoint", options.getDataflowEndpoint()); + assertEquals("testTempLocation", options.getTempLocation()); + assertEquals("testServiceAccountName", options.getServiceAccountName()); + assertEquals("testServiceAccountKeyfile", options.getServiceAccountKeyfile()); + assertEquals("testZone", options.getZone()); + assertEquals(2, options.getDiskSizeGb()); + assertEquals(3, options.getShuffleDiskSizeGb()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java new file mode 100644 index 0000000000000..b0493491634c2 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java @@ -0,0 +1,287 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static com.google.cloud.dataflow.sdk.TestUtils.checkCombineFn; +import static com.google.cloud.dataflow.sdk.TestUtils.createInts; +import static org.hamcrest.collection.IsIterableContainingInOrder.contains; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.runners.DirectPipeline; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.ApproximateQuantiles.ApproximateQuantilesCombineFn; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.hamcrest.CoreMatchers; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeDiagnosingMatcher; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Tests for ApproximateQuantiles + */ +@RunWith(JUnit4.class) +public class ApproximateQuantilesTest { + + static final List> TABLE = Arrays.asList( + KV.of("a", 1), + KV.of("a", 2), + KV.of("a", 3), + KV.of("b", 1), + KV.of("b", 10), + KV.of("b", 10), + KV.of("b", 100) + ); + + public PCollection> createInputTable(Pipeline p) { + return p.apply(Create.of(TABLE)).setCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + } + + @Test + public void testQuantilesGlobally() { + DirectPipeline p = DirectPipeline.createForTest(); + + PCollection input = intRangeCollection(p, 101); + PCollection> quantiles = + input.apply(ApproximateQuantiles.globally(5)); + + DirectPipelineRunner.EvaluationResults results = p.run(); + + DataflowAssert.that(quantiles) + .containsInAnyOrder(Arrays.asList(0, 25, 50, 75, 100)); + } + + @Test + public void testQuantilesGobally_comparable() { + DirectPipeline p = DirectPipeline.createForTest(); + + PCollection input = intRangeCollection(p, 101); + PCollection> quantiles = + input.apply( + ApproximateQuantiles.globally(5, new DescendingIntComparator())); + + DirectPipelineRunner.EvaluationResults results = p.run(); + + DataflowAssert.that(quantiles) + .containsInAnyOrder(Arrays.asList(100, 75, 50, 25, 0)); + } + + @Test + public void testQuantilesPerKey() { + Pipeline p = TestPipeline.create(); + + PCollection> input = createInputTable(p); + PCollection>> quantiles = input.apply( + ApproximateQuantiles.perKey(2)); + + DataflowAssert.that(quantiles) + .containsInAnyOrder( + KV.of("a", Arrays.asList(1, 3)), + KV.of("b", Arrays.asList(1, 100))); + p.run(); + + } + + @Test + public void testQuantilesPerKey_reversed() { + Pipeline p = TestPipeline.create(); + + PCollection> input = createInputTable(p); + PCollection>> quantiles = input.apply( + ApproximateQuantiles.perKey( + 2, new DescendingIntComparator())); + + DataflowAssert.that(quantiles) + .containsInAnyOrder( + KV.of("a", Arrays.asList(3, 1)), + KV.of("b", Arrays.asList(100, 1))); + p.run(); + } + + @Test + public void testSingleton() { + checkCombineFn( + ApproximateQuantilesCombineFn.create(5), + Arrays.asList(389), + Arrays.asList(389, 389, 389, 389, 389)); + } + + @Test + public void testSimpleQuantiles() { + checkCombineFn( + ApproximateQuantilesCombineFn.create(5), + intRange(101), + Arrays.asList(0, 25, 50, 75, 100)); + } + + @Test + public void testUnevenQuantiles() { + checkCombineFn( + ApproximateQuantilesCombineFn.create(37), + intRange(5000), + quantileMatcher(5000, 37, 20 /* tolerance */)); + } + + @Test + public void testLargerQuantiles() { + checkCombineFn( + ApproximateQuantilesCombineFn.create(50), + intRange(10001), + quantileMatcher(10001, 50, 20 /* tolerance */)); + } + + @Test + public void testTightEpsilon() { + checkCombineFn( + ApproximateQuantilesCombineFn.create(10).withEpsilon(0.01), + intRange(10001), + quantileMatcher(10001, 10, 5 /* tolerance */)); + } + + @Test + public void testDuplicates() { + int size = 101; + List all = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + all.addAll(intRange(size)); + } + checkCombineFn( + ApproximateQuantilesCombineFn.create(5), + all, + Arrays.asList(0, 25, 50, 75, 100)); + } + + @Test + public void testLotsOfDuplicates() { + List all = new ArrayList<>(); + all.add(1); + for (int i = 1; i < 300; i++) { + all.add(2); + } + for (int i = 300; i < 1000; i++) { + all.add(3); + } + checkCombineFn( + ApproximateQuantilesCombineFn.create(5), + all, + Arrays.asList(1, 2, 3, 3, 3)); + } + + @Test + public void testLogDistribution() { + List all = new ArrayList<>(); + for (int i = 1; i < 1000; i++) { + all.add((int) Math.log(i)); + } + checkCombineFn( + ApproximateQuantilesCombineFn.create(5), + all, + Arrays.asList(0, 5, 6, 6, 6)); + } + + @Test + public void testZipfianDistribution() { + List all = new ArrayList<>(); + for (int i = 1; i < 1000; i++) { + all.add(1000 / i); + } + checkCombineFn( + ApproximateQuantilesCombineFn.create(5), + all, + Arrays.asList(1, 1, 2, 4, 1000)); + } + + @Test + public void testAlternateComparator() { + List inputs = Arrays.asList( + "aa", "aaa", "aaaa", "b", "ccccc", "dddd", "zz"); + checkCombineFn( + ApproximateQuantilesCombineFn.create(3), + inputs, + Arrays.asList("aa", "b", "zz")); + checkCombineFn( + ApproximateQuantilesCombineFn.create(3, new TopTest.OrderByLength()), + inputs, + Arrays.asList("b", "aaa", "ccccc")); + } + + private Matcher> quantileMatcher( + int size, int numQuantiles, int absoluteError) { + List> quantiles = new ArrayList<>(); + quantiles.add(CoreMatchers.is(0)); + for (int k = 1; k < numQuantiles - 1; k++) { + int expected = (int) (((double) (size - 1)) * k / (numQuantiles - 1)); + quantiles.add(new Between<>( + expected - absoluteError, expected + absoluteError)); + } + quantiles.add(CoreMatchers.is(size - 1)); + return contains(quantiles); + } + + private static class Between> + extends TypeSafeDiagnosingMatcher { + private final T min; + private final T max; + private Between(T min, T max) { + this.min = min; + this.max = max; + } + @Override + public void describeTo(Description description) { + description.appendText("is between " + min + " and " + max); + } + + @Override + protected boolean matchesSafely(T item, Description mismatchDescription) { + return min.compareTo(item) <= 0 && item.compareTo(max) <= 0; + } + } + + private static class DescendingIntComparator implements + SerializableComparator { + @Override + public int compare(Integer o1, Integer o2) { + return o2.compareTo(o1); + } + } + + private PCollection intRangeCollection(Pipeline p, int size) { + return createInts(p, intRange(size)); + } + + private List intRange(int size) { + List all = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + all.add(i); + } + return all; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java new file mode 100644 index 0000000000000..2b2ff0ac9c967 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java @@ -0,0 +1,302 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.TestUtils; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.coders.DoubleCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.SerializableCoder; +import com.google.cloud.dataflow.sdk.runners.DirectPipeline; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.EvaluationResults; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * Tests for the ApproximateUnique aggregator transform. + */ +@RunWith(JUnit4.class) +public class ApproximateUniqueTest { + + @Test + public void testEstimationErrorToSampleSize() { + assertEquals(40000, ApproximateUnique.sampleSizeFromEstimationError(0.01)); + assertEquals(10000, ApproximateUnique.sampleSizeFromEstimationError(0.02)); + assertEquals(2500, ApproximateUnique.sampleSizeFromEstimationError(0.04)); + assertEquals(1600, ApproximateUnique.sampleSizeFromEstimationError(0.05)); + assertEquals(400, ApproximateUnique.sampleSizeFromEstimationError(0.1)); + assertEquals(100, ApproximateUnique.sampleSizeFromEstimationError(0.2)); + assertEquals(25, ApproximateUnique.sampleSizeFromEstimationError(0.4)); + assertEquals(16, ApproximateUnique.sampleSizeFromEstimationError(0.5)); + } + + public PCollection createInput(Pipeline p, Iterable input, + Coder coder) { + return p.apply(Create.of(input)).setCoder(coder); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testApproximateUniqueWithSmallInput() { + Pipeline p = TestPipeline.create(); + + PCollection input = + createInput(p, Arrays.asList(1, 2, 3, 3), BigEndianIntegerCoder.of()); + + PCollection estimate = input + .apply(ApproximateUnique.globally(1000)); + + DataflowAssert.that(estimate).containsInAnyOrder(3L); + + p.run(); + } + + @Test + public void testApproximateUniqueWithDuplicates() { + runApproximateUniqueWithDuplicates(100, 100, 100); + runApproximateUniqueWithDuplicates(1000, 1000, 100); + runApproximateUniqueWithDuplicates(1500, 1000, 100); + runApproximateUniqueWithDuplicates(10000, 1000, 100); + } + + private void runApproximateUniqueWithDuplicates(int elementCount, + int uniqueCount, int sampleSize) { + + assert elementCount >= uniqueCount; + List elements = Lists.newArrayList(); + for (int i = 0; i < elementCount; i++) { + elements.add(1.0 / (i % uniqueCount + 1)); + } + Collections.shuffle(elements); + + DirectPipeline p = DirectPipeline.createForTest(); + PCollection input = createInput(p, elements, DoubleCoder.of()); + PCollection estimate = + input.apply(ApproximateUnique.globally(sampleSize)); + + EvaluationResults results = p.run(); + + verifyEstimate(uniqueCount, sampleSize, + results.getPCollection(estimate).get(0)); + } + + @Test + public void testApproximateUniqueWithSkewedDistributions() { + runApproximateUniqueWithSkewedDistributions(100, 100, 100); + runApproximateUniqueWithSkewedDistributions(10000, 10000, 100); + runApproximateUniqueWithSkewedDistributions(10000, 1000, 100); + runApproximateUniqueWithSkewedDistributions(10000, 200, 100); + } + + @Test + public void testApproximateUniqueWithSkewedDistributionsAndLargeSampleSize() { + runApproximateUniqueWithSkewedDistributions(10000, 2000, 1000); + } + + private void runApproximateUniqueWithSkewedDistributions(int elementCount, + final int uniqueCount, final int sampleSize) { + List elements = Lists.newArrayList(); + // Zipf distribution with approximately elementCount items. + double s = 1 - 1.0 * uniqueCount / elementCount; + double maxCount = Math.pow(uniqueCount, s); + for (int k = 0; k < uniqueCount; k++) { + int count = Math.max(1, (int) Math.round(maxCount * Math.pow(k, -s))); + // Element k occurs count times. + for (int c = 0; c < count; c++) { + elements.add(k); + } + } + + DirectPipeline p = DirectPipeline.createForTest(); + PCollection input = + createInput(p, elements, BigEndianIntegerCoder.of()); + PCollection estimate = + input.apply(ApproximateUnique.globally(sampleSize)); + + EvaluationResults results = p.run(); + + verifyEstimate(uniqueCount, sampleSize, + results.getPCollection(estimate).get(0).longValue()); + } + + @Test + public void testApproximateUniquePerKey() { + List> elements = Lists.newArrayList(); + List keys = ImmutableList.of(20, 50, 100); + int elementCount = 1000; + int sampleSize = 100; + // Use the key as the number of unique values. + for (int uniqueCount : keys) { + for (int value = 0; value < elementCount; value++) { + elements.add(KV.of(uniqueCount, value % uniqueCount)); + } + } + + DirectPipeline p = DirectPipeline.createForTest(); + PCollection> input = createInput(p, elements, + KvCoder.of(BigEndianIntegerCoder.of(), BigEndianIntegerCoder.of())); + PCollection> counts = + input.apply(ApproximateUnique.perKey(sampleSize)); + + EvaluationResults results = p.run(); + + for (KV result : results.getPCollection(counts)) { + verifyEstimate(result.getKey(), sampleSize, result.getValue()); + } + } + + /** + * Applies {@link ApproximateUnique} for different sample sizes and verifies + * that the estimation error falls within the maximum allowed error of + * {@code 2 / sqrt(sampleSize)}. + */ + @Test + public void testApproximateUniqueWithDifferentSampleSizes() { + runApproximateUniquePipeline(16); + runApproximateUniquePipeline(64); + runApproximateUniquePipeline(128); + runApproximateUniquePipeline(256); + runApproximateUniquePipeline(512); + runApproximateUniquePipeline(1000); + runApproximateUniquePipeline(1024); + try { + runApproximateUniquePipeline(15); + fail("Accepted sampleSize < 16"); + } catch (IllegalArgumentException e) { + assertTrue("Expected an exception due to sampleSize < 16", e.getMessage() + .startsWith("ApproximateUnique needs a sampleSize >= 16")); + } + } + + /** + * Applies {@code ApproximateUnique(sampleSize)} verifying that the estimation + * error falls within the maximum allowed error of {@code 2/sqrt(sampleSize)}. + */ + private void runApproximateUniquePipeline(int sampleSize) { + DirectPipeline p = DirectPipeline.createForTest(); + PCollection collection = readPCollection(p); + + PCollection exact = collection.apply(RemoveDuplicates.create()) + .apply(Combine.globally(new CountElements())); + + PCollection approximate = + collection.apply(ApproximateUnique.globally(sampleSize)); + + EvaluationResults results = p.run(); + + verifyEstimate(results.getPCollection(exact).get(0).longValue(), sampleSize, + results.getPCollection(approximate).get(0).longValue()); + } + + /** + * Reads a large {@code PCollection}. + */ + private PCollection readPCollection(Pipeline p) { + // TODO: Read PCollection from a set of text files. + List page = TestUtils.LINES; + final int pages = 1000; + ArrayList file = new ArrayList<>(pages * page.size()); + for (int i = 0; i < pages; i++) { + file.addAll(page); + } + assert file.size() == pages * page.size(); + PCollection words = TestUtils.createStrings(p, file); + return words; + } + + /** + * Checks that the estimation error, i.e., the difference between + * {@code uniqueCount} and {@code estimate} is less than + * {@code 2 / sqrt(sampleSize}). + */ + private static void verifyEstimate(long uniqueCount, int sampleSize, + long estimate) { + if (uniqueCount < sampleSize) { + assertEquals("Number of hashes is less than the sample size. " + + "Estimate should be exact", uniqueCount, estimate); + } + + double error = 100.0 * Math.abs(estimate - uniqueCount) / uniqueCount; + double maxError = 100.0 * 2 / Math.sqrt(sampleSize); + + assertTrue("Estimate= " + estimate + " Actual=" + uniqueCount + " Error=" + + error + "%, MaxError=" + maxError + "%.", error < maxError); + } + + /** + * Combiner function counting the number of elements in an input PCollection. + * + * @param the type of elements in the input PCollection. + */ + private static class CountElements extends CombineFn { + + @Override + public Long[] createAccumulator() { + Long[] accumulator = new Long[1]; + accumulator[0] = 0L; + return accumulator; + } + + @Override + public void addInput(Long[] accumulator, E input) { + accumulator[0]++; + } + + @Override + public Long[] mergeAccumulators(Iterable accumulators) { + Long[] sum = new Long[1]; + sum[0] = 0L; + for (Long[] accumulator : accumulators) { + sum[0] += accumulator[0]; + } + return sum; + } + + @Override + public Long extractOutput(Long[] accumulator) { + return accumulator[0]; + } + + @Override + public Coder getAccumulatorCoder(CoderRegistry registry, + Coder inputCoder) { + return SerializableCoder.of(Long[].class); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java new file mode 100644 index 0000000000000..52b0b230a19d1 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java @@ -0,0 +1,527 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static org.junit.Assert.assertThat; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.coders.CustomCoder; +import com.google.cloud.dataflow.sdk.coders.DoubleCoder; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.SerializableCoder; +import com.google.cloud.dataflow.sdk.coders.StandardCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.RecordingPipelineVisitor; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.hamcrest.Matchers; +import org.joda.time.Duration; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; + +/** + * Tests for Combine transforms. + */ +@RunWith(JUnit4.class) +public class CombineTest { + + @SuppressWarnings("unchecked") + static final KV[] TABLE = new KV[] { + KV.of("a", 1), + KV.of("a", 1), + KV.of("a", 4), + KV.of("b", 1), + KV.of("b", 13), + }; + + @SuppressWarnings("unchecked") + static final KV[] EMPTY_TABLE = new KV[] { + }; + + static final Integer[] NUMBERS = new Integer[] { + 1, 1, 2, 3, 5, 8, 13, 21, 34, 55 + }; + + PCollection> createInput(Pipeline p, + KV[] table) { + return p.apply(Create.of(Arrays.asList(table))).setCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + } + + private void runTestSimpleCombine(KV[] table, + int globalSum, + KV[] perKeySums) { + Pipeline p = TestPipeline.create(); + PCollection> input = createInput(p, table); + + PCollection sum = input + .apply(Values.create()) + .apply(Combine.globally(new SumInts())); + + // Java 8 will infer. + PCollection> sumPerKey = input + .apply(Combine.perKey(new SumInts())); + + DataflowAssert.that(sum).containsInAnyOrder(globalSum); + DataflowAssert.that(sumPerKey).containsInAnyOrder(perKeySums); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testSimpleCombine() { + runTestSimpleCombine(TABLE, 20, new KV[] { + KV.of("a", 6), KV.of("b", 14) }); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testSimpleCombineEmpty() { + runTestSimpleCombine(EMPTY_TABLE, 0, new KV[] { }); + } + + private void runTestBasicCombine(KV[] table, + Set globalUnique, + KV>[] perKeyUnique) { + Pipeline p = TestPipeline.create(); + p.getCoderRegistry().registerCoder(Set.class, SetCoder.class); + PCollection> input = createInput(p, table); + + PCollection> unique = input + .apply(Values.create()) + .apply(Combine.globally(new UniqueInts())); + + // Java 8 will infer. + PCollection>> uniquePerKey = input + .apply(Combine.>perKey(new UniqueInts())); + + DataflowAssert.that(unique).containsInAnyOrder(globalUnique); + DataflowAssert.that(uniquePerKey).containsInAnyOrder(perKeyUnique); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testBasicCombine() { + runTestBasicCombine(TABLE, ImmutableSet.of(1, 13, 4), new KV[] { + KV.of("a", (Set) ImmutableSet.of(1, 4)), + KV.of("b", (Set) ImmutableSet.of(1, 13)) }); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testBasicCombineEmpty() { + runTestBasicCombine(EMPTY_TABLE, ImmutableSet.of(), new KV[] { }); + } + + private void runTestAccumulatingCombine(KV[] table, + Double globalMean, + KV[] perKeyMeans) { + Pipeline p = TestPipeline.create(); + PCollection> input = createInput(p, table); + + PCollection mean = input + .apply(Values.create()) + .apply(Combine.globally(new MeanInts())); + + // Java 8 will infer. + PCollection> meanPerKey = input.apply( + Combine.perKey(new MeanInts())); + + DataflowAssert.that(mean).containsInAnyOrder(globalMean); + DataflowAssert.that(meanPerKey).containsInAnyOrder(perKeyMeans); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testWindowedCombineEmpty() { + Pipeline p = TestPipeline.create(); + + PCollection mean = p + .apply(Create.of()).setCoder(BigEndianIntegerCoder.of()) + .apply(Window.into(FixedWindows.of(Duration.millis(1)))) + .apply(Combine.globally(new MeanInts())); + + DataflowAssert.that(mean).containsInAnyOrder(); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testAccumulatingCombine() { + runTestAccumulatingCombine(TABLE, 4.0, new KV[] { + KV.of("a", 2.0), KV.of("b", 7.0) }); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testAccumulatingCombineEmpty() { + runTestAccumulatingCombine(EMPTY_TABLE, 0.0, new KV[] { }); + } + + // Checks that Min, Max, Mean, Sum (operations which pass-through to Combine), + // provide their own top-level name. + @Test + public void testCombinerNames() { + Pipeline p = TestPipeline.create(); + PCollection> input = createInput(p, TABLE); + + Combine.PerKey min = Min.integersPerKey(); + Combine.PerKey max = Max.integersPerKey(); + Combine.PerKey mean = Mean.perKey(); + Combine.PerKey sum = Sum.integersPerKey(); + + input.apply(min); + input.apply(max); + input.apply(mean); + input.apply(sum); + + p.traverseTopologically(new RecordingPipelineVisitor()); + + assertThat(p.getFullName(min), Matchers.startsWith("Min")); + assertThat(p.getFullName(max), Matchers.startsWith("Max")); + assertThat(p.getFullName(mean), Matchers.startsWith("Mean")); + assertThat(p.getFullName(sum), Matchers.startsWith("Sum")); + } + + @Test + public void testAddInputsRandomly() { + TestCounter counter = new TestCounter(); + Combine.KeyedCombineFn< + String, Integer, TestCounter.Counter, Iterable> fn = + counter.asKeyedFn(); + + List accums = DirectPipelineRunner.TestCombineDoFn.addInputsRandomly( + fn, "bob", Arrays.asList(NUMBERS), new Random(42)); + + assertThat(accums, Matchers.contains( + counter.new Counter(3, 2, 0, 0), + counter.new Counter(131, 5, 0, 0), + counter.new Counter(8, 2, 0, 0), + counter.new Counter(1, 1, 0, 0))); + } + + //////////////////////////////////////////////////////////////////////////// + // Test classes, for different kinds of combining fns. + + /** Example SerializableFunction combiner. */ + public static class SumInts + implements SerializableFunction, Integer> { + @Override + public Integer apply(Iterable input) { + int sum = 0; + for (int item : input) { + sum += item; + } + return sum; + } + } + + /** Example CombineFn. */ + public static class UniqueInts extends + Combine.CombineFn, Set> { + + @Override + public Set createAccumulator() { + return new HashSet<>(); + } + + @Override + public void addInput(Set accumulator, Integer input) { + accumulator.add(input); + } + + @Override + public Set mergeAccumulators(Iterable> accumulators) { + Set all = new HashSet<>(); + for (Set part : accumulators) { + all.addAll(part); + } + return all; + } + + @Override + public Set extractOutput(Set accumulator) { + return accumulator; + } + } + + // Note: not a deterministic encoding + private static class SetCoder extends StandardCoder> { + + public static SetCoder of(Coder elementCoder) { + return new SetCoder<>(elementCoder); + } + + @JsonCreator + public static SetCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> components) { + Preconditions.checkArgument(components.size() == 1, + "Expecting 1 component, got " + components.size()); + return of((Coder) components.get(0)); + } + + public static List getInstanceComponents(Set exampleValue) { + return IterableCoder.getInstanceComponents(exampleValue); + } + + private final Coder> iterableCoder; + + private SetCoder(Coder elementCoder) { + iterableCoder = IterableCoder.of(elementCoder); + } + + @Override + public void encode(Set value, OutputStream outStream, Context context) + throws CoderException, IOException { + iterableCoder.encode(value, outStream, context); + } + + @Override + public Set decode(InputStream inStream, Context context) + throws CoderException, IOException { + // TODO: Eliminate extra copy if used in production. + return Sets.newHashSet(iterableCoder.decode(inStream, context)); + } + + @Override + public List> getCoderArguments() { + return iterableCoder.getCoderArguments(); + } + + @Override + public boolean isDeterministic() { + return false; + } + + @Override + public boolean isRegisterByteSizeObserverCheap(Set value, Context context) { + return iterableCoder.isRegisterByteSizeObserverCheap(value, context); + } + + @Override + public void registerByteSizeObserver( + Set value, ElementByteSizeObserver observer, Context context) + throws Exception { + iterableCoder.registerByteSizeObserver(value, observer, context); + } + } + + /** Example AccumulatingCombineFn. */ + public static class MeanInts extends + Combine.AccumulatingCombineFn { + private static final Coder LONG_CODER = BigEndianLongCoder.of(); + private static final Coder DOUBLE_CODER = DoubleCoder.of(); + + class CountSum extends + Combine.AccumulatingCombineFn.Accumulator { + long count = 0; + double sum = 0.0; + + CountSum(long count, double sum) { + this.count = count; + this.sum = sum; + } + + @Override + public void addInput(Integer element) { + count++; + sum += element.doubleValue(); + } + + @Override + public void mergeAccumulator(CountSum accumulator) { + count += accumulator.count; + sum += accumulator.sum; + } + + @Override + public Double extractOutput() { + return count == 0 ? 0.0 : sum / count; + } + } + + @Override + public CountSum createAccumulator() { + return new CountSum(0, 0.0); + } + + @Override + public Coder getAccumulatorCoder( + CoderRegistry registry, Coder inputCoder) { + return new CountSumCoder(); + } + + /** + * A Coder for CountSum + */ + public class CountSumCoder extends CustomCoder { + @Override + public void encode(CountSum value, OutputStream outStream, + Context context) throws CoderException, IOException { + LONG_CODER.encode(value.count, outStream, context); + DOUBLE_CODER.encode(value.sum, outStream, context); + } + + @Override + public CountSum decode(InputStream inStream, Coder.Context context) + throws CoderException, IOException { + long count = LONG_CODER.decode(inStream, context); + double sum = DOUBLE_CODER.decode(inStream, context); + return new CountSum(count, sum); + } + + @Override + public boolean isDeterministic() { + return true; + } + + @Override + public boolean isRegisterByteSizeObserverCheap( + CountSum value, Context context) { + return true; + } + + @Override + public void registerByteSizeObserver( + CountSum value, ElementByteSizeObserver observer, Context context) + throws Exception { + LONG_CODER.registerByteSizeObserver(value.count, observer, context); + DOUBLE_CODER.registerByteSizeObserver(value.sum, observer, context); + } + } + } + + /** Another example AccumulatingCombineFn. */ + public static class TestCounter extends + Combine.AccumulatingCombineFn< + Integer, TestCounter.Counter, Iterable> { + + /** An accumulator that observes its merges and outputs */ + public class Counter extends + Combine.AccumulatingCombineFn< + Integer, Counter, Iterable>.Accumulator { + + public long sum = 0; + public long inputs = 0; + public long merges = 0; + public long outputs = 0; + + public Counter(long sum, long inputs, long merges, long outputs) { + this.sum = sum; + this.inputs = inputs; + this.merges = merges; + this.outputs = outputs; + } + + @Override + public void addInput(Integer element) { + Preconditions.checkState(merges == 0); + Preconditions.checkState(outputs == 0); + + inputs++; + sum += element; + } + + @Override + public void mergeAccumulator(Counter accumulator) { + Preconditions.checkState(outputs == 0); + Preconditions.checkArgument(accumulator.outputs == 0); + + merges += accumulator.merges + 1; + inputs += accumulator.inputs; + sum += accumulator.sum; + } + + @Override + public Iterable extractOutput() { + Preconditions.checkState(outputs == 0); + + return Arrays.asList(sum, inputs, merges, outputs); + } + + @Override + public int hashCode() { + return (int) (sum * 17 + inputs * 31 + merges * 43 + outputs * 181); + } + + @Override + public boolean equals(Object otherObj) { + if (otherObj instanceof Counter) { + Counter other = (Counter) otherObj; + return (sum == other.sum + && inputs == other.inputs + && merges == other.merges + && outputs == other.outputs); + } + return false; + } + + public String toString() { + return sum + ":" + inputs + ":" + merges + ":" + outputs; + } + } + + @Override + public Counter createAccumulator() { + return new Counter(0, 0, 0, 0); + } + + @Override + public Coder getAccumulatorCoder( + CoderRegistry registry, Coder inputCoder) { + return SerializableCoder.of(Counter.class); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CountTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CountTest.java new file mode 100644 index 0000000000000..05375bd7c5364 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CountTest.java @@ -0,0 +1,112 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES; +import static com.google.cloud.dataflow.sdk.TestUtils.createStrings; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Tests for Count. + */ +@RunWith(JUnit4.class) +public class CountTest { + static final String[] WORDS_ARRAY = new String[] { + "hi", "there", "hi", "hi", "sue", "bob", + "hi", "sue", "", "", "ZOW", "bob", "" }; + + static final List WORDS = Arrays.asList(WORDS_ARRAY); + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testCountPerElementBasic() { + Pipeline p = TestPipeline.create(); + + PCollection input = createStrings(p, WORDS); + + PCollection> output = + input.apply(Count.perElement()); + + DataflowAssert.that(output) + .containsInAnyOrder( + KV.of("hi", 4L), + KV.of("there", 1L), + KV.of("sue", 2L), + KV.of("bob", 2L), + KV.of("", 3L), + KV.of("ZOW", 1L)); + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testCountPerElementEmpty() { + Pipeline p = TestPipeline.create(); + + PCollection input = createStrings(p, NO_LINES); + + PCollection> output = + input.apply(Count.perElement()); + + DataflowAssert.that(output) + .containsInAnyOrder(); + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testCountGloballyBasic() { + Pipeline p = TestPipeline.create(); + + PCollection input = createStrings(p, WORDS); + + PCollection output = + input.apply(Count.globally()); + + DataflowAssert.that(output) + .containsInAnyOrder(13L); + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testCountGloballyEmpty() { + Pipeline p = TestPipeline.create(); + + PCollection input = createStrings(p, NO_LINES); + + PCollection output = + input.apply(Count.globally()); + + DataflowAssert.that(output) + .containsInAnyOrder(0L); + p.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CreateTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CreateTest.java new file mode 100644 index 0000000000000..8202da086240c --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CreateTest.java @@ -0,0 +1,189 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static com.google.cloud.dataflow.sdk.TestUtils.LINES; +import static com.google.cloud.dataflow.sdk.TestUtils.LINES_ARRAY; +import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES; +import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES_ARRAY; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TimestampedValue; + +import org.hamcrest.Matchers; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Tests for Create. + */ +@RunWith(JUnit4.class) +public class CreateTest { + @Rule public final ExpectedException thrown = ExpectedException.none(); + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testCreate() { + Pipeline p = TestPipeline.create(); + + PCollection output = + p.apply(Create.of(LINES)); + + DataflowAssert.that(output) + .containsInAnyOrder(LINES_ARRAY); + p.run(); + } + + // TODO: setOrdered(true) isn't supported yet by the Dataflow service. + @Test + public void testCreateOrdered() { + Pipeline p = TestPipeline.create(); + + PCollection output = + p.apply(Create.of(LINES)) + .setOrdered(true); + + DataflowAssert.that(output) + .containsInOrder(LINES_ARRAY); + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testCreateEmpty() { + Pipeline p = TestPipeline.create(); + + PCollection output = + p.apply(Create.of(NO_LINES)) + .setCoder(StringUtf8Coder.of()); + + DataflowAssert.that(output) + .containsInAnyOrder(NO_LINES_ARRAY); + p.run(); + } + + static class Record implements Serializable { + } + + static class Record2 extends Record { + } + + @Test + public void testPolymorphicType() throws Exception { + thrown.expect(RuntimeException.class); + thrown.expectMessage( + Matchers.containsString("unable to infer a default Coder")); + + Pipeline p = TestPipeline.create(); + + // Create won't infer a default coder in this case. + p.apply(Create.of(new Record(), new Record2())); + + p.run(); + } + + @Test + public void testCreateParameterizedType() throws Exception { + Pipeline p = TestPipeline.create(); + + PCollection> output = + p.apply(Create.of( + TimestampedValue.of("a", new Instant(0)), + TimestampedValue.of("b", new Instant(0)))); + + DataflowAssert.that(output) + .containsInAnyOrder( + TimestampedValue.of("a", new Instant(0)), + TimestampedValue.of("b", new Instant(0))); + } + + private static class PrintTimestamps extends DoFn { + @Override + public void processElement(ProcessContext c) { + c.output(c.element() + ":" + c.timestamp().getMillis()); + } + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testCreateTimestamped() { + Pipeline p = TestPipeline.create(); + + List> data = Arrays.asList( + TimestampedValue.of("a", new Instant(1L)), + TimestampedValue.of("b", new Instant(2L)), + TimestampedValue.of("c", new Instant(3L))); + + PCollection output = + p.apply(Create.timestamped(data)) + .apply(ParDo.of(new PrintTimestamps())); + + DataflowAssert.that(output) + .containsInAnyOrder("a:1", "b:2", "c:3"); + p.run(); + } + + @Test + // This test fails when run on the service! + // TODO: @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testCreateTimestampedEmpty() { + Pipeline p = TestPipeline.create(); + + PCollection output = p + .apply(Create.timestamped(new ArrayList>())) + .setCoder(StringUtf8Coder.of()); + + DataflowAssert.that(output) + .containsInAnyOrder(); + p.run(); + } + + @Test + public void testCreateTimestampedPolymorphicType() throws Exception { + thrown.expect(RuntimeException.class); + thrown.expectMessage( + Matchers.containsString("unable to infer a default Coder")); + + Pipeline p = TestPipeline.create(); + + // Create won't infer a default coder in this case. + PCollection c = p.apply(Create.timestamped( + TimestampedValue.of(new Record(), new Instant(0)), + TimestampedValue.of(new Record2(), new Instant(0)))); + + p.run(); + + + throw new RuntimeException("Coder: " + c.getCoder()); + + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FirstTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FirstTest.java new file mode 100644 index 0000000000000..bcd14d0c6e894 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FirstTest.java @@ -0,0 +1,140 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static com.google.cloud.dataflow.sdk.TestUtils.LINES; +import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +/** + * Tests for First. + */ +@RunWith(JUnit4.class) +public class FirstTest + implements Serializable /* to allow anon inner classes */ { + // PRE: lines contains no duplicates. + void runTestFirst(final List lines, int limit, boolean ordered) { + Pipeline p = TestPipeline.create(); + + PCollection input = p.apply(Create.of(lines)) + .setCoder(StringUtf8Coder.of()); + + if (ordered) { + input.setOrdered(true); + } + + PCollection output = + input.apply(First.of(limit)); + + if (ordered) { + output.setOrdered(true); + } + + final int expectedSize = Math.min(limit, lines.size()); + if (ordered) { + List expected = lines.subList(0, expectedSize); + if (expected.isEmpty()) { + DataflowAssert.that(output) + .containsInAnyOrder(expected); + } else { + DataflowAssert.that(output) + .containsInOrder(expected); + } + } else { + DataflowAssert.that(output) + .satisfies(new SerializableFunction, Void>() { + @Override + public Void apply(Iterable actualIter) { + // Make sure actual is the right length, and is a + // subset of expected. + List actual = new ArrayList<>(); + for (String s : actualIter) { + actual.add(s); + } + assertEquals(expectedSize, actual.size()); + Set actualAsSet = new TreeSet<>(actual); + Set linesAsSet = new TreeSet<>(lines); + assertEquals(actual.size(), actualAsSet.size()); + assertEquals(lines.size(), linesAsSet.size()); + assertTrue(linesAsSet.containsAll(actualAsSet)); + return null; + } + }); + } + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testFirst() { + runTestFirst(LINES, 0, false); + runTestFirst(LINES, LINES.size() / 2, false); + runTestFirst(LINES, LINES.size() * 2, false); + } + + @Test + // Extra tests, not worth the time to run on the real service. + public void testFirstMore() { + runTestFirst(LINES, LINES.size() - 1, false); + runTestFirst(LINES, LINES.size(), false); + runTestFirst(LINES, LINES.size() + 1, false); + } + + // TODO: setOrdered(true) isn't supported yet by the Dataflow service. + @Test + public void testFirstOrdered() { + runTestFirst(LINES, 0, true); + runTestFirst(LINES, LINES.size() / 2, true); + runTestFirst(LINES, LINES.size() - 1, true); + runTestFirst(LINES, LINES.size(), true); + runTestFirst(LINES, LINES.size() + 1, true); + runTestFirst(LINES, LINES.size() * 2, true); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testFirstEmpty() { + runTestFirst(NO_LINES, 0, false); + runTestFirst(NO_LINES, 1, false); + } + + @Test + // TODO: setOrdered(true) isn't supported yet by the Dataflow service. + public void testFirstEmptyOrdered() { + runTestFirst(NO_LINES, 0, true); + runTestFirst(NO_LINES, 1, true); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FlattenTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FlattenTest.java new file mode 100644 index 0000000000000..70cc4f1eaf88d --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FlattenTest.java @@ -0,0 +1,244 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static com.google.cloud.dataflow.sdk.TestUtils.LINES; +import static com.google.cloud.dataflow.sdk.TestUtils.LINES2; +import static com.google.cloud.dataflow.sdk.TestUtils.LINES_ARRAY; +import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES; +import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES_ARRAY; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; + +import org.joda.time.Duration; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.List; + +/** + * Tests for Flatten. + */ +@RunWith(JUnit4.class) +public class FlattenTest { + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testFlattenPCollectionListUnordered() { + Pipeline p = TestPipeline.create(); + + List[] inputs = new List[] { + LINES, NO_LINES, LINES2, NO_LINES, LINES, NO_LINES }; + + PCollection output = + makePCollectionListOfStrings(false /* not ordered */, p, inputs) + .apply(Flatten.pCollections()); + + DataflowAssert.that(output).containsInAnyOrder(flatten(inputs)); + p.run(); + } + + // TODO: setOrdered(true) isn't supported yet by the Dataflow service. + @Test + public void testFlattenPCollectionListOrdered() { + Pipeline p = TestPipeline.create(); + + List[] inputs = new List[] { + LINES, NO_LINES, LINES2, NO_LINES, LINES, NO_LINES }; + + PCollection output = + makePCollectionListOfStrings(true /* ordered */, p, inputs) + .apply(Flatten.pCollections()).setOrdered(true); + + DataflowAssert.that(output).containsInOrder(flatten(inputs)); + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testFlattenPCollectionListEmpty() { + Pipeline p = TestPipeline.create(); + + PCollection output = + PCollectionList.empty(p) + .apply(Flatten.pCollections()).setCoder(StringUtf8Coder.of()); + + DataflowAssert.that(output).containsInAnyOrder(); + p.run(); + } + + @Test + public void testWindowingFnPropagationFailure() { + Pipeline p = TestPipeline.create(); + + PCollection input1 = + p.apply(Create.of("Input1")) + .apply(Window.into(FixedWindows.of(Duration.standardMinutes(1)))); + PCollection input2 = + p.apply(Create.of("Input2")) + .apply(Window.into(FixedWindows.of(Duration.standardMinutes(2)))); + + try { + PCollection output = + PCollectionList.of(input1).and(input2) + .apply(Flatten.create()); + Assert.fail("Exception should have been thrown"); + } catch (IllegalStateException e) { + Assert.assertTrue(e.getMessage().startsWith( + "Inputs to Flatten had incompatible window windowingFns")); + } + } + + @Test + public void testWindowingFnPropagation() { + Pipeline p = TestPipeline.create(); + + PCollection input1 = + p.apply(Create.of("Input1")) + .apply(Window.into(FixedWindows.of(Duration.standardMinutes(1)))); + PCollection input2 = + p.apply(Create.of("Input2")) + .apply(Window.into(FixedWindows.of(Duration.standardMinutes(1)))); + + PCollection output = + PCollectionList.of(input1).and(input2) + .apply(Flatten.create()); + + p.run(); + + Assert.assertTrue(output.getWindowingFn().isCompatible( + FixedWindows.of(Duration.standardMinutes(1)))); + } + + @Test + public void testEqualWindowingFnPropagation() { + Pipeline p = TestPipeline.create(); + + PCollection input1 = + p.apply(Create.of("Input1")) + .apply(Window.into(Sessions.withGapDuration(Duration.standardMinutes(1)))); + PCollection input2 = + p.apply(Create.of("Input2")) + .apply(Window.into(Sessions.withGapDuration(Duration.standardMinutes(2)))); + + PCollection output = + PCollectionList.of(input1).and(input2) + .apply(Flatten.create()); + + p.run(); + + Assert.assertTrue(output.getWindowingFn().isCompatible( + Sessions.withGapDuration(Duration.standardMinutes(2)))); + } + + + PCollectionList makePCollectionListOfStrings(boolean ordered, + Pipeline p, + List... lists) { + return makePCollectionList(ordered, p, StringUtf8Coder.of(), lists); + } + + PCollectionList makePCollectionList(boolean ordered, + Pipeline p, + Coder coder, + List... lists) { + List> pcs = new ArrayList<>(); + for (List list : lists) { + PCollection pc = p.apply(Create.of(list)).setCoder(coder); + if (ordered) { + pc.setOrdered(true); + } + pcs.add(pc); + } + return PCollectionList.of(pcs); + } + + T[] flatten(List... lists) { + List flattened = new ArrayList<>(); + for (List list : lists) { + flattened.addAll(list); + } + return flattened.toArray((T[]) new Object[flattened.size()]); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testFlattenIterables() { + Pipeline p = TestPipeline.create(); + + PCollection> input = p + .apply(Create.>of(LINES)) + .setCoder(IterableCoder.of(StringUtf8Coder.of())); + + PCollection output = + input.apply(Flatten.iterables()); + + DataflowAssert.that(output) + .containsInAnyOrder(LINES_ARRAY); + + p.run(); + } + + @Test + public void testFlattenIterablesOrdered() { + Pipeline p = TestPipeline.create(); + + PCollection> input = p + .apply(Create.>of(LINES)) + .setCoder(IterableCoder.of(StringUtf8Coder.of())); + + PCollection output = + input.apply(Flatten.iterables()).setOrdered(true); + + DataflowAssert.that(output) + .containsInOrder(LINES_ARRAY); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testFlattenIterablesEmpty() { + Pipeline p = TestPipeline.create(); + + PCollection> input = p + .apply(Create.>of(NO_LINES)) + .setCoder(IterableCoder.of(StringUtf8Coder.of())); + + PCollection output = + input.apply(Flatten.iterables()); + + DataflowAssert.that(output) + .containsInAnyOrder(NO_LINES_ARRAY); + + p.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java new file mode 100644 index 0000000000000..ebb141f38b7b6 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java @@ -0,0 +1,280 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static com.google.cloud.dataflow.sdk.TestUtils.KvMatcher.isKv; +import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; +import static org.hamcrest.core.Is.is; +import static org.junit.Assert.assertThat; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.MapCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.hamcrest.Matchers; +import org.joda.time.Duration; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +/** + * Tests for GroupByKey. + */ +@RunWith(JUnit4.class) +public class GroupByKeyTest { + + @Rule + public ExpectedException expectedEx = ExpectedException.none(); + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testGroupByKey() { + List> ungroupedPairs = Arrays.asList( + KV.of("k1", 3), + KV.of("k5", Integer.MAX_VALUE), + KV.of("k5", Integer.MIN_VALUE), + KV.of("k2", 66), + KV.of("k1", 4), + KV.of("k2", -33), + KV.of("k3", 0)); + + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.of(ungroupedPairs)) + .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + + PCollection>> output = + input.apply(GroupByKey.create()); + + DataflowAssert.that(output) + .satisfies(new AssertThatHasExpectedContentsForTestGroupByKey()); + + p.run(); + } + + static class AssertThatHasExpectedContentsForTestGroupByKey + implements SerializableFunction>>, + Void> { + @Override + public Void apply(Iterable>> actual) { + assertThat(actual, containsInAnyOrder( + isKv(is("k1"), containsInAnyOrder(3, 4)), + isKv(is("k5"), containsInAnyOrder(Integer.MAX_VALUE, + Integer.MIN_VALUE)), + isKv(is("k2"), containsInAnyOrder(66, -33)), + isKv(is("k3"), containsInAnyOrder(0)))); + return null; + } + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testGroupByKeyAndWindows() { + List> ungroupedPairs = Arrays.asList( + KV.of("k1", 3), // window [0, 5) + KV.of("k5", Integer.MAX_VALUE), // window [0, 5) + KV.of("k5", Integer.MIN_VALUE), // window [0, 5) + KV.of("k2", 66), // window [0, 5) + KV.of("k1", 4), // window [5, 10) + KV.of("k2", -33), // window [5, 10) + KV.of("k3", 0)); // window [5, 10) + + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.timestamped(ungroupedPairs, Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L))) + .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + PCollection>> output = + input.apply(Window.>into(FixedWindows.of(new Duration(5)))) + .apply(GroupByKey.create()); + + DataflowAssert.that(output) + .satisfies(new AssertThatHasExpectedContentsForTestGroupByKeyAndWindows()); + + p.run(); + } + + static class AssertThatHasExpectedContentsForTestGroupByKeyAndWindows + implements SerializableFunction>>, + Void> { + @Override + public Void apply(Iterable>> actual) { + assertThat(actual, containsInAnyOrder( + isKv(is("k1"), containsInAnyOrder(3)), + isKv(is("k1"), containsInAnyOrder(4)), + isKv(is("k5"), containsInAnyOrder(Integer.MAX_VALUE, + Integer.MIN_VALUE)), + isKv(is("k2"), containsInAnyOrder(66)), + isKv(is("k2"), containsInAnyOrder(-33)), + isKv(is("k3"), containsInAnyOrder(0)))); + return null; + } + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testGroupByKeyEmpty() { + List> ungroupedPairs = Arrays.asList(); + + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.of(ungroupedPairs)) + .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + + PCollection>> output = + input.apply(GroupByKey.create()); + + DataflowAssert.that(output) + .containsInAnyOrder(); + + p.run(); + } + + @Test + public void testGroupByKeyNonDeterministic() throws Exception { + expectedEx.expect(IllegalStateException.class); + expectedEx.expectMessage(Matchers.containsString("must be deterministic")); + + List, Integer>> ungroupedPairs = Arrays.asList(); + + Pipeline p = TestPipeline.create(); + + PCollection, Integer>> input = + p.apply(Create.of(ungroupedPairs)) + .setCoder( + KvCoder.of(MapCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()), + BigEndianIntegerCoder.of())); + + input.apply(GroupByKey., Integer>create()); + + p.run(); + } + + @Test + public void testIdentityWindowingFnPropagation() { + Pipeline p = TestPipeline.create(); + + List> ungroupedPairs = Arrays.asList(); + + PCollection> input = + p.apply(Create.of(ungroupedPairs)) + .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())) + .apply(Window.>into(FixedWindows.of(Duration.standardMinutes(1)))); + + PCollection>> output = + input.apply(GroupByKey.create()); + + p.run(); + + Assert.assertTrue(output.getWindowingFn().isCompatible( + FixedWindows.>of(Duration.standardMinutes(1)))); + + } + + @Test + public void testWindowingFnInvalidation() { + Pipeline p = TestPipeline.create(); + + List> ungroupedPairs = Arrays.asList(); + + PCollection> input = + p.apply(Create.of(ungroupedPairs)) + .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())) + .apply(Window.>into( + Sessions.withGapDuration(Duration.standardMinutes(1)))); + + PCollection>> output = + input.apply(GroupByKey.create()); + + p.run(); + + Assert.assertTrue( + output.getWindowingFn().isCompatible( + new InvalidWindowingFn( + "Invalid", + Sessions.>withGapDuration( + Duration.standardMinutes(1))))); + } + + @Test + public void testInvalidWindowingFn() { + Pipeline p = TestPipeline.create(); + + List> ungroupedPairs = Arrays.asList(); + + PCollection> input = + p.apply(Create.of(ungroupedPairs)) + .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())) + .apply(Window.>into( + Sessions.withGapDuration(Duration.standardMinutes(1)))); + + try { + PCollection>>> output = input + .apply(GroupByKey.create()) + .apply(GroupByKey.>create()); + Assert.fail("Exception should have been thrown"); + } catch (IllegalStateException e) { + Assert.assertTrue(e.getMessage().startsWith( + "GroupByKey must have a valid Window merge function.")); + } + } + + @Test + public void testRemerge() { + Pipeline p = TestPipeline.create(); + + List> ungroupedPairs = Arrays.asList(); + + PCollection> input = + p.apply(Create.of(ungroupedPairs)) + .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())) + .apply(Window.>into( + Sessions.withGapDuration(Duration.standardMinutes(1)))); + + PCollection>>> middle = input + .apply(GroupByKey.create()) + .apply(Window.>>remerge()) + .apply(GroupByKey.>create()) + .apply(Window.>>>remerge()); + + p.run(); + + Assert.assertTrue( + middle.getWindowingFn().isCompatible( + Sessions.withGapDuration(Duration.standardMinutes(1)))); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KeysTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KeysTest.java new file mode 100644 index 0000000000000..1d6e233adef85 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KeysTest.java @@ -0,0 +1,100 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; + +/** + * Tests for Keys transform. + */ +@RunWith(JUnit4.class) +public class KeysTest { + static final KV[] TABLE = new KV[] { + KV.of("one", 1), + KV.of("two", 2), + KV.of("three", 3), + KV.of("dup", 4), + KV.of("dup", 5) + }; + + static final KV[] EMPTY_TABLE = new KV[] { + }; + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testKeys() { + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.of(Arrays.asList(TABLE))).setCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + + PCollection output = input.apply(Keys.create()); + DataflowAssert.that(output) + .containsInAnyOrder("one", "two", "three", "dup", "dup"); + + p.run(); + } + + // TODO: setOrdered(true) isn't supported yet by the Dataflow service. + @Test + public void testKeysOrdered() { + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.of(Arrays.asList(TABLE))).setCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + + input.setOrdered(true); + PCollection output = + input.apply(Keys.create()).setOrdered(true); + DataflowAssert.that(output) + .containsInOrder("one", "two", "three", "dup", "dup"); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testKeysEmpty() { + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.of(Arrays.asList(EMPTY_TABLE))).setCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + + PCollection output = input.apply(Keys.create()); + DataflowAssert.that(output) + .containsInAnyOrder(); + + p.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KvSwapTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KvSwapTest.java new file mode 100644 index 0000000000000..15c2ff2ff7366 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KvSwapTest.java @@ -0,0 +1,112 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; + +/** + * Tests for KvSwap transform. + */ +@RunWith(JUnit4.class) +public class KvSwapTest { + static final KV[] TABLE = new KV[] { + KV.of("one", 1), + KV.of("two", 2), + KV.of("three", 3), + KV.of("four", 4), + KV.of("dup", 4), + KV.of("dup", 5) + }; + + static final KV[] EMPTY_TABLE = new KV[] { + }; + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testKvSwap() { + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.of(Arrays.asList(TABLE))).setCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + + PCollection> output = input.apply( + KvSwap.create()); + + DataflowAssert.that(output).containsInAnyOrder( + KV.of(1, "one"), + KV.of(2, "two"), + KV.of(3, "three"), + KV.of(4, "four"), + KV.of(4, "dup"), + KV.of(5, "dup")); + p.run(); + } + + // TODO: setOrdered(true) isn't supported yet by the Dataflow service. + @Test + public void testKvSwapOrdered() { + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.of(Arrays.asList(TABLE))).setCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + + input.setOrdered(true); + PCollection> output = input.apply( + KvSwap.create()).setOrdered(true); + + DataflowAssert.that(output).containsInOrder( + KV.of(1, "one"), + KV.of(2, "two"), + KV.of(3, "three"), + KV.of(4, "four"), + KV.of(4, "dup"), + KV.of(5, "dup")); + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testKvSwapEmpty() { + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.of(Arrays.asList(EMPTY_TABLE))).setCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + + PCollection> output = input.apply( + KvSwap.create()); + + DataflowAssert.that(output).containsInAnyOrder(); + p.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java new file mode 100644 index 0000000000000..7e46bb31a7858 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java @@ -0,0 +1,986 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static com.google.cloud.dataflow.sdk.TestUtils.createInts; +import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray; +import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString; +import static com.google.cloud.dataflow.sdk.util.StringUtils.jsonStringToByteArray; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; +import static org.hamcrest.collection.IsIterableContainingInOrder.contains; +import static org.hamcrest.core.AnyOf.anyOf; +import static org.hamcrest.core.IsEqual.equalTo; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.TestUtils; +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionTuple; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TupleTagList; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * Tests for ParDo. + */ +@RunWith(JUnit4.class) +public class ParDoTest implements Serializable { + // This test is Serializable, just so that it's easy to have + // anonymous inner classes inside the non-static test methods. + + @Rule + public transient ExpectedException thrown = ExpectedException.none(); + + + static class TestDoFn extends DoFn { + enum State { UNSTARTED, STARTED, PROCESSING, FINISHED } + State state = State.UNSTARTED; + + final List> sideInputViews = new ArrayList<>(); + final List> sideOutputTupleTags = new ArrayList<>(); + + public TestDoFn() { + } + + public TestDoFn(List> sideInputViews, + List> sideOutputTupleTags) { + this.sideInputViews.addAll(sideInputViews); + this.sideOutputTupleTags.addAll(sideOutputTupleTags); + } + + @Override + public void startBundle(Context c) { + assertEquals(State.UNSTARTED, state); + state = State.STARTED; + outputToAll(c, "started"); + } + + @Override + public void processElement(ProcessContext c) { + assertThat(state, + anyOf(equalTo(State.STARTED), equalTo(State.PROCESSING))); + state = State.PROCESSING; + outputToAll(c, "processing: " + c.element()); + } + + @Override + public void finishBundle(Context c) { + assertThat(state, + anyOf(equalTo(State.STARTED), equalTo(State.PROCESSING))); + state = State.FINISHED; + outputToAll(c, "finished"); + } + + private void outputToAll(Context c, String value) { + if (!sideInputViews.isEmpty()) { + List sideInputValues = new ArrayList<>(); + for (PCollectionView sideInputView : sideInputViews) { + sideInputValues.add(c.sideInput(sideInputView)); + } + value += ": " + sideInputValues; + } + c.output(value); + for (TupleTag sideOutputTupleTag : sideOutputTupleTags) { + c.sideOutput(sideOutputTupleTag, + sideOutputTupleTag.getId() + ": " + value); + } + } + + /** DataflowAssert "matcher" for expected output. */ + static class HasExpectedOutput + implements SerializableFunction, Void>, Serializable { + private final List inputs; + private final List sideInputs; + private final String sideOutput; + private final boolean ordered; + + public static HasExpectedOutput forInput(List inputs) { + return new HasExpectedOutput( + new ArrayList(inputs), + new ArrayList(), + "", + false); + } + + private HasExpectedOutput(List inputs, + List sideInputs, + String sideOutput, + boolean ordered) { + this.inputs = inputs; + this.sideInputs = sideInputs; + this.sideOutput = sideOutput; + this.ordered = ordered; + } + + public HasExpectedOutput andSideInputs(Integer... sideInputValues) { + List sideInputs = new ArrayList<>(); + for (Integer sideInputValue : sideInputValues) { + sideInputs.add(sideInputValue); + } + return new HasExpectedOutput(inputs, sideInputs, sideOutput, ordered); + } + + public HasExpectedOutput fromSideOutput(TupleTag sideOutputTag) { + return fromSideOutput(sideOutputTag.getId()); + } + public HasExpectedOutput fromSideOutput(String sideOutput) { + return new HasExpectedOutput(inputs, sideInputs, sideOutput, ordered); + } + + public HasExpectedOutput inOrder() { + return new HasExpectedOutput(inputs, sideInputs, sideOutput, true); + } + + @Override + public Void apply(Iterable outputs) { + List starteds = new ArrayList<>(); + List processeds = new ArrayList<>(); + List finisheds = new ArrayList<>(); + for (String output : outputs) { + if (output.contains("started")) { + starteds.add(output); + } else if (output.contains("finished")) { + finisheds.add(output); + } else { + processeds.add(output); + } + } + + String sideInputsSuffix; + if (sideInputs.isEmpty()) { + sideInputsSuffix = ""; + } else { + sideInputsSuffix = ": " + sideInputs; + } + + String sideOutputPrefix; + if (sideOutput.isEmpty()) { + sideOutputPrefix = ""; + } else { + sideOutputPrefix = sideOutput + ": "; + } + + List expectedProcesseds = new ArrayList<>(); + for (Integer input : inputs) { + expectedProcesseds.add( + sideOutputPrefix + "processing: " + input + sideInputsSuffix); + } + String[] expectedProcessedsArray = + expectedProcesseds.toArray(new String[expectedProcesseds.size()]); + if (!ordered || expectedProcesseds.isEmpty()) { + assertThat(processeds, containsInAnyOrder(expectedProcessedsArray)); + } else { + assertThat(processeds, contains(expectedProcessedsArray)); + } + + assertEquals(starteds.size(), finisheds.size()); + assertTrue(starteds.size() > 0); + for (String started : starteds) { + assertEquals(sideOutputPrefix + "started" + sideInputsSuffix, + started); + } + for (String finished : finisheds) { + assertEquals(sideOutputPrefix + "finished" + sideInputsSuffix, + finished); + } + + return null; + } + } + } + + static class TestStartBatchErrorDoFn extends DoFn { + @Override + public void startBundle(Context c) { + throw new RuntimeException("test error in initialize"); + } + + @Override + public void processElement(ProcessContext c) { + // This has to be here. + } + } + + static class TestProcessElementErrorDoFn extends DoFn { + @Override + public void processElement(ProcessContext c) { + throw new RuntimeException("test error in process"); + } + } + + static class TestFinishBatchErrorDoFn extends DoFn { + @Override + public void processElement(ProcessContext c) { + // This has to be here. + } + + @Override + public void finishBundle(Context c) { + throw new RuntimeException("test error in finalize"); + } + } + + static class TestUnexpectedKeyedStateDoFn extends DoFn { + @Override + public void processElement(ProcessContext c) { + // Will fail since this DoFn doesn't implement RequiresKeyedState. + c.keyedState(); + } + } + + private static class StrangelyNamedDoer extends DoFn { + @Override + public void processElement(ProcessContext c) { + } + } + + static class TestOutputTimestampDoFn extends DoFn { + @Override + public void processElement(ProcessContext c) { + Integer value = c.element(); + c.outputWithTimestamp(value, new Instant(value.longValue())); + } + } + + static class TestShiftTimestampDoFn extends DoFn { + private Duration allowedTimestampSkew; + private Duration durationToShift; + + public TestShiftTimestampDoFn(Duration allowedTimestampSkew, + Duration durationToShift) { + this.allowedTimestampSkew = allowedTimestampSkew; + this.durationToShift = durationToShift; + } + + @Override + public Duration getAllowedTimestampSkew() { + return allowedTimestampSkew; + } + @Override + public void processElement(ProcessContext c) { + Instant timestamp = c.timestamp(); + Preconditions.checkNotNull(timestamp); + Integer value = c.element(); + c.outputWithTimestamp(value, timestamp.plus(durationToShift)); + } + } + + static class TestFormatTimestampDoFn extends DoFn { + @Override + public void processElement(ProcessContext c) { + Preconditions.checkNotNull(c.timestamp()); + c.output("processing: " + c.element() + ", timestamp: " + c.timestamp().getMillis()); + } + } + + static class MultiFilter + extends PTransform, PCollectionTuple> { + + private static final TupleTag BY2 = new TupleTag("by2"){}; + private static final TupleTag BY3 = new TupleTag("by3"){}; + + @Override + public PCollectionTuple apply(PCollection input) { + PCollection by2 = input.apply(ParDo.of(new FilterFn(2))); + PCollection by3 = input.apply(ParDo.of(new FilterFn(3))); + return PCollectionTuple.of(BY2, by2).and(BY3, by3); + } + + static class FilterFn extends DoFn { + private final int divisor; + + FilterFn(int divisor) { + this.divisor = divisor; + } + + @Override + public void processElement(ProcessContext c) throws Exception { + if (c.element() % divisor == 0) { + c.output(c.element()); + } + } + } + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testParDo() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollection input = createInts(p, inputs); + + PCollection output = + input + .apply(ParDo.of(new TestDoFn())); + + DataflowAssert.that(output) + .satisfies(TestDoFn.HasExpectedOutput.forInput(inputs)); + + p.run(); + } + + // TODO: setOrdered(true) isn't supported yet by the Dataflow service. + @Test + public void testParDoOrdered() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollection input = createInts(p, inputs).setOrdered(true); + + PCollection output = + input + .apply(ParDo.of(new TestDoFn())).setOrdered(true); + + DataflowAssert.that(output) + .satisfies(TestDoFn.HasExpectedOutput.forInput(inputs).inOrder()); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testParDoEmpty() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(); + + PCollection input = createInts(p, inputs); + + PCollection output = + input + .apply(ParDo.of(new TestDoFn())); + + DataflowAssert.that(output) + .satisfies(TestDoFn.HasExpectedOutput.forInput(inputs)); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testParDoWithSideOutputs() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollection input = createInts(p, inputs); + + TupleTag mainTag = new TupleTag("main"){}; + TupleTag sideTag1 = new TupleTag("side1"){}; + TupleTag sideTag2 = new TupleTag("side2"){}; + TupleTag sideTag3 = new TupleTag("side3"){}; + TupleTag sideTagUnwritten = new TupleTag("sideUnwritten"){}; + + PCollectionTuple outputs = + input + .apply(ParDo + .of(new TestDoFn( + Arrays.>asList(), + Arrays.asList(sideTag1, sideTag2, sideTag3))) + .withOutputTags( + mainTag, + TupleTagList.of(sideTag3).and(sideTag1) + .and(sideTagUnwritten).and(sideTag2))); + + DataflowAssert.that(outputs.get(mainTag)) + .satisfies(TestDoFn.HasExpectedOutput.forInput(inputs)); + + DataflowAssert.that(outputs.get(sideTag1)) + .satisfies(TestDoFn.HasExpectedOutput.forInput(inputs) + .fromSideOutput(sideTag1)); + DataflowAssert.that(outputs.get(sideTag2)) + .satisfies(TestDoFn.HasExpectedOutput.forInput(inputs) + .fromSideOutput(sideTag2)); + DataflowAssert.that(outputs.get(sideTag3)) + .satisfies(TestDoFn.HasExpectedOutput.forInput(inputs) + .fromSideOutput(sideTag3)); + DataflowAssert.that(outputs.get(sideTagUnwritten)).containsInAnyOrder(); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testParDoWithOnlySideOutputs() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollection input = createInts(p, inputs); + + final TupleTag mainTag = new TupleTag("main"){}; + final TupleTag sideTag = new TupleTag("side"){}; + + PCollectionTuple outputs = input.apply( + ParDo + .withOutputTags(mainTag, TupleTagList.of(sideTag)) + .of(new DoFn(){ + @Override + public void processElement(ProcessContext c) { + c.sideOutput(sideTag, c.element()); + }})); + + DataflowAssert.that(outputs.get(mainTag)).containsInAnyOrder(); + DataflowAssert.that(outputs.get(sideTag)).containsInAnyOrder(inputs); + + p.run(); + } + + @Test + public void testParDoWritingToUndeclaredSideOutput() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollection input = createInts(p, inputs); + + TupleTag sideTag = new TupleTag("side"){}; + + PCollection output = + input + .apply(ParDo.of(new TestDoFn( + Arrays.>asList(), + Arrays.asList(sideTag)))); + + DataflowAssert.that(output) + .satisfies(TestDoFn.HasExpectedOutput.forInput(inputs)); + + p.run(); + } + + @Test + public void testParDoUndeclaredSideOutputLimit() { + Pipeline p = TestPipeline.create(); + PCollection input = createInts(p, Arrays.asList(3)); + + // Success for a total of 1000 outputs. + input + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + TupleTag specialSideTag = new TupleTag(){}; + c.sideOutput(specialSideTag, "side"); + c.sideOutput(specialSideTag, "side"); + c.sideOutput(specialSideTag, "side"); + + for (int i = 0; i < 998; i++) { + c.sideOutput(new TupleTag(){}, "side"); + } + }})); + p.run(); + + // Failure for a total of 1001 outputs. + input + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + for (int i = 0; i < 1000; i++) { + c.sideOutput(new TupleTag(){}, "side"); + } + }})); + try { + p.run(); + fail("should have failed"); + } catch (RuntimeException exn) { + assertThat(exn.toString(), + containsString("the number of side outputs has exceeded a limit")); + } + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testParDoWithSideInputs() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollection input = createInts(p, inputs); + + PCollectionView sideInput1 = TestUtils.createSingletonInt(p, 11); + PCollectionView sideInputUnread = TestUtils.createSingletonInt(p, -3333); + PCollectionView sideInput2 = TestUtils.createSingletonInt(p, 222); + + PCollection output = + input + .apply(ParDo + .withSideInputs(sideInput1, sideInputUnread, sideInput2) + .of(new TestDoFn( + Arrays.asList(sideInput1, sideInput2), + Arrays.>asList()))); + + DataflowAssert.that(output) + .satisfies(TestDoFn.HasExpectedOutput + .forInput(inputs) + .andSideInputs(11, 222)); + + p.run(); + } + + @Test + public void testParDoReadingFromUnknownSideInput() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollection input = createInts(p, inputs); + + PCollectionView sideView = TestUtils.createSingletonInt(p, 3); + + input + .apply(ParDo.of(new TestDoFn( + Arrays.>asList(sideView), + Arrays.>asList()))); + + try { + p.run(); + fail("should have failed"); + } catch (RuntimeException exn) { + assertThat(exn.toString(), + containsString("calling sideInput() with unknown view")); + } + } + + @Test + public void testParDoWithErrorInStartBatch() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollection input = createInts(p, inputs); + + input + .apply(ParDo.of(new TestStartBatchErrorDoFn())); + + try { + p.run(); + fail("should have failed"); + } catch (RuntimeException exn) { + assertThat(exn.toString(), containsString("test error in initialize")); + } + } + + @Test + public void testParDoWithErrorInProcessElement() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollection input = createInts(p, inputs); + + input + .apply(ParDo.of(new TestProcessElementErrorDoFn())); + + try { + p.run(); + fail("should have failed"); + } catch (RuntimeException exn) { + assertThat(exn.toString(), containsString("test error in process")); + } + } + + @Test + public void testParDoWithErrorInFinishBatch() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollection input = createInts(p, inputs); + + input + .apply(ParDo.of(new TestFinishBatchErrorDoFn())); + + try { + p.run(); + fail("should have failed"); + } catch (RuntimeException exn) { + assertThat(exn.toString(), containsString("test error in finalize")); + } + } + + @Test + public void testParDoWithUnexpectedKeyedState() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollection input = createInts(p, inputs); + + input + .apply(ParDo.of(new TestUnexpectedKeyedStateDoFn())); + + try { + p.run(); + fail("should have failed"); + } catch (RuntimeException exn) { + assertThat(exn.toString(), + containsString("Keyed state is only available")); + } + } + + @Test + public void testParDoName() { + Pipeline p = TestPipeline.create(); + + PCollection input = + createInts(p, Arrays.asList(3, -42, 666)) + .setName("MyInput"); + + { + PCollection output1 = + input + .apply(ParDo.of(new TestDoFn())); + assertEquals("Test.out", output1.getName()); + } + + { + PCollection output2 = + input + .apply(ParDo.named("MyParDo").of(new TestDoFn())); + assertEquals("MyParDo.out", output2.getName()); + } + + { + PCollection output3 = + input + .apply(ParDo.of(new TestDoFn()).named("HerParDo")); + assertEquals("HerParDo.out", output3.getName()); + } + + { + PCollection output4 = + input + .apply(ParDo.of(new TestDoFn()).named("TestDoFn")); + assertEquals("TestDoFn.out", output4.getName()); + } + + { + PCollection output5 = + input + .apply(ParDo.of(new StrangelyNamedDoer())); + assertEquals("StrangelyNamedDoer.out", + output5.getName()); + } + } + + @Test + public void testParDoWithSideOutputsName() { + Pipeline p = TestPipeline.create(); + + PCollection input = + createInts(p, Arrays.asList(3, -42, 666)) + .setName("MyInput"); + + TupleTag mainTag = new TupleTag("main"){}; + TupleTag sideTag1 = new TupleTag("side1"){}; + TupleTag sideTag2 = new TupleTag("side2"){}; + TupleTag sideTag3 = new TupleTag("side3"){}; + TupleTag sideTagUnwritten = new TupleTag("sideUnwritten"){}; + + PCollectionTuple outputs = + input + .apply(ParDo + .named("MyParDo") + .of(new TestDoFn( + Arrays.>asList(), + Arrays.asList(sideTag1, sideTag2, sideTag3))) + .withOutputTags( + mainTag, + TupleTagList.of(sideTag3).and(sideTag1) + .and(sideTagUnwritten).and(sideTag2))); + + assertEquals("MyParDo.main", outputs.get(mainTag).getName()); + assertEquals("MyParDo.side1", outputs.get(sideTag1).getName()); + assertEquals("MyParDo.side2", outputs.get(sideTag2).getName()); + assertEquals("MyParDo.side3", outputs.get(sideTag3).getName()); + assertEquals("MyParDo.sideUnwritten", + outputs.get(sideTagUnwritten).getName()); + } + + @Test + public void testParDoInCustomTransform() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollection input = createInts(p, inputs); + + PCollection output = + input + .apply(new PTransform, PCollection>() { + @Override + public PCollection apply(PCollection input) { + return input.apply(ParDo.of(new TestDoFn())); + } + }); + + // Test that Coder inference of the result works through + // user-defined PTransforms. + DataflowAssert.that(output) + .satisfies(TestDoFn.HasExpectedOutput.forInput(inputs)); + + p.run(); + } + + @Test + public void testMultiOutputChaining() { + Pipeline p = TestPipeline.create(); + + PCollection input = createInts(p, Arrays.asList(3, 4, 5, 6)); + + PCollectionTuple filters = input.apply(new MultiFilter()); + PCollection by2 = filters.get(MultiFilter.BY2); + PCollection by3 = filters.get(MultiFilter.BY3); + + // Apply additional filters to each operation. + PCollection by2then3 = by2 + .apply(ParDo.of(new MultiFilter.FilterFn(3))); + PCollection by3then2 = by3 + .apply(ParDo.of(new MultiFilter.FilterFn(2))); + + DataflowAssert.that(by2then3).containsInAnyOrder(6); + DataflowAssert.that(by3then2).containsInAnyOrder(6); + p.run(); + } + + @Test + public void testJsonEscaping() { + // Declare an arbitrary function and make sure we can serialize it + DoFn doFn = new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element() + 1); + } + }; + + byte[] serializedBytes = serializeToByteArray(doFn); + String serializedJson = byteArrayToJsonString(serializedBytes); + assertArrayEquals( + serializedBytes, jsonStringToByteArray(serializedJson)); + } + + private static class TestDummy { } + + private static class TestDummyCoder extends AtomicCoder { + private TestDummyCoder() { } + private static final TestDummyCoder INSTANCE = new TestDummyCoder(); + + @JsonCreator + public static TestDummyCoder of() { + return INSTANCE; + } + + public static List getInstanceComponents(TestDummy exampleValue) { + return Collections.emptyList(); + } + + @Override + public void encode(TestDummy value, OutputStream outStream, Context context) + throws CoderException, IOException { + } + + @Override + public TestDummy decode(InputStream inStream, Context context) + throws CoderException, IOException { + return new TestDummy(); + } + + @Override + public boolean isDeterministic() { return true; } + + @Override + public boolean isRegisterByteSizeObserverCheap(TestDummy value, Context context) { + return true; + } + + @Override + public void registerByteSizeObserver( + TestDummy value, ElementByteSizeObserver observer, Context context) + throws Exception { + observer.update(0L); + } + } + + private static class SideOutputDummyFn extends DoFn { + private TupleTag sideTag; + public SideOutputDummyFn(TupleTag sideTag) { + this.sideTag = sideTag; + } + @Override + public void processElement(ProcessContext c) { + c.output(1); + c.sideOutput(sideTag, new TestDummy()); + } + } + + private static class MainOutputDummyFn extends DoFn { + private TupleTag sideTag; + public MainOutputDummyFn(TupleTag sideTag) { + this.sideTag = sideTag; + } + @Override + public void processElement(ProcessContext c) { + c.output(new TestDummy()); + c.sideOutput(sideTag, 1); + } + } + + @Test + public void testSideOutputUnknownCoder() { + Pipeline pipeline = TestPipeline.create(); + PCollection input = pipeline + .apply(Create.of(Arrays.asList(1, 2, 3))); + + // Expect a fail, but it should be a NoCoderException + final TupleTag mainTag = new TupleTag(); + final TupleTag sideTag = new TupleTag(); + input.apply(ParDo.of(new SideOutputDummyFn(sideTag)) + .withOutputTags(mainTag, TupleTagList.of(sideTag))); + + thrown.expect(IllegalStateException.class); + thrown.expectMessage("unable to infer a default Coder"); + pipeline.run(); + } + + @Test + public void testSideOutputUnregisteredExplicitCoder() { + Pipeline pipeline = TestPipeline.create(); + PCollection input = pipeline + .apply(Create.of(Arrays.asList(1, 2, 3))); + + final TupleTag mainTag = new TupleTag(); + final TupleTag sideTag = new TupleTag(); + PCollectionTuple outputTuple = input.apply(ParDo.of(new SideOutputDummyFn(sideTag)) + .withOutputTags(mainTag, TupleTagList.of(sideTag))); + + outputTuple.get(sideTag) + .setCoder(new TestDummyCoder()); + + pipeline.run(); + } + + @Test + public void testMainOutputUnregisteredExplicitCoder() { + Pipeline pipeline = TestPipeline.create(); + PCollection input = pipeline + .apply(Create.of(Arrays.asList(1, 2, 3))); + + final TupleTag mainTag = new TupleTag(); + final TupleTag sideTag = new TupleTag() {}; + PCollectionTuple outputTuple = input.apply(ParDo.of(new MainOutputDummyFn(sideTag)) + .withOutputTags(mainTag, TupleTagList.of(sideTag))); + + outputTuple.get(mainTag) + .setCoder(new TestDummyCoder()); + + pipeline.run(); + } + + @Test + public void testParDoOutputWithTimestamp() { + Pipeline p = TestPipeline.create(); + + PCollection input = + createInts(p, Arrays.asList(3, 42, 6)).setOrdered(true); + + PCollection output = + input + .apply(ParDo.of(new TestOutputTimestampDoFn())) + .apply(ParDo.of(new TestShiftTimestampDoFn(Duration.ZERO, Duration.ZERO))) + .apply(ParDo.of(new TestFormatTimestampDoFn())); + + DataflowAssert.that(output).containsInAnyOrder( + "processing: 3, timestamp: 3", + "processing: 42, timestamp: 42", + "processing: 6, timestamp: 6"); + + p.run(); + } + + @Test + public void testParDoShiftTimestamp() { + Pipeline p = TestPipeline.create(); + + PCollection input = + createInts(p, Arrays.asList(3, 42, 6)).setOrdered(true); + + PCollection output = + input + .apply(ParDo.of(new TestOutputTimestampDoFn())) + .apply(ParDo.of(new TestShiftTimestampDoFn(Duration.millis(1000), + Duration.millis(-1000)))) + .apply(ParDo.of(new TestFormatTimestampDoFn())); + + DataflowAssert.that(output).containsInAnyOrder( + "processing: 3, timestamp: -997", + "processing: 42, timestamp: -958", + "processing: 6, timestamp: -994"); + + p.run(); + } + + @Test + public void testParDoShiftTimestampInvalid() { + Pipeline p = TestPipeline.create(); + + createInts(p, Arrays.asList(3, 42, 6)).setOrdered(true) + .apply(ParDo.of(new TestOutputTimestampDoFn())) + .apply(ParDo.of(new TestShiftTimestampDoFn(Duration.millis(1000), + Duration.millis(-1001)))) + .apply(ParDo.of(new TestFormatTimestampDoFn())); + + try { + p.run(); + fail("should have failed"); + } catch (RuntimeException exn) { + // expected + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java new file mode 100644 index 0000000000000..0d19f082ee07e --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java @@ -0,0 +1,141 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static com.google.cloud.dataflow.sdk.TestUtils.createInts; +import static com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn; +import static org.hamcrest.Matchers.containsString; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; + +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Tests for Partition + */ +@RunWith(JUnit4.class) +public class PartitionTest implements Serializable { + static class ModFn implements PartitionFn { + public int partitionFor(Integer elem, int numPartitions) { + return elem % numPartitions; + } + } + + static class IdentityFn implements PartitionFn { + public int partitionFor(Integer elem, int numPartitions) { + return elem; + } + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testEvenOddPartition() { + TestPipeline p = TestPipeline.create(); + + PCollection input = + createInts(p, Arrays.asList(591, 11789, 1257, 24578, 24799, 307)); + + PCollectionList outputs = input.apply(Partition.of(2, new ModFn())); + assertTrue(outputs.size() == 2); + DataflowAssert.that(outputs.get(0)).containsInAnyOrder(24578); + DataflowAssert.that(outputs.get(1)).containsInAnyOrder(591, 11789, 1257, + 24799, 307); + p.run(); + } + + @Test + public void testModPartition() { + TestPipeline p = TestPipeline.create(); + + PCollection input = + createInts(p, Arrays.asList(1, 2, 4, 5)); + + PCollectionList outputs = input.apply(Partition.of(3, new ModFn())); + assertTrue(outputs.size() == 3); + DataflowAssert.that(outputs.get(0)).containsInAnyOrder(); + DataflowAssert.that(outputs.get(1)).containsInAnyOrder(1, 4); + DataflowAssert.that(outputs.get(2)).containsInAnyOrder(2, 5); + p.run(); + } + + @Test + public void testOutOfBoundsPartitions() { + TestPipeline p = TestPipeline.create(); + + PCollection input = createInts(p, Arrays.asList(-1)); + + PCollectionList outputs = + input.apply(Partition.of(5, new IdentityFn())); + + try { + p.run(); + } catch (RuntimeException e) { + assertThat(e.toString(), containsString( + "Partition function returned out of bounds index: -1 not in [0..5)")); + } + } + + @Test + public void testZeroNumPartitions() { + TestPipeline p = TestPipeline.create(); + + PCollection input = createInts(p, Arrays.asList(591)); + + try { + PCollectionList outputs = + input.apply(Partition.of(0, new IdentityFn())); + fail("should have failed"); + } catch (IllegalArgumentException exn) { + assertThat(exn.toString(), containsString("numPartitions must be > 0")); + } + } + + @Test + public void testDroppedPartition() { + TestPipeline p = TestPipeline.create(); + + PCollection input = createInts(p, + Arrays.asList(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)); + + // Compute the set of integers either 1 or 2 mod 3, the hard way. + PCollectionList outputs = + input.apply(Partition.of(3, new ModFn())); + + List> outputsList = new ArrayList<>(outputs.getAll()); + outputsList.remove(0); + outputs = PCollectionList.of(outputsList); + assertTrue(outputs.size() == 2); + + PCollection output = outputs.apply(Flatten.create()); + DataflowAssert.that(output).containsInAnyOrder(2, 4, 5, 7, 8, 10, 11); + p.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java new file mode 100644 index 0000000000000..d6de05af6d3f4 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java @@ -0,0 +1,225 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static org.hamcrest.Matchers.both; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThan; + +import com.google.cloud.dataflow.sdk.TestUtils; +import com.google.cloud.dataflow.sdk.runners.DirectPipeline; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Tests for RateLimiter. + */ +@RunWith(JUnit4.class) +public class RateLimitingTest { + + /** + * Pass-thru function. + */ + private static class IdentityFn extends DoFn { + @Override + public void processElement(ProcessContext c) { + c.output(c.element()); + } + } + + /** + * Introduces a delay in processing, then passes thru elements. + */ + private static class DelayFn extends DoFn { + public static final long DELAY_MS = 250; + + @Override + public void processElement(ProcessContext c) { + try { + Thread.sleep(DELAY_MS); + } catch (InterruptedException e) { + e.printStackTrace(); + throw new RuntimeException("Interrupted"); + } + c.output(c.element()); + } + } + + /** + * Throws an exception after some number of calls. + */ + private static class ExceptionThrowingFn extends DoFn { + private final AtomicInteger numSuccesses; + private final AtomicInteger numProcessed = new AtomicInteger(); + private final AtomicInteger numFailures = new AtomicInteger(); + + private ExceptionThrowingFn(int numSuccesses) { + this.numSuccesses = new AtomicInteger(numSuccesses); + } + + @Override + public void processElement(ProcessContext c) { + numProcessed.incrementAndGet(); + if (numSuccesses.decrementAndGet() > 0) { + c.output(c.element()); + return; + } + + numFailures.incrementAndGet(); + throw new RuntimeException("Expected failure"); + } + } + + /** + * Measures concurrency of the processElement method. + * + *

Note: this only works when + * {@link DirectPipelineRunner#testSerializability} is disabled, otherwise + * the counters are not available after the run. + */ + private static class ConcurrencyMeasuringFn extends DoFn { + private int concurrentElements = 0; + private int maxConcurrency = 0; + + @Override + public void processElement(ProcessContext c) { + synchronized (this) { + concurrentElements++; + if (concurrentElements > maxConcurrency) { + maxConcurrency = concurrentElements; + } + } + + c.output(c.element()); + + synchronized (this) { + concurrentElements--; + } + } + } + + @Test + public void testRateLimitingMax() { + int n = 10; + double rate = 10.0; + long duration = runWithRate(n, rate, new IdentityFn()); + + long perElementPause = (long) (1000L / rate); + long minDuration = (n - 1) * perElementPause; + Assert.assertThat(duration, greaterThan(minDuration)); + } + + @Test(timeout = 5000L) + public void testExceptionHandling() { + ExceptionThrowingFn fn = new ExceptionThrowingFn<>(10); + try { + runWithRate(100, 0.0, fn); + Assert.fail("Expected exception to propagate"); + } catch (RuntimeException e) { + Assert.assertThat(e.getMessage(), containsString("Expected failure")); + } + + // Should have processed 10 elements, but stopped before processing all + // of them. + Assert.assertThat(fn.numProcessed.get(), + is(both(greaterThanOrEqualTo(10)) + .and(lessThan(100)))); + + // The first failure should prevent the scheduling of any more elements. + Assert.assertThat(fn.numFailures.get(), + is(both(greaterThanOrEqualTo(1)) + .and(lessThan(RateLimiting.DEFAULT_MAX_PARALLELISM)))); + } + + /** + * Test exception handling on the last element to be processed. + */ + @Test(timeout = 5000L) + public void testExceptionHandling2() { + ExceptionThrowingFn fn = new ExceptionThrowingFn<>(10); + try { + runWithRate(10, 0.0, fn); + Assert.fail("Expected exception to propagate"); + } catch (RuntimeException e) { + Assert.assertThat(e.getMessage(), containsString("Expected failure")); + } + + // Should have processed 10 elements, but stopped before processing all + // of them. + Assert.assertEquals(10, fn.numProcessed.get()); + Assert.assertEquals(1, fn.numFailures.get()); + } + + /** + * Provides more elements than can be scheduled at once, testing that the + * backlog limit is applied. + */ + @Test + public void testBacklogLimiter() { + long duration = runWithRate(2 * RateLimiting.DEFAULT_MAX_PARALLELISM, + -1.0 /* unlimited */, new DelayFn()); + + // Should take > 2x the delay interval, since no more than half the elements + // can be scheduled at once. + Assert.assertThat(duration, + greaterThan(2 * DelayFn.DELAY_MS)); + } + + private long runWithRate(int numElements, double rateLimit, + DoFn doFn) { + DirectPipeline p = DirectPipeline.createForTest(); + // Run with serializability testing disabled so that our tests can inspect + // the DoFns after the test. + p.getRunner().withSerializabilityTesting(false); + + ArrayList data = new ArrayList<>(numElements); + for (int i = 0; i < numElements; ++i) { + data.add(i); + } + + PCollection input = TestUtils.createInts(p, data); + + ConcurrencyMeasuringFn downstream = new ConcurrencyMeasuringFn<>(); + + PCollection output = input + .apply(RateLimiting.perWorker(doFn) + .withRateLimit(rateLimit)) + .apply(ParDo + .of(downstream)); + + long startTime = System.currentTimeMillis(); + + DirectPipelineRunner.EvaluationResults results = p.run(); + + // Downstream methods should not see parallel threads. + Assert.assertEquals(1, downstream.maxConcurrency); + + long endTime = System.currentTimeMillis(); + return endTime - startTime; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicatesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicatesTest.java new file mode 100644 index 0000000000000..a44fa2d39103c --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicatesTest.java @@ -0,0 +1,82 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Tests for RemovedDuplicates. + */ +@RunWith(JUnit4.class) +public class RemoveDuplicatesTest { + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testRemoveDuplicates() { + List strings = Arrays.asList( + "k1", + "k5", + "k5", + "k2", + "k1", + "k2", + "k3"); + + Pipeline p = TestPipeline.create(); + + PCollection input = + p.apply(Create.of(strings)) + .setCoder(StringUtf8Coder.of()); + + PCollection output = + input.apply(RemoveDuplicates.create()); + + DataflowAssert.that(output) + .containsInAnyOrder("k1", "k5", "k2", "k3"); + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testRemoveDuplicatesEmpty() { + List strings = Arrays.asList(); + + Pipeline p = TestPipeline.create(); + + PCollection input = + p.apply(Create.of(strings)) + .setCoder(StringUtf8Coder.of()); + + PCollection output = + input.apply(RemoveDuplicates.create()); + + DataflowAssert.that(output) + .containsInAnyOrder(); + p.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java new file mode 100644 index 0000000000000..7c51d096fe4e1 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java @@ -0,0 +1,175 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.google.api.client.util.Joiner; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * Tests for Sample transform. + */ +@RunWith(JUnit4.class) +public class SampleTest { + static final Integer[] EMPTY = new Integer[] { }; + static final Integer[] DATA = new Integer[] {1, 2, 3, 4, 5}; + static final Integer[] REPEATED_DATA = new Integer[] {1, 1, 2, 2, 3, 3, 4, 4, 5, 5}; + + /** + * Verifies that the result of a Sample operation contains the expected number of elements, + * and that those elements are a subset of the items in expected. + */ + public static class VerifyCorrectSample + implements SerializableFunction, Void> { + private T[] expectedValues; + private int expectedSize; + + /** + * expectedSize is the number of elements that the Sample should contain. expected is the set + * of elements that the sample may contain. + */ + VerifyCorrectSample(int expectedSize, T... expected) { + this.expectedValues = expected; + this.expectedSize = expectedSize; + } + + @Override + public Void apply(Iterable in) { + List actual = new ArrayList<>(); + for (T elem : in) { + actual.add(elem); + } + + assertEquals(expectedSize, actual.size()); + + Collections.sort(actual); // We assume that @expected is already sorted. + int i = 0; // Index into @expected + for (T s : actual) { + boolean matchFound = false; + for (; i < expectedValues.length; i++) { + if (s.equals(expectedValues[i])) { + matchFound = true; + break; + } + } + assertTrue("Invalid sample: " + Joiner.on(',').join(actual), matchFound); + i++; // Don't match the same element again. + } + return null; + } + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testSample() { + Pipeline p = TestPipeline.create(); + + PCollection input = p.apply(Create.of(DATA)) + .setCoder(BigEndianIntegerCoder.of()); + PCollection> output = input.apply( + Sample.fixedSizeGlobally(3)); + + DataflowAssert.thatSingletonIterable(output) + .satisfies(new VerifyCorrectSample<>(3, DATA)); + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testSampleEmpty() { + Pipeline p = TestPipeline.create(); + + PCollection input = p.apply(Create.of(EMPTY)) + .setCoder(BigEndianIntegerCoder.of()); + PCollection> output = input.apply( + Sample.fixedSizeGlobally(3)); + + DataflowAssert.thatSingletonIterable(output) + .satisfies(new VerifyCorrectSample<>(0, EMPTY)); + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testSampleZero() { + Pipeline p = TestPipeline.create(); + + PCollection input = p.apply(Create.of(DATA)) + .setCoder(BigEndianIntegerCoder.of()); + PCollection> output = input.apply( + Sample.fixedSizeGlobally(0)); + + DataflowAssert.thatSingletonIterable(output) + .satisfies(new VerifyCorrectSample<>(0, DATA)); + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testSampleInsufficientElements() { + Pipeline p = TestPipeline.create(); + + PCollection input = p.apply(Create.of(DATA)) + .setCoder(BigEndianIntegerCoder.of()); + PCollection> output = input.apply( + Sample.fixedSizeGlobally(10)); + + DataflowAssert.thatSingletonIterable(output) + .satisfies(new VerifyCorrectSample<>(5, DATA)); + p.run(); + } + + @Test(expected = IllegalArgumentException.class) + public void testSampleNegative() { + Pipeline p = TestPipeline.create(); + + PCollection input = p.apply(Create.of(DATA)) + .setCoder(BigEndianIntegerCoder.of()); + input.apply(Sample.fixedSizeGlobally(-1)); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testSampleMultiplicity() { + Pipeline p = TestPipeline.create(); + + PCollection input = p.apply(Create.of(REPEATED_DATA)) + .setCoder(BigEndianIntegerCoder.of()); + // At least one value must be selected with multiplicity. + PCollection> output = input.apply( + Sample.fixedSizeGlobally(6)); + + DataflowAssert.thatSingletonIterable(output) + .satisfies(new VerifyCorrectSample<>(6, REPEATED_DATA)); + p.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SimpleStatsFnsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SimpleStatsFnsTest.java new file mode 100644 index 0000000000000..909dcba9981fd --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SimpleStatsFnsTest.java @@ -0,0 +1,130 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static org.junit.Assert.assertEquals; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.DoubleCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.VarLongCoder; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Tests of Min, Max, Mean, and Sum. + */ +@RunWith(JUnit4.class) +public class SimpleStatsFnsTest { + static final double DOUBLE_COMPARISON_ACCURACY = 1e-7; + + private static class TestCase> { + final List data; + final N min; + final N max; + final N sum; + final Double mean; + + public TestCase(N min, N max, N sum, N... values) { + this.data = Arrays.asList(values); + this.min = min; + this.max = max; + this.sum = sum; + this.mean = + values.length == 0 ? 0.0 : sum.doubleValue() / values.length; + } + } + + static final List> DOUBLE_CASES = Arrays.asList( + new TestCase<>(-312.31, 6312.31, 11629.13, + -312.31, 29.13, 112.158, 6312.31, -312.158, -312.158, 112.158, + -312.31, 6312.31, 0.0), + new TestCase<>(3.14, 3.14, 3.14, 3.14), + new TestCase<>(Double.MAX_VALUE, Double.MIN_NORMAL, 0.0)); + + static final List> LONG_CASES = Arrays.asList( + new TestCase<>(-50000000000000000L, + 70000000000000000L, + 60000033123213121L, + 0L, 1L, 10000000000000000L, -50000000000000000L, + 70000000000000000L, 0L, 10000000000000000L, -1L, + -50000000000000000L, 70000000000000000L, 33123213121L), + new TestCase<>(3L, 3L, 3L, 3L), + new TestCase<>(Long.MAX_VALUE, Long.MIN_VALUE, 0L)); + + static final List> INTEGER_CASES = Arrays.asList( + new TestCase<>(-3, 6, 22, + 1, -3, 2, 6, 3, 4, -3, 5, 6, 1), + new TestCase<>(3, 3, 3, 3), + new TestCase<>(Integer.MAX_VALUE, Integer.MIN_VALUE, 0)); + + @Test + public void testDoubleStats() { + for (TestCase t : DOUBLE_CASES) { + assertEquals(t.sum, new Sum.SumDoubleFn().apply(t.data), + DOUBLE_COMPARISON_ACCURACY); + assertEquals(t.min, new Min.MinDoubleFn().apply(t.data), + DOUBLE_COMPARISON_ACCURACY); + assertEquals(t.max, new Max.MaxDoubleFn().apply(t.data), + DOUBLE_COMPARISON_ACCURACY); + assertEquals(t.mean, new Mean.MeanFn().apply(t.data), + DOUBLE_COMPARISON_ACCURACY); + } + } + + @Test + public void testIntegerStats() { + for (TestCase t : INTEGER_CASES) { + assertEquals(t.sum, new Sum.SumIntegerFn().apply(t.data)); + assertEquals(t.min, new Min.MinIntegerFn().apply(t.data)); + assertEquals(t.max, new Max.MaxIntegerFn().apply(t.data)); + assertEquals(t.mean, new Mean.MeanFn().apply(t.data)); + } + } + + @Test + public void testLongStats() { + for (TestCase t : LONG_CASES) { + assertEquals(t.sum, new Sum.SumLongFn().apply(t.data)); + assertEquals(t.min, new Min.MinLongFn().apply(t.data)); + assertEquals(t.max, new Max.MaxLongFn().apply(t.data)); + assertEquals(t.mean, new Mean.MeanFn().apply(t.data)); + } + } + + @Test + public void testMeanCountSumSerializable() { + Pipeline p = TestPipeline.create(); + + PCollection> input = p + .apply(Create.of(KV.of(1L, 1.5), KV.of(2L, 7.3))) + .setCoder(KvCoder.of(VarLongCoder.of(), DoubleCoder.of())); + + PCollection> meanPerKey = + input.apply(Mean.perKey()); + + p.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java new file mode 100644 index 0000000000000..63625a7f5f2b1 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java @@ -0,0 +1,244 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; +import static org.hamcrest.collection.IsIterableContainingInOrder.contains; +import static org.junit.Assert.assertThat; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.runners.DirectPipeline; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.EvaluationResults; +import com.google.cloud.dataflow.sdk.runners.RecordingPipelineVisitor; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.hamcrest.Matchers; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; + +/** Tests for Top */ +@RunWith(JUnit4.class) +public class TopTest { + + @Rule + public ExpectedException expectedEx = ExpectedException.none(); + + @SuppressWarnings("unchecked") + static final String[] COLLECTION = new String[] { + "a", "bb", "c", "c", "z" + }; + + @SuppressWarnings("unchecked") + static final String[] EMPTY_COLLECTION = new String[] { + }; + + @SuppressWarnings("unchecked") + static final KV[] TABLE = new KV[] { + KV.of("a", 1), + KV.of("a", 2), + KV.of("a", 3), + KV.of("b", 1), + KV.of("b", 10), + KV.of("b", 10), + KV.of("b", 100), + }; + + @SuppressWarnings("unchecked") + static final KV[] EMPTY_TABLE = new KV[] { + }; + + public PCollection> createInputTable(Pipeline p) { + return p.apply(Create.of(Arrays.asList(TABLE))).setCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + } + + public PCollection> createEmptyInputTable(Pipeline p) { + return p.apply(Create.of(Arrays.asList(EMPTY_TABLE))).setCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + } + + @Test + @SuppressWarnings("unchecked") + public void testTop() { + DirectPipeline p = DirectPipeline.createForTest(); + PCollection input = + p.apply(Create.of(Arrays.asList(COLLECTION))) + .setCoder(StringUtf8Coder.of()); + + PCollection> top1 = input.apply(Top.of(1, new OrderByLength())); + PCollection> top2 = input.apply(Top.largest(2)); + PCollection> top3 = input.apply(Top.smallest(3)); + + PCollection>> largestPerKey = createInputTable(p) + .apply(Top.largestPerKey(2)); + PCollection>> smallestPerKey = createInputTable(p) + .apply(Top.smallestPerKey(2)); + + EvaluationResults results = p.run(); + + assertThat(results.getPCollection(top1).get(0), contains("bb")); + assertThat(results.getPCollection(top2).get(0), contains("z", "c")); + assertThat(results.getPCollection(top3).get(0), contains("a", "bb", "c")); + assertThat(results.getPCollection(largestPerKey), containsInAnyOrder( + KV.of("a", Arrays.asList(3, 2)), + KV.of("b", Arrays.asList(100, 10)))); + assertThat(results.getPCollection(smallestPerKey), containsInAnyOrder( + KV.of("a", Arrays.asList(1, 2)), + KV.of("b", Arrays.asList(1, 10)))); + } + + @Test + @SuppressWarnings("unchecked") + public void testTopEmpty() { + DirectPipeline p = DirectPipeline.createForTest(); + PCollection input = + p.apply(Create.of(Arrays.asList(EMPTY_COLLECTION))) + .setCoder(StringUtf8Coder.of()); + + PCollection> top1 = input.apply(Top.of(1, new OrderByLength())); + PCollection> top2 = input.apply(Top.largest(2)); + PCollection> top3 = input.apply(Top.smallest(3)); + + PCollection>> largestPerKey = createEmptyInputTable(p) + .apply(Top.largestPerKey(2)); + PCollection>> smallestPerKey = createEmptyInputTable(p) + .apply(Top.smallestPerKey(2)); + + EvaluationResults results = p.run(); + + assertThat(results.getPCollection(top1).get(0), containsInAnyOrder()); + assertThat(results.getPCollection(top2).get(0), containsInAnyOrder()); + assertThat(results.getPCollection(top3).get(0), containsInAnyOrder()); + assertThat(results.getPCollection(largestPerKey), containsInAnyOrder()); + assertThat(results.getPCollection(smallestPerKey), containsInAnyOrder()); + } + + @Test + @SuppressWarnings("unchecked") + public void testTopZero() { + DirectPipeline p = DirectPipeline.createForTest(); + PCollection input = + p.apply(Create.of(Arrays.asList(COLLECTION))) + .setCoder(StringUtf8Coder.of()); + + PCollection> top1 = input.apply(Top.of(0, new OrderByLength())); + PCollection> top2 = input.apply(Top.largest(0)); + PCollection> top3 = input.apply(Top.smallest(0)); + + PCollection>> largestPerKey = createInputTable(p) + .apply(Top.largestPerKey(0)); + + PCollection>> smallestPerKey = createInputTable(p) + .apply(Top.smallestPerKey(0)); + + EvaluationResults results = p.run(); + + assertThat(results.getPCollection(top1).get(0), containsInAnyOrder()); + assertThat(results.getPCollection(top2).get(0), containsInAnyOrder()); + assertThat(results.getPCollection(top3).get(0), containsInAnyOrder()); + assertThat(results.getPCollection(largestPerKey), containsInAnyOrder( + KV.of("a", Arrays.asList()), + KV.of("b", Arrays.asList()))); + assertThat(results.getPCollection(smallestPerKey), containsInAnyOrder( + KV.of("a", Arrays.asList()), + KV.of("b", Arrays.asList()))); + } + + // This is a purely compile-time test. If the code compiles, then it worked. + @Test + public void testPerKeySerializabilityRequirement() { + DirectPipeline p = DirectPipeline.createForTest(); + PCollection input = + p.apply(Create.of(Arrays.asList(COLLECTION))) + .setCoder(StringUtf8Coder.of()); + + PCollection>> top1 = createInputTable(p) + .apply(Top.perKey(1, + new IntegerComparator())); + + PCollection>> top2 = createInputTable(p) + .apply(Top.perKey(1, + new IntegerComparator2())); + } + + @Test + public void testCountConstraint() { + DirectPipeline p = DirectPipeline.createForTest(); + PCollection input = + p.apply(Create.of(Arrays.asList(COLLECTION))) + .setCoder(StringUtf8Coder.of()); + + expectedEx.expect(IllegalArgumentException.class); + expectedEx.expectMessage(Matchers.containsString(">= 0")); + + input.apply(Top.of(-1, new OrderByLength())); + } + + @Test + public void testTransformName() { + DirectPipeline p = DirectPipeline.createForTest(); + PCollection input = + p.apply(Create.of(Arrays.asList(COLLECTION))) + .setCoder(StringUtf8Coder.of()); + + PTransform, PCollection>> top = Top + .of(10, new OrderByLength()); + input.apply(top); + + p.traverseTopologically(new RecordingPipelineVisitor()); + // Check that the transform is named "Top" rather than "Combine". + assertThat(p.getFullName(top), Matchers.startsWith("Top")); + } + + static class OrderByLength implements Comparator, Serializable { + @Override + public int compare(String a, String b) { + if (a.length() != b.length()) { + return a.length() - b.length(); + } else { + return a.compareTo(b); + } + } + } + + static class IntegerComparator implements Comparator, Serializable { + @Override + public int compare(Integer o1, Integer o2) { + return o1.compareTo(o2); + } + } + + static class IntegerComparator2 implements SerializableComparator { + @Override + public int compare(Integer o1, Integer o2) { + return o1.compareTo(o2); + } + } + +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ValuesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ValuesTest.java new file mode 100644 index 0000000000000..497d8fc8406e0 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ValuesTest.java @@ -0,0 +1,103 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; + +/** + * Tests for Values transform. + */ +@RunWith(JUnit4.class) +public class ValuesTest { + static final KV[] TABLE = new KV[] { + KV.of("one", 1), + KV.of("two", 2), + KV.of("three", 3), + KV.of("four", 4), + KV.of("dup", 4) + }; + + static final KV[] EMPTY_TABLE = new KV[] { + }; + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testValues() { + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.of(Arrays.asList(TABLE))).setCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + + PCollection output = input.apply(Values.create()); + + DataflowAssert.that(output) + .containsInAnyOrder(1, 2, 3, 4, 4); + + p.run(); + } + + // TODO: setOrdered(true) isn't supported yet by the Dataflow service. + @Test + public void testValuesOrdered() { + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.of(Arrays.asList(TABLE))).setCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + + input.setOrdered(true); + PCollection output = + input.apply(Values.create()).setOrdered(true); + + DataflowAssert.that(output) + .containsInOrder(1, 2, 3, 4, 4); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testValuesEmpty() { + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.of(Arrays.asList(EMPTY_TABLE))).setCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + + PCollection output = input.apply(Values.create()); + + DataflowAssert.that(output) + .containsInAnyOrder(); + + p.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ViewTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ViewTest.java new file mode 100644 index 0000000000000..3a7c8187d9237 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ViewTest.java @@ -0,0 +1,159 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static org.hamcrest.CoreMatchers.isA; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.VarIntCoder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionView; + + +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; +import java.util.NoSuchElementException; + +/** + * Tests for {@link View}. See also {@link ParDoTest} which + * provides additional coverage since views can only be + * observed via {@link ParDo}. + */ +@RunWith(JUnit4.class) +public class ViewTest implements Serializable { + // This test is Serializable, just so that it's easy to have + // anonymous inner classes inside the non-static test methods. + + @Rule + public transient ExpectedException thrown = ExpectedException.none(); + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testSingletonSideInput() { + Pipeline pipeline = TestPipeline.create(); + + final PCollectionView view = pipeline + .apply(Create.of(47)) + .apply(View.asSingleton()); + + PCollection output = pipeline + .apply(Create.of(1, 2, 3)) + .apply(ParDo.withSideInputs(view).of( + new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.sideInput(view)); + } + })); + + DataflowAssert.that(output) + .containsInAnyOrder(47, 47, 47); + + pipeline.run(); + } + + @Test + public void testEmptySingletonSideInput() throws Exception { + Pipeline pipeline = TestPipeline.create(); + + final PCollectionView view = pipeline + .apply(Create.of()) + .setCoder(VarIntCoder.of()) + .apply(View.asSingleton()); + + PCollection output = pipeline + .apply(Create.of(1, 2, 3)) + .apply(ParDo.withSideInputs(view).of( + new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.sideInput(view)); + } + })); + + thrown.expect(RuntimeException.class); + thrown.expectCause(isA(NoSuchElementException.class)); + thrown.expectMessage("Empty"); + thrown.expectMessage("PCollection"); + thrown.expectMessage("singleton"); + + pipeline.run(); + } + + @Test + public void testNonSingletonSideInput() throws Exception { + Pipeline pipeline = TestPipeline.create(); + + final PCollectionView view = pipeline + .apply(Create.of(1, 2, 3)) + .apply(View.asSingleton()); + + PCollection output = pipeline + .apply(Create.of(1, 2, 3)) + .apply(ParDo.withSideInputs(view).of( + new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.sideInput(view)); + } + })); + + thrown.expect(RuntimeException.class); + thrown.expectCause(isA(IllegalArgumentException.class)); + thrown.expectMessage("PCollection"); + thrown.expectMessage("more than one"); + thrown.expectMessage("singleton"); + + pipeline.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testIterableSideInput() { + Pipeline pipeline = TestPipeline.create(); + + final PCollectionView, ?> view = pipeline + .apply(Create.of(11, 13, 17, 23)) + .apply(View.asIterable()); + + PCollection output = pipeline + .apply(Create.of(29, 31)) + .apply(ParDo.withSideInputs(view).of( + new DoFn() { + @Override + public void processElement(ProcessContext c) { + for (Integer i : c.sideInput(view)) { + c.output(i); + } + } + })); + + DataflowAssert.that(output).containsInAnyOrder( + 11, 13, 17, 23, + 11, 13, 17, 23); + + pipeline.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/WithKeysTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/WithKeysTest.java new file mode 100644 index 0000000000000..3e4e359022c96 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/WithKeysTest.java @@ -0,0 +1,122 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Tests for ExtractKeys transform. + */ +@RunWith(JUnit4.class) +public class WithKeysTest { + static final String[] COLLECTION = new String[] { + "a", + "aa", + "b", + "bb", + "bbb" + }; + + static final List> WITH_KEYS = Arrays.asList( + KV.of(1, "a"), + KV.of(2, "aa"), + KV.of(1, "b"), + KV.of(2, "bb"), + KV.of(3, "bbb") + ); + + static final List> WITH_CONST_KEYS = Arrays.asList( + KV.of(100, "a"), + KV.of(100, "aa"), + KV.of(100, "b"), + KV.of(100, "bb"), + KV.of(100, "bbb") + ); + + @Test + public void testExtractKeys() { + Pipeline p = TestPipeline.create(); + + PCollection input = + p.apply(Create.of(Arrays.asList(COLLECTION))).setCoder( + StringUtf8Coder.of()); + + PCollection> output = input.apply(WithKeys.of( + new LengthAsKey())); + DataflowAssert.that(output) + .containsInAnyOrder(WITH_KEYS); + + p.run(); + } + + // TODO: setOrdered(true) isn't supported yet by the Dataflow service. + @Test + public void testExtractKeysOrdered() { + Pipeline p = TestPipeline.create(); + + PCollection input = + p.apply(Create.of(Arrays.asList(COLLECTION))).setCoder( + StringUtf8Coder.of()); + + input.setOrdered(true); + PCollection> output = input.apply(WithKeys.of( + new LengthAsKey())).setOrdered(true); + DataflowAssert.that(output) + .containsInAnyOrder(WITH_KEYS); + + p.run(); + } + + @Test + public void testConstantKeys() { + Pipeline p = TestPipeline.create(); + + PCollection input = + p.apply(Create.of(Arrays.asList(COLLECTION))).setCoder( + StringUtf8Coder.of()); + + PCollection> output = + input.apply(WithKeys.of(100)); + DataflowAssert.that(output) + .containsInAnyOrder(WITH_CONST_KEYS); + + p.run(); + } + + /** + * Key a value by its length. + */ + public static class LengthAsKey + implements SerializableFunction { + @Override + public Integer apply(String value) { + return value.length(); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultCoderTest.java new file mode 100644 index 0000000000000..afb8a998798ec --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultCoderTest.java @@ -0,0 +1,55 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +import static org.junit.Assert.assertEquals; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.DoubleCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult.CoGbkResultCoder; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.Serializer; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TupleTagList; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; + +/** + * Tests the CoGbkResult.CoGbkResultCoder. + */ +@RunWith(JUnit4.class) +public class CoGbkResultCoderTest { + + @Test + public void testSerializationDeserialization() { + CoGbkResultSchema schema = + new CoGbkResultSchema(TupleTagList.of(new TupleTag()).and( + new TupleTag())); + UnionCoder unionCoder = + UnionCoder.of(Arrays.>asList(StringUtf8Coder.of(), + DoubleCoder.of())); + CoGbkResultCoder newCoder = CoGbkResultCoder.of(schema, unionCoder); + CloudObject encoding = newCoder.asCloudObject(); + Coder decodedCoder = Serializer.deserialize(encoding, Coder.class); + assertEquals(newCoder, decodedCoder); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java new file mode 100644 index 0000000000000..016ba15d5ae44 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java @@ -0,0 +1,348 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.runners.DirectPipeline; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.EvaluationResults; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.DoFnTester; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import org.hamcrest.Matcher; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; + +/** + * Tests for CoGroupByKeyTest. Implements Serializable for anonymous DoFns. + */ +@RunWith(JUnit4.class) +public class CoGroupByKeyTest implements Serializable { + + /** + * Converts the given list into a PCollection belonging to the provided + * Pipeline in such a way that coder inference needs to be performed. + */ + private PCollection> createInput( + Pipeline p, List> list) { + return p + .apply(Create.of(list)) + // Create doesn't infer coders for parameterized types. + .setCoder( + KvCoder.of(BigEndianIntegerCoder.of(), StringUtf8Coder.of())) + // Do a dummy transform so consumers must deal with coder inference. + .apply(ParDo.of(new DoFn, + KV>() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element()); + } + })); + } + + /** + * Returns a PCollection> containing the + * results of the CoGbk over 3 PCollection>, each of + * which correlates a customer id to purchases, addresses, or names, + * respectively. + */ + private PCollection> buildPurchasesCoGbk( + Pipeline p, + TupleTag purchasesTag, + TupleTag addressesTag, + TupleTag namesTag) { + List> idToPurchases = + Arrays.asList( + KV.of(2, "Boat"), + KV.of(1, "Shoes"), + KV.of(3, "Car"), + KV.of(1, "Book"), + KV.of(10, "Pens"), + KV.of(8, "House"), + KV.of(4, "Suit"), + KV.of(11, "House"), + KV.of(14, "Shoes"), + KV.of(2, "Suit"), + KV.of(8, "Suit Case"), + KV.of(3, "House")); + + List> idToAddress = + Arrays.asList( + KV.of(2, "53 S. 3rd"), + KV.of(10, "383 Jackson Street"), + KV.of(20, "3 W. Arizona"), + KV.of(3, "29 School Rd"), + KV.of(8, "6 Watling Rd")); + + List> idToName = + Arrays.asList( + KV.of(1, "John Smith"), + KV.of(2, "Sally James"), + KV.of(8, "Jeffery Spalding"), + KV.of(20, "Joan Lichtfield")); + + PCollection> purchasesTable = + createInput(p, idToPurchases); + + PCollection> addressTable = + createInput(p, idToAddress); + + PCollection> nameTable = + createInput(p, idToName); + + PCollection> coGbkResults = + KeyedPCollectionTuple.of(namesTag, nameTable) + .and(addressesTag, addressTable) + .and(purchasesTag, purchasesTable) + .apply(CoGroupByKey.create()); + return coGbkResults; + } + + @Test + public void testCoGroupByKey() { + TupleTag namesTag = new TupleTag<>(); + TupleTag addressesTag = new TupleTag<>(); + TupleTag purchasesTag = new TupleTag<>(); + + DirectPipeline p = DirectPipeline.createForTest(); + + PCollection> coGbkResults = + buildPurchasesCoGbk(p, purchasesTag, addressesTag, namesTag); + + EvaluationResults results = p.run(); + + List> finalResult = + results.getPCollection(coGbkResults); + + HashMap>> namesMatchers = + new HashMap>>() { + { + put(1, containsInAnyOrder("John Smith")); + put(2, containsInAnyOrder("Sally James")); + put(8, containsInAnyOrder("Jeffery Spalding")); + put(20, containsInAnyOrder("Joan Lichtfield")); + } + }; + + HashMap>> addressesMatchers = + new HashMap>>() { + { + put(2, containsInAnyOrder("53 S. 3rd")); + put(3, containsInAnyOrder("29 School Rd")); + put(8, containsInAnyOrder("6 Watling Rd")); + put(10, containsInAnyOrder("383 Jackson Street")); + put(20, containsInAnyOrder("3 W. Arizona")); + } + }; + + HashMap>> purchasesMatchers = + new HashMap>>() { + { + put(1, containsInAnyOrder("Shoes", "Book")); + put(2, containsInAnyOrder("Suit", "Boat")); + put(3, containsInAnyOrder("Car", "House")); + put(4, containsInAnyOrder("Suit")); + put(8, containsInAnyOrder("House", "Suit Case")); + put(10, containsInAnyOrder("Pens")); + put(11, containsInAnyOrder("House")); + put(14, containsInAnyOrder("Shoes")); + } + }; + + // TODO: Figure out a way to do a hamcrest matcher for CoGbkResults. + for (KV result : finalResult) { + int key = result.getKey(); + CoGbkResult row = result.getValue(); + checkValuesMatch(key, namesMatchers, row, namesTag); + checkValuesMatch(key, addressesMatchers, row, addressesTag); + checkValuesMatch(key, purchasesMatchers, row, purchasesTag); + + } + + } + + /** + * Checks that the values for the given tag in the given row matches the + * expected values for the given key in the given matchers map. + */ + private void checkValuesMatch( + K key, + HashMap>> matchers, + CoGbkResult row, + TupleTag tag) { + Iterable taggedValues = row.getAll(tag); + if (taggedValues.iterator().hasNext()) { + assertThat(taggedValues, matchers.get(key)); + } else { + assertNull(matchers.get(key)); + } + } + + /** + * A DoFn used in testCoGroupByKeyHandleResults(), to test processing the + * results of a CoGroupByKey. + */ + private static class CorrelatePurchaseCountForAddressesWithoutNamesFn extends + DoFn, KV> { + private final TupleTag purchasesTag; + + private final TupleTag addressesTag; + + private final TupleTag namesTag; + + private CorrelatePurchaseCountForAddressesWithoutNamesFn( + TupleTag purchasesTag, + TupleTag addressesTag, + TupleTag namesTag) { + this.purchasesTag = purchasesTag; + this.addressesTag = addressesTag; + this.namesTag = namesTag; + } + + @Override + public void processElement(ProcessContext c) { + KV e = c.element(); + CoGbkResult row = e.getValue(); + // Don't actually care about the id. + Iterable names = row.getAll(namesTag); + if (names.iterator().hasNext()) { + // Nothing to do. There was a name. + return; + } + Iterable addresses = row.getAll(addressesTag); + if (!addresses.iterator().hasNext()) { + // Nothing to do, there was no address. + return; + } + // Buffer the addresses so we can accredit all of them with + // corresponding purchases. All addresses are for the same id, so + // if there are multiple, we apply the same purchase count to all. + ArrayList addressList = new ArrayList(); + for (String address : addresses) { + addressList.add(address); + } + + Iterable purchases = row.getAll(purchasesTag); + + int purchaseCount = 0; + for (String purchase : purchases) { + purchaseCount++; + } + + for (String address : addressList) { + c.output(KV.of(address, purchaseCount)); + } + } + } + + /** + * Tests that the consuming DoFn + * (CorrelatePurchaseCountForAddressesWithoutNamesFn) performs as expected. + */ + @SuppressWarnings("unchecked") + @Test + public void testConsumingDoFn() { + TupleTag purchasesTag = new TupleTag<>(); + TupleTag addressesTag = new TupleTag<>(); + TupleTag namesTag = new TupleTag<>(); + + // result1 should get filtered out because it has a name. + CoGbkResult result1 = CoGbkResult + .of(purchasesTag, Arrays.asList("3a", "3b")) + .and(addressesTag, Arrays.asList("2a", "2b")) + .and(namesTag, Arrays.asList("1a")); + // result 2 should be counted because it has an address and purchases. + CoGbkResult result2 = CoGbkResult + .of(purchasesTag, Arrays.asList("5a", "5b")) + .and(addressesTag, Arrays.asList("4a")) + .and(namesTag, new ArrayList()); + // result 3 should not be counted because it has no addresses. + CoGbkResult result3 = CoGbkResult + .of(purchasesTag, Arrays.asList("7a", "7b")) + .and(addressesTag, new ArrayList()) + .and(namesTag, new ArrayList()); + // result 4 should be counted as 0, because it has no purchases. + CoGbkResult result4 = CoGbkResult + .of(purchasesTag, new ArrayList()) + .and(addressesTag, Arrays.asList("8a")) + .and(namesTag, new ArrayList()); + + List> results = + DoFnTester.of( + new CorrelatePurchaseCountForAddressesWithoutNamesFn( + purchasesTag, + addressesTag, + namesTag)) + .processBatch( + KV.of(1, result1), + KV.of(2, result2), + KV.of(3, result3), + KV.of(4, result4)); + assertThat(results, containsInAnyOrder(KV.of("4a", 2), KV.of("8a", 0))); + } + + /** + * Tests the pipeline end-to-end. Builds the purchases CoGroupByKey, and + * applies CorrelatePurchaseCountForAddressesWithoutNamesFn to the results. + */ + @SuppressWarnings("unchecked") + @Test + public void testCoGroupByKeyHandleResults() { + TupleTag namesTag = new TupleTag<>(); + TupleTag addressesTag = new TupleTag<>(); + TupleTag purchasesTag = new TupleTag<>(); + + Pipeline p = TestPipeline.create(); + + PCollection> coGbkResults = + buildPurchasesCoGbk(p, purchasesTag, addressesTag, namesTag); + + // Do some simple processing on the result of the CoGroupByKey. Count the + // purchases for each address on record that has no associated name. + PCollection> + purchaseCountByKnownAddressesWithoutKnownNames = + coGbkResults.apply(ParDo.of( + new CorrelatePurchaseCountForAddressesWithoutNamesFn( + purchasesTag, addressesTag, namesTag))); + + DataflowAssert.that(purchaseCountByKnownAddressesWithoutKnownNames) + .containsInAnyOrder( + KV.of("29 School Rd", 2), + KV.of("383 Jackson Street", 1)); + p.run(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoderTest.java new file mode 100644 index 0000000000000..24e6dde65c4fe --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoderTest.java @@ -0,0 +1,48 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +import static org.junit.Assert.assertEquals; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.DoubleCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.Serializer; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; + +/** + * Tests the UnionCoder. + */ +@RunWith(JUnit4.class) +public class UnionCoderTest { + + @Test + public void testSerializationDeserialization() { + UnionCoder newCoder = + UnionCoder.of(Arrays.>asList(StringUtf8Coder.of(), + DoubleCoder.of())); + CloudObject encoding = newCoder.asCloudObject(); + Coder decodedCoder = Serializer.deserialize(encoding, Coder.class); + assertEquals(newCoder, decodedCoder); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindowsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindowsTest.java new file mode 100644 index 0000000000000..36028e493a755 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindowsTest.java @@ -0,0 +1,260 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.runWindowingFn; +import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.set; +import static org.junit.Assert.assertEquals; + +import org.joda.time.DateTime; +import org.joda.time.DateTimeConstants; +import org.joda.time.DateTimeZone; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Tests for CalendarWindows WindowingFn. + */ +@RunWith(JUnit4.class) +public class CalendarWindowsTest { + + private static Instant makeTimestamp(int year, int month, int day, int hours, int minutes) { + return new DateTime(year, month, day, hours, minutes, DateTimeZone.UTC).toInstant(); + } + + @Test + public void testDays() throws Exception { + Map> expected = new HashMap<>(); + + final List timestamps = Arrays.asList( + makeTimestamp(2014, 1, 1, 0, 0).getMillis(), + makeTimestamp(2014, 1, 1, 23, 59).getMillis(), + + makeTimestamp(2014, 1, 2, 0, 0).getMillis(), + makeTimestamp(2014, 1, 2, 5, 5).getMillis(), + + makeTimestamp(2015, 1, 1, 0, 0).getMillis(), + makeTimestamp(2015, 1, 1, 5, 5).getMillis()); + + + expected.put( + new IntervalWindow( + makeTimestamp(2014, 1, 1, 0, 0), + makeTimestamp(2014, 1, 2, 0, 0)), + set(timestamps.get(0), timestamps.get(1))); + + expected.put( + new IntervalWindow( + makeTimestamp(2014, 1, 2, 0, 0), + makeTimestamp(2014, 1, 3, 0, 0)), + set(timestamps.get(2), timestamps.get(3))); + + expected.put( + new IntervalWindow( + makeTimestamp(2015, 1, 1, 0, 0), + makeTimestamp(2015, 1, 2, 0, 0)), + set(timestamps.get(4), timestamps.get(5))); + + assertEquals(expected, runWindowingFn(CalendarWindows.days(1), timestamps)); + } + + @Test + public void testWeeks() throws Exception { + Map> expected = new HashMap<>(); + + final List timestamps = Arrays.asList( + makeTimestamp(2014, 1, 1, 0, 0).getMillis(), + makeTimestamp(2014, 1, 5, 5, 5).getMillis(), + + makeTimestamp(2014, 1, 8, 0, 0).getMillis(), + makeTimestamp(2014, 1, 12, 5, 5).getMillis(), + + makeTimestamp(2015, 1, 1, 0, 0).getMillis(), + makeTimestamp(2015, 1, 6, 5, 5).getMillis()); + + + expected.put( + new IntervalWindow( + makeTimestamp(2014, 1, 1, 0, 0), + makeTimestamp(2014, 1, 8, 0, 0)), + set(timestamps.get(0), timestamps.get(1))); + + expected.put( + new IntervalWindow( + makeTimestamp(2014, 1, 8, 0, 0), + makeTimestamp(2014, 1, 15, 0, 0)), + set(timestamps.get(2), timestamps.get(3))); + + expected.put( + new IntervalWindow( + makeTimestamp(2014, 12, 31, 0, 0), + makeTimestamp(2015, 1, 7, 0, 0)), + set(timestamps.get(4), timestamps.get(5))); + + assertEquals(expected, + runWindowingFn(CalendarWindows.weeks(1, DateTimeConstants.WEDNESDAY), timestamps)); + } + + @Test + public void testMonths() throws Exception { + Map> expected = new HashMap<>(); + + final List timestamps = Arrays.asList( + makeTimestamp(2014, 1, 1, 0, 0).getMillis(), + makeTimestamp(2014, 1, 31, 5, 5).getMillis(), + + makeTimestamp(2014, 2, 1, 0, 0).getMillis(), + makeTimestamp(2014, 2, 15, 5, 5).getMillis(), + + makeTimestamp(2015, 1, 1, 0, 0).getMillis(), + makeTimestamp(2015, 1, 31, 5, 5).getMillis()); + + + expected.put( + new IntervalWindow( + makeTimestamp(2014, 1, 1, 0, 0), + makeTimestamp(2014, 2, 1, 0, 0)), + set(timestamps.get(0), timestamps.get(1))); + + expected.put( + new IntervalWindow( + makeTimestamp(2014, 2, 1, 0, 0), + makeTimestamp(2014, 3, 1, 0, 0)), + set(timestamps.get(2), timestamps.get(3))); + + expected.put( + new IntervalWindow( + makeTimestamp(2015, 1, 1, 0, 0), + makeTimestamp(2015, 2, 1, 0, 0)), + set(timestamps.get(4), timestamps.get(5))); + + assertEquals(expected, + runWindowingFn(CalendarWindows.months(1), timestamps)); + } + + @Test + public void testMultiMonths() throws Exception { + Map> expected = new HashMap<>(); + + final List timestamps = Arrays.asList( + makeTimestamp(2014, 3, 5, 0, 0).getMillis(), + makeTimestamp(2014, 10, 4, 23, 59).getMillis(), + + makeTimestamp(2014, 10, 5, 0, 0).getMillis(), + makeTimestamp(2015, 3, 1, 0, 0).getMillis(), + + makeTimestamp(2016, 1, 5, 0, 0).getMillis(), + makeTimestamp(2016, 1, 31, 5, 5).getMillis()); + + + expected.put( + new IntervalWindow( + makeTimestamp(2014, 3, 5, 0, 0), + makeTimestamp(2014, 10, 5, 0, 0)), + set(timestamps.get(0), timestamps.get(1))); + + expected.put( + new IntervalWindow( + makeTimestamp(2014, 10, 5, 0, 0), + makeTimestamp(2015, 5, 5, 0, 0)), + set(timestamps.get(2), timestamps.get(3))); + + expected.put( + new IntervalWindow( + makeTimestamp(2015, 12, 5, 0, 0), + makeTimestamp(2016, 7, 5, 0, 0)), + set(timestamps.get(4), timestamps.get(5))); + + assertEquals(expected, runWindowingFn( + CalendarWindows.months(7).withStartingMonth(2014, 3).beginningOnDay(5), timestamps)); + } + + @Test + public void testYears() throws Exception { + Map> expected = new HashMap<>(); + + final List timestamps = Arrays.asList( + makeTimestamp(2000, 5, 5, 0, 0).getMillis(), + makeTimestamp(2010, 5, 4, 23, 59).getMillis(), + + makeTimestamp(2010, 5, 5, 0, 0).getMillis(), + makeTimestamp(2015, 3, 1, 0, 0).getMillis(), + + makeTimestamp(2052, 1, 5, 0, 0).getMillis(), + makeTimestamp(2060, 5, 4, 5, 5).getMillis()); + + + expected.put( + new IntervalWindow( + makeTimestamp(2000, 5, 5, 0, 0), + makeTimestamp(2010, 5, 5, 0, 0)), + set(timestamps.get(0), timestamps.get(1))); + + expected.put( + new IntervalWindow( + makeTimestamp(2010, 5, 5, 0, 0), + makeTimestamp(2020, 5, 5, 0, 0)), + set(timestamps.get(2), timestamps.get(3))); + + expected.put( + new IntervalWindow( + makeTimestamp(2050, 5, 5, 0, 0), + makeTimestamp(2060, 5, 5, 0, 0)), + set(timestamps.get(4), timestamps.get(5))); + + assertEquals(expected, runWindowingFn( + CalendarWindows.years(10).withStartingYear(2000).beginningOnDay(5, 5), timestamps)); + } + + @Test + public void testTimeZone() throws Exception { + Map> expected = new HashMap<>(); + + DateTimeZone timeZone = DateTimeZone.forID("America/Los_Angeles"); + + final List timestamps = Arrays.asList( + new DateTime(2014, 1, 1, 0, 0, timeZone).getMillis(), + new DateTime(2014, 1, 1, 23, 59, timeZone).getMillis(), + + new DateTime(2014, 1, 2, 8, 0, DateTimeZone.UTC).getMillis(), + new DateTime(2014, 1, 3, 7, 59, DateTimeZone.UTC).getMillis()); + + expected.put( + new IntervalWindow( + new DateTime(2014, 1, 1, 0, 0, timeZone).toInstant(), + new DateTime(2014, 1, 2, 0, 0, timeZone).toInstant()), + set(timestamps.get(0), timestamps.get(1))); + + expected.put( + new IntervalWindow( + new DateTime(2014, 1, 2, 0, 0, timeZone).toInstant(), + new DateTime(2014, 1, 3, 0, 0, timeZone).toInstant()), + set(timestamps.get(2), timestamps.get(3))); + + assertEquals(expected, runWindowingFn( + CalendarWindows.days(1).withTimeZone(timeZone), + timestamps)); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindowsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindowsTest.java new file mode 100644 index 0000000000000..0a68e72348f73 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindowsTest.java @@ -0,0 +1,114 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.runWindowingFn; +import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.set; +import static org.hamcrest.CoreMatchers.containsString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** + * Tests for FixedWindows WindowingFn. + */ +@RunWith(JUnit4.class) +public class FixedWindowsTest { + + @Test + public void testSimpleFixedWindow() throws Exception { + Map> expected = new HashMap<>(); + expected.put(new IntervalWindow(new Instant(0), new Instant(10)), set(1, 2, 5, 9)); + expected.put(new IntervalWindow(new Instant(10), new Instant(20)), set(10, 11)); + expected.put(new IntervalWindow(new Instant(100), new Instant(110)), set(100)); + assertEquals( + expected, + runWindowingFn( + FixedWindows.of(new Duration(10)), + Arrays.asList(1L, 2L, 5L, 9L, 10L, 11L, 100L))); + } + + @Test + public void testFixedOffsetWindow() throws Exception { + Map> expected = new HashMap<>(); + expected.put(new IntervalWindow(new Instant(-5), new Instant(5)), set(1, 2)); + expected.put(new IntervalWindow(new Instant(5), new Instant(15)), set(5, 9, 10, 11)); + expected.put(new IntervalWindow(new Instant(95), new Instant(105)), set(100)); + assertEquals( + expected, + runWindowingFn( + FixedWindows.of(new Duration(10)).withOffset(new Duration(5)), + Arrays.asList(1L, 2L, 5L, 9L, 10L, 11L, 100L))); + } + + @Test + public void testTimeUnit() throws Exception { + Map> expected = new HashMap<>(); + expected.put(new IntervalWindow(new Instant(-5000), new Instant(5000)), set(1, 2, 1000)); + expected.put(new IntervalWindow(new Instant(5000), new Instant(15000)), set(5000, 5001, 10000)); + assertEquals( + expected, + runWindowingFn( + FixedWindows.of(Duration.standardSeconds(10)).withOffset(Duration.standardSeconds(5)), + Arrays.asList(1L, 2L, 1000L, 5000L, 5001L, 10000L))); + } + + void checkConstructionFailure(int size, int offset) { + try { + FixedWindows.of(Duration.standardSeconds(size)).withOffset(Duration.standardSeconds(offset)); + fail("should have failed"); + } catch (IllegalArgumentException e) { + assertThat(e.toString(), + containsString("FixedWindows WindowingStrategies must have 0 <= offset < size")); + } + } + + @Test + public void testInvalidInput() throws Exception { + checkConstructionFailure(-1, 0); + checkConstructionFailure(1, 2); + checkConstructionFailure(1, -1); + } + + @Test + public void testEquality() { + assertTrue(FixedWindows.of(new Duration(10)).isCompatible(FixedWindows.of(new Duration(10)))); + assertTrue( + FixedWindows.of(new Duration(10)).isCompatible( + FixedWindows.of(new Duration(10)))); + assertTrue( + FixedWindows.of(new Duration(10)).isCompatible( + FixedWindows.of(new Duration(10)))); + + assertFalse(FixedWindows.of(new Duration(10)).isCompatible(FixedWindows.of(new Duration(20)))); + assertFalse(FixedWindows.of(new Duration(10)).isCompatible( + FixedWindows.of(new Duration(20)))); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java new file mode 100644 index 0000000000000..ccb1ddecc4968 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java @@ -0,0 +1,100 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.runWindowingFn; +import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.set; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** + * Tests for Sessions WindowingFn. + */ +@RunWith(JUnit4.class) +public class SessionsTest { + + @Test + public void testSimple() throws Exception { + Map> expected = new HashMap<>(); + expected.put(new IntervalWindow(new Instant(0), new Instant(10)), set(0)); + expected.put(new IntervalWindow(new Instant(10), new Instant(20)), set(10)); + expected.put(new IntervalWindow(new Instant(101), new Instant(111)), set(101)); + assertEquals( + expected, + runWindowingFn( + Sessions.withGapDuration(new Duration(10)), + Arrays.asList(0L, 10L, 101L))); + } + + @Test + public void testConsecutive() throws Exception { + Map> expected = new HashMap<>(); + expected.put(new IntervalWindow(new Instant(1), new Instant(19)), set(1, 2, 5, 9)); + expected.put(new IntervalWindow(new Instant(100), new Instant(111)), set(100, 101)); + assertEquals( + expected, + runWindowingFn( + Sessions.withGapDuration(new Duration(10)), + Arrays.asList(1L, 2L, 5L, 9L, 100L, 101L))); + } + + @Test + public void testMerging() throws Exception { + Map> expected = new HashMap<>(); + expected.put(new IntervalWindow(new Instant(1), new Instant(40)), set(1, 10, 15, 22, 30)); + expected.put(new IntervalWindow(new Instant(95), new Instant(111)), set(95, 100, 101)); + assertEquals( + expected, + runWindowingFn( + Sessions.withGapDuration(new Duration(10)), + Arrays.asList(1L, 15L, 30L, 100L, 101L, 95L, 22L, 10L))); + } + + @Test + public void testTimeUnit() throws Exception { + Map> expected = new HashMap<>(); + expected.put(new IntervalWindow(new Instant(1), new Instant(2000)), set(1, 2, 1000)); + expected.put(new IntervalWindow(new Instant(5000), new Instant(6001)), set(5000, 5001)); + expected.put(new IntervalWindow(new Instant(10000), new Instant(11000)), set(10000)); + assertEquals( + expected, + runWindowingFn( + Sessions.withGapDuration(Duration.standardSeconds(1)), + Arrays.asList(1L, 2L, 1000L, 5000L, 5001L, 10000L))); + } + + @Test + public void testEquality() { + assertTrue( + Sessions.withGapDuration(new Duration(10)).isCompatible( + Sessions.withGapDuration(new Duration(10)))); + assertTrue( + Sessions.withGapDuration(new Duration(10)).isCompatible( + Sessions.withGapDuration(new Duration(20)))); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java new file mode 100644 index 0000000000000..f187cb429940e --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java @@ -0,0 +1,127 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.runWindowingFn; +import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.set; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** + * Tests for the SlidingWindows WindowingFn. + */ +@RunWith(JUnit4.class) +public class SlidingWindowsTest { + + @Test + public void testSimple() throws Exception { + Map> expected = new HashMap<>(); + expected.put(new IntervalWindow(new Instant(-5), new Instant(5)), set(1, 2)); + expected.put(new IntervalWindow(new Instant(0), new Instant(10)), set(1, 2, 5, 9)); + expected.put(new IntervalWindow(new Instant(5), new Instant(15)), set(5, 9, 10, 11)); + expected.put(new IntervalWindow(new Instant(10), new Instant(20)), set(10, 11)); + assertEquals( + expected, + runWindowingFn( + SlidingWindows.of(new Duration(10)).every(new Duration(5)), + Arrays.asList(1L, 2L, 5L, 9L, 10L, 11L))); + } + + @Test + public void testSlightlyOverlapping() throws Exception { + Map> expected = new HashMap<>(); + expected.put(new IntervalWindow(new Instant(-5), new Instant(2)), set(1)); + expected.put(new IntervalWindow(new Instant(0), new Instant(7)), set(1, 2, 5)); + expected.put(new IntervalWindow(new Instant(5), new Instant(12)), set(5, 9, 10, 11)); + expected.put(new IntervalWindow(new Instant(10), new Instant(17)), set(10, 11)); + assertEquals( + expected, + runWindowingFn( + SlidingWindows.of(new Duration(7)).every(new Duration(5)), + Arrays.asList(1L, 2L, 5L, 9L, 10L, 11L))); + } + + @Test + public void testElidings() throws Exception { + Map> expected = new HashMap<>(); + expected.put(new IntervalWindow(new Instant(0), new Instant(3)), set(1, 2)); + expected.put(new IntervalWindow(new Instant(10), new Instant(13)), set(10, 11)); + expected.put(new IntervalWindow(new Instant(100), new Instant(103)), set(100)); + assertEquals( + expected, + runWindowingFn( + // Only look at the first 3 millisecs of every 10-millisec interval. + SlidingWindows.of(new Duration(3)).every(new Duration(10)), + Arrays.asList(1L, 2L, 3L, 5L, 9L, 10L, 11L, 100L))); + } + + @Test + public void testOffset() throws Exception { + Map> expected = new HashMap<>(); + expected.put(new IntervalWindow(new Instant(-8), new Instant(2)), set(1)); + expected.put(new IntervalWindow(new Instant(-3), new Instant(7)), set(1, 2, 5)); + expected.put(new IntervalWindow(new Instant(2), new Instant(12)), set(2, 5, 9, 10, 11)); + expected.put(new IntervalWindow(new Instant(7), new Instant(17)), set(9, 10, 11)); + assertEquals( + expected, + runWindowingFn( + SlidingWindows.of(new Duration(10)).every(new Duration(5)).withOffset(new Duration(2)), + Arrays.asList(1L, 2L, 5L, 9L, 10L, 11L))); + } + + @Test + public void testTimeUnit() throws Exception { + Map> expected = new HashMap<>(); + expected.put(new IntervalWindow(new Instant(-5000), new Instant(5000)), set(1, 2, 1000)); + expected.put(new IntervalWindow(new Instant(0), new Instant(10000)), + set(1, 2, 1000, 5000, 5001)); + expected.put(new IntervalWindow(new Instant(5000), new Instant(15000)), set(5000, 5001, 10000)); + expected.put(new IntervalWindow(new Instant(10000), new Instant(20000)), set(10000)); + assertEquals( + expected, + runWindowingFn( + SlidingWindows.of(Duration.standardSeconds(10)).every(Duration.standardSeconds(5)), + Arrays.asList(1L, 2L, 1000L, 5000L, 5001L, 10000L))); + } + + @Test + public void testEquality() { + assertTrue( + SlidingWindows.of(new Duration(10)).isCompatible( + SlidingWindows.of(new Duration(10)))); + assertTrue( + SlidingWindows.of(new Duration(10)).isCompatible( + SlidingWindows.of(new Duration(10)))); + + assertFalse(SlidingWindows.of(new Duration(10)).isCompatible( + SlidingWindows.of(new Duration(20)))); + assertFalse(SlidingWindows.of(new Duration(10)).isCompatible( + SlidingWindows.of(new Duration(20)))); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java new file mode 100644 index 0000000000000..85c0bf6b8b6d2 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java @@ -0,0 +1,277 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Flatten; +import com.google.cloud.dataflow.sdk.transforms.GroupByKey; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; +import com.google.cloud.dataflow.sdk.values.TimestampedValue; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.PrintStream; +import java.io.Serializable; +import java.util.Arrays; + +/** Unit tests for bucketing. */ +@RunWith(JUnit4.class) +public class WindowingTest implements Serializable { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + private static class WindowedCount extends PTransform, PCollection> { + private WindowingFn windowingFn; + public WindowedCount(WindowingFn windowingFn) { + this.windowingFn = (WindowingFn) windowingFn; + } + @Override + public PCollection apply(PCollection in) { + return in + .apply(Window.named("Window").into(windowingFn)) + .apply(Count.perElement()) + .apply(ParDo + .named("FormatCounts") + .of(new DoFn, String>() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().getKey() + ":" + c.element().getValue() + + ":" + c.timestamp().getMillis() + ":" + c.windows()); + } + })) + .setCoder(StringUtf8Coder.of()); + } + } + + private String output(String value, int count, int timestamp, int windowStart, int windowEnd) { + return value + ":" + count + ":" + timestamp + + ":[[" + new Instant(windowStart) + ".." + new Instant(windowEnd) + ")]"; + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testPartitioningWindowing() { + Pipeline p = TestPipeline.create(); + PCollection input = + p.apply( + Create.timestamped( + TimestampedValue.of("a", new Instant(1)), + TimestampedValue.of("b", new Instant(2)), + TimestampedValue.of("b", new Instant(3)), + TimestampedValue.of("c", new Instant(11)), + TimestampedValue.of("d", new Instant(11)))); + + PCollection output = + input + .apply(new WindowedCount(FixedWindows.of(new Duration(10)))); + + DataflowAssert.that(output).containsInAnyOrder( + output("a", 1, 9, 0, 10), + output("b", 2, 9, 0, 10), + output("c", 1, 19, 10, 20), + output("d", 1, 19, 10, 20)); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testNonPartitioningWindowing() { + Pipeline p = TestPipeline.create(); + PCollection input = + p.apply( + Create.timestamped( + TimestampedValue.of("a", new Instant(1)), + TimestampedValue.of("a", new Instant(7)), + TimestampedValue.of("b", new Instant(8)))); + + PCollection output = + input + .apply(new WindowedCount( + SlidingWindows.of(new Duration(10)).every(new Duration(5)))); + + DataflowAssert.that(output).containsInAnyOrder( + output("a", 1, 4, -5, 5), + output("a", 2, 9, 0, 10), + output("a", 1, 14, 5, 15), + output("b", 1, 9, 0, 10), + output("b", 1, 14, 5, 15)); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testMergingWindowing() { + Pipeline p = TestPipeline.create(); + PCollection input = + p.apply( + Create.timestamped( + TimestampedValue.of("a", new Instant(1)), + TimestampedValue.of("a", new Instant(5)), + TimestampedValue.of("a", new Instant(20)))); + + PCollection output = + input + .apply(new WindowedCount(Sessions.withGapDuration(new Duration(10)))); + + DataflowAssert.that(output).containsInAnyOrder( + output("a", 2, 14, 1, 15), + output("a", 1, 29, 20, 30)); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testWindowPreservation() { + Pipeline p = TestPipeline.create(); + PCollection input1 = p.apply( + Create.timestamped( + TimestampedValue.of("a", new Instant(1)), + TimestampedValue.of("b", new Instant(2)))); + + PCollection input2 = p.apply( + Create.timestamped( + TimestampedValue.of("a", new Instant(3)), + TimestampedValue.of("b", new Instant(4)))); + + PCollectionList input = PCollectionList.of(input1).and(input2); + + PCollection output = + input + .apply(Flatten.create()) + .apply(new WindowedCount(FixedWindows.of(new Duration(5)))); + + DataflowAssert.that(output).containsInAnyOrder( + output("a", 2, 4, 0, 5), + output("b", 2, 4, 0, 5)); + + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testElementsSortedByTimestamp() { + // The Windowing API does not guarantee that elements will be sorted by + // timestamp, but the implementation currently relies on this, so it + // needs to be tested. + + Pipeline p = TestPipeline.create(); + + PCollection> a = p + .apply(Create.timestamped( + TimestampedValue.of(KV.of("k", "a"), new Instant(1)), + TimestampedValue.of(KV.of("k", "b"), new Instant(4)), + TimestampedValue.of(KV.of("k", "c"), new Instant(3)), + TimestampedValue.of(KV.of("k", "d"), new Instant(5)), + TimestampedValue.of(KV.of("k", "e"), new Instant(2)), + TimestampedValue.of(KV.of("k", "f"), new Instant(-5)), + TimestampedValue.of(KV.of("k", "g"), new Instant(-6)), + TimestampedValue.of(KV.of("k", "h"), new Instant(-255)), + TimestampedValue.of(KV.of("k", "i"), new Instant(-256)), + TimestampedValue.of(KV.of("k", "j"), new Instant(255)))); + + PCollection> b = a + .apply(Window.>into( + FixedWindows.of(new Duration(1000)).withOffset(new Duration(500)))); + + PCollection>> output = b + .apply(GroupByKey.create()); + + DataflowAssert.that(output).containsInAnyOrder( + KV.of("k", + (Iterable) Arrays.asList("i", "h", "g", "f", "a", "e", "c", "b", "d", "j"))); + + p.run(); + } + + @Test + public void testEmptyInput() { + Pipeline p = TestPipeline.create(); + PCollection input = + p.apply(Create.timestamped()) + .setCoder(StringUtf8Coder.of()); + + PCollection output = + input + .apply(new WindowedCount(FixedWindows.of(new Duration(10)))); + + DataflowAssert.that(output).containsInAnyOrder(); + + p.run(); + } + + @Test + public void testTextIoInput() throws Exception { + File tmpFile = tmpFolder.newFile("file.txt"); + String filename = tmpFile.getPath(); + + try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) { + writer.println("a 1"); + writer.println("b 2"); + writer.println("b 3"); + writer.println("c 11"); + writer.println("d 11"); + } + + Pipeline p = TestPipeline.create(); + PCollection output = p.begin() + .apply(TextIO.Read.named("ReadLines").from(filename)) + .apply(ParDo.of(new ExtractWordsWithTimestampsFn())) + .apply(new WindowedCount(FixedWindows.of(Duration.millis(10)))); + + DataflowAssert.that(output).containsInAnyOrder( + output("a", 1, 9, 0, 10), + output("b", 2, 9, 0, 10), + output("c", 1, 19, 10, 20), + output("d", 1, 19, 10, 20)); + + p.run(); + } + + /** A DoFn that tokenizes lines of text into individual words. */ + static class ExtractWordsWithTimestampsFn extends DoFn { + @Override + public void processElement(ProcessContext c) { + String[] words = c.element().split("[^a-zA-Z0-9']+"); + if (words.length == 2) { + c.outputWithTimestamp(words[0], new Instant(Long.parseLong(words[1]))); + } + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java new file mode 100644 index 0000000000000..45cc267d2d740 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java @@ -0,0 +1,194 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MAX; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MIN; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; + +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.Max; +import com.google.cloud.dataflow.sdk.transforms.Min; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.CounterTestUtils; + +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Unit tests for the {@link Aggregator} API. + */ +@RunWith(JUnit4.class) +public class AggregatorImplTest { + @Rule + public final ExpectedException expectedEx = ExpectedException.none(); + + private static final String AGGREGATOR_NAME = "aggregator_name"; + + private void testAggregator(List items, + SerializableFunction, V> combiner, + Counter expectedCounter) { + CounterSet counters = new CounterSet(); + Aggregator aggregator = new AggregatorImpl, V>( + AGGREGATOR_NAME, combiner, counters.getAddCounterMutator()); + for (V item : items) { + aggregator.addValue(item); + } + + List cloudCounterSet = CounterTestUtils.extractCounterUpdates(counters, false); + Assert.assertEquals(cloudCounterSet.size(), 1); + Assert.assertEquals(cloudCounterSet.get(0), + CounterTestUtils.extractCounterUpdate(expectedCounter, false)); + } + + @Test + public void testSumInteger() throws Exception { + testAggregator(Arrays.asList(2, 4, 1, 3), new Sum.SumIntegerFn(), + Counter.ints(AGGREGATOR_NAME, SUM).resetToValue(10)); + } + + @Test + public void testSumLong() throws Exception { + testAggregator(Arrays.asList(2L, 4L, 1L, 3L), new Sum.SumLongFn(), + Counter.longs(AGGREGATOR_NAME, SUM).resetToValue(10L)); + } + + @Test + public void testSumDouble() throws Exception { + testAggregator(Arrays.asList(2.0, 4.1, 1.0, 3.1), new Sum.SumDoubleFn(), + Counter.doubles(AGGREGATOR_NAME, SUM).resetToValue(10.2)); + } + + @Test + public void testMinInteger() throws Exception { + testAggregator(Arrays.asList(2, 4, 1, 3), new Min.MinIntegerFn(), + Counter.ints(AGGREGATOR_NAME, MIN).resetToValue(1)); + } + + @Test + public void testMinLong() throws Exception { + testAggregator(Arrays.asList(2L, 4L, 1L, 3L), new Min.MinLongFn(), + Counter.longs(AGGREGATOR_NAME, MIN).resetToValue(1L)); + } + + @Test + public void testMinDouble() throws Exception { + testAggregator(Arrays.asList(2.0, 4.1, 1.0, 3.1), new Min.MinDoubleFn(), + Counter.doubles(AGGREGATOR_NAME, MIN).resetToValue(1.0)); + } + + @Test + public void testMaxInteger() throws Exception { + testAggregator(Arrays.asList(2, 4, 1, 3), new Max.MaxIntegerFn(), + Counter.ints(AGGREGATOR_NAME, MAX).resetToValue(4)); + } + + @Test + public void testMaxLong() throws Exception { + testAggregator(Arrays.asList(2L, 4L, 1L, 3L), new Max.MaxLongFn(), + Counter.longs(AGGREGATOR_NAME, MAX).resetToValue(4L)); + } + + @Test + public void testMaxDouble() throws Exception { + testAggregator(Arrays.asList(2.0, 4.1, 1.0, 3.1), new Max.MaxDoubleFn(), + Counter.doubles(AGGREGATOR_NAME, MAX).resetToValue(4.1)); + } + + @Test + public void testCompatibleDuplicateNames() throws Exception { + CounterSet counters = new CounterSet(); + Aggregator aggregator1 = + new AggregatorImpl, Integer>( + AGGREGATOR_NAME, new Sum.SumIntegerFn(), + counters.getAddCounterMutator()); + + Aggregator aggregator2 = + new AggregatorImpl, Integer>( + AGGREGATOR_NAME, new Sum.SumIntegerFn(), + counters.getAddCounterMutator()); + + // The duplicate aggregators should update the same counter. + aggregator1.addValue(3); + aggregator2.addValue(4); + Assert.assertEquals( + new CounterSet(Counter.ints(AGGREGATOR_NAME, SUM).resetToValue(7)), + counters); + } + + @Test + public void testIncompatibleDuplicateNames() throws Exception { + CounterSet counters = new CounterSet(); + new AggregatorImpl, Integer>( + AGGREGATOR_NAME, new Sum.SumIntegerFn(), + counters.getAddCounterMutator()); + + expectedEx.expect(IllegalArgumentException.class); + expectedEx.expectMessage(Matchers.containsString( + "aggregator's name collides with an existing aggregator or " + + "system-provided counter of an incompatible type")); + new AggregatorImpl, Long>( + AGGREGATOR_NAME, new Sum.SumLongFn(), + counters.getAddCounterMutator()); + } + + @Test + public void testUnsupportedCombineFn() throws Exception { + expectedEx.expect(IllegalArgumentException.class); + expectedEx.expectMessage(Matchers.containsString("unsupported combiner")); + new AggregatorImpl<>( + AGGREGATOR_NAME, + new Combine.CombineFn, Integer>() { + @Override + public List createAccumulator() { return null; } + @Override + public void addInput(List accumulator, Integer input) { } + @Override + public List mergeAccumulators(Iterable> accumulators) { + return null; } + @Override + public Integer extractOutput(List accumulator) { return null; } + }, + (new CounterSet()).getAddCounterMutator()); + } + + @Test + public void testUnsupportedSerializableFunction() throws Exception { + expectedEx.expect(IllegalArgumentException.class); + expectedEx.expectMessage(Matchers.containsString("unsupported combiner")); + new AggregatorImpl, Integer>( + AGGREGATOR_NAME, + new SerializableFunction, Integer>() { + @Override + public Integer apply(Iterable input) { return null; } + }, + (new CounterSet()).getAddCounterMutator()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOffTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOffTest.java new file mode 100644 index 0000000000000..0c262e2f1cb04 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOffTest.java @@ -0,0 +1,71 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.lessThan; +import static org.junit.Assert.assertEquals; + +import com.google.api.client.util.BackOff; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link AttemptBoundedExponentialBackOff}. */ +@RunWith(JUnit4.class) +public class AttemptBoundedExponentialBackOffTest { + @Rule public ExpectedException exception = ExpectedException.none(); + + @Test + public void testUsingInvalidInitialInterval() throws Exception { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Initial interval must be greater than zero."); + new AttemptBoundedExponentialBackOff(10, 0L); + } + + @Test + public void testUsingInvalidMaximumNumberOfRetries() throws Exception { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Maximum number of attempts must be greater than zero."); + new AttemptBoundedExponentialBackOff(-1, 10L); + } + + @Test + public void testThatFixedNumberOfAttemptsExits() throws Exception { + BackOff backOff = new AttemptBoundedExponentialBackOff(3, 500); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); + assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); + } + + @Test + public void testThatResettingAllowsReuse() throws Exception { + BackOff backOff = new AttemptBoundedExponentialBackOff(3, 500); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); + assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); + backOff.reset(); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); + assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/Base64UtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/Base64UtilsTest.java new file mode 100644 index 0000000000000..d557284ce0806 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/Base64UtilsTest.java @@ -0,0 +1,53 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.api.client.util.Base64.encodeBase64URLSafeString; + +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThan; +import static org.junit.Assert.assertThat; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link Base64Utils}. */ +@RunWith(JUnit4.class) +public class Base64UtilsTest { + void testLength(int length) { + byte[] b = new byte[length]; + // Make sure that the estimated length is an upper bound. + assertThat( + Base64Utils.getBase64Length(length), + greaterThanOrEqualTo(encodeBase64URLSafeString(b).length())); + // Make sure that it's a tight upper bound (no more than 4 characters off). + assertThat( + Base64Utils.getBase64Length(length), + lessThan(4 + encodeBase64URLSafeString(b).length())); + } + + @Test + public void getBase64Length() { + for (int i = 0; i < 100; ++i) { + testLength(i); + } + for (int i = 1000; i < 1100; ++i) { + testLength(i); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BigQueryUtilTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BigQueryUtilTest.java new file mode 100644 index 0000000000000..ca75e6f94ca7b --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BigQueryUtilTest.java @@ -0,0 +1,306 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableCell; +import com.google.api.services.bigquery.model.TableDataList; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.common.base.Function; +import com.google.common.collect.Iterators; + +import org.hamcrest.Matchers; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.io.IOException; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; + +/** + * Tests for util classes related to BigQuery. + */ +@RunWith(JUnit4.class) +public class BigQueryUtilTest { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Mock private Bigquery mockClient; + @Mock private Bigquery.Tables mockTables; + @Mock private Bigquery.Tables.Get mockTablesGet; + @Mock private Bigquery.Tabledata mockTabledata; + @Mock private Bigquery.Tabledata.List mockTabledataList; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + } + + @After + public void tearDown() { + verifyNoMoreInteractions(mockClient); + verifyNoMoreInteractions(mockTables); + verifyNoMoreInteractions(mockTablesGet); + verifyNoMoreInteractions(mockTabledata); + verifyNoMoreInteractions(mockTabledataList); + } + + private void onTableGet(Table table) throws IOException { + when(mockClient.tables()) + .thenReturn(mockTables); + when(mockTables.get(anyString(), anyString(), anyString())) + .thenReturn(mockTablesGet); + when(mockTablesGet.execute()) + .thenReturn(table); + } + + private void verifyTableGet() throws IOException { + verify(mockClient).tables(); + verify(mockTables).get("project", "dataset", "table"); + verify(mockTablesGet).execute(); + } + + private void onTableList(TableDataList result) throws IOException { + when(mockClient.tabledata()) + .thenReturn(mockTabledata); + when(mockTabledata.list(anyString(), anyString(), anyString())) + .thenReturn(mockTabledataList); + when(mockTabledataList.execute()) + .thenReturn(result); + } + + private void verifyTabledataList() throws IOException { + verify(mockClient, atLeastOnce()).tabledata(); + verify(mockTabledata, atLeastOnce()).list("project", "dataset", "table"); + verify(mockTabledataList, atLeastOnce()).execute(); + // Max results may be set when testing for an empty table. + verify(mockTabledataList, atLeast(0)).setMaxResults(anyLong()); + } + + private Table basicTableSchema() { + return new Table() + .setSchema(new TableSchema() + .setFields(Arrays.asList( + new TableFieldSchema() + .setName("name") + .setType("STRING"), + new TableFieldSchema() + .setName("answer") + .setType("INTEGER") + ))); + } + + private TableRow rawRow(Object...args) { + List cells = new LinkedList<>(); + for (Object a : args) { + cells.add(new TableCell().setV(a)); + } + return new TableRow().setF(cells); + } + + private TableDataList rawDataList(TableRow...rows) { + return new TableDataList() + .setRows(Arrays.asList(rows)); + } + + @Test + public void testRead() throws IOException { + onTableGet(basicTableSchema()); + + TableDataList dataList = rawDataList(rawRow("Arthur", 42)); + onTableList(dataList); + + BigQueryTableRowIterator iterator = new BigQueryTableRowIterator( + mockClient, + BigQueryIO.parseTableSpec("project:dataset.table")); + + Assert.assertTrue(iterator.hasNext()); + TableRow row = iterator.next(); + + Assert.assertTrue(row.containsKey("name")); + Assert.assertTrue(row.containsKey("answer")); + Assert.assertEquals("Arthur", row.get("name")); + Assert.assertEquals(42, row.get("answer")); + + Assert.assertFalse(iterator.hasNext()); + + verifyTableGet(); + verifyTabledataList(); + } + + @Test + public void testReadEmpty() throws IOException { + onTableGet(basicTableSchema()); + + // BigQuery may respond with a page token for an empty table, ensure we + // handle it. + TableDataList dataList = new TableDataList() + .setPageToken("FEED==") + .setTotalRows(0L); + onTableList(dataList); + + BigQueryTableRowIterator iterator = new BigQueryTableRowIterator( + mockClient, + BigQueryIO.parseTableSpec("project:dataset.table")); + + Assert.assertFalse(iterator.hasNext()); + + verifyTableGet(); + verifyTabledataList(); + } + + @Test + public void testReadMultiPage() throws IOException { + onTableGet(basicTableSchema()); + + TableDataList page1 = rawDataList(rawRow("Row1", 1)) + .setPageToken("page2"); + TableDataList page2 = rawDataList(rawRow("Row2", 2)) + .setTotalRows(2L); + + when(mockClient.tabledata()) + .thenReturn(mockTabledata); + when(mockTabledata.list(anyString(), anyString(), anyString())) + .thenReturn(mockTabledataList); + when(mockTabledataList.execute()) + .thenReturn(page1) + .thenReturn(page2); + + BigQueryTableRowIterator iterator = new BigQueryTableRowIterator( + mockClient, + BigQueryIO.parseTableSpec("project:dataset.table")); + List names = new LinkedList<>(); + Iterators.addAll(names, + Iterators.transform(iterator, new Function(){ + @Override + public String apply(TableRow input) { + return (String) input.get("name"); + } + })); + + Assert.assertThat(names, Matchers.hasItems("Row1", "Row2")); + + verifyTableGet(); + verifyTabledataList(); + // The second call should have used a page token. + verify(mockTabledataList).setPageToken("page2"); + } + + @Test + public void testReadOpenFailure() throws IOException { + thrown.expect(RuntimeException.class); + + when(mockClient.tables()) + .thenReturn(mockTables); + when(mockTables.get(anyString(), anyString(), anyString())) + .thenReturn(mockTablesGet); + when(mockTablesGet.execute()) + .thenThrow(new IOException("No such table")); + + BigQueryTableRowIterator iterator = new BigQueryTableRowIterator( + mockClient, + BigQueryIO.parseTableSpec("project:dataset.table")); + try { + Assert.assertFalse(iterator.hasNext()); // throws. + } finally { + verifyTableGet(); + } + } + + @Test + public void testWriteAppend() throws IOException { + onTableGet(basicTableSchema()); + + TableReference ref = BigQueryIO + .parseTableSpec("project:dataset.table"); + + BigQueryTableInserter inserter = + new BigQueryTableInserter(mockClient, ref); + + inserter.getOrCreateTable(BigQueryIO.Write.WriteDisposition.WRITE_APPEND, + BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null); + + verifyTableGet(); + } + + @Test + public void testWriteEmpty() throws IOException { + onTableGet(basicTableSchema()); + + TableDataList dataList = new TableDataList().setTotalRows(0L); + onTableList(dataList); + + TableReference ref = BigQueryIO + .parseTableSpec("project:dataset.table"); + + BigQueryTableInserter inserter = + new BigQueryTableInserter(mockClient, ref); + + inserter.getOrCreateTable(BigQueryIO.Write.WriteDisposition.WRITE_EMPTY, + BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null); + + verifyTableGet(); + verifyTabledataList(); + } + + @Test + public void testWriteEmptyFail() throws IOException { + thrown.expect(IOException.class); + + onTableGet(basicTableSchema()); + + TableDataList dataList = rawDataList(rawRow("Arthur", 42)); + onTableList(dataList); + + TableReference ref = BigQueryIO + .parseTableSpec("project:dataset.table"); + + BigQueryTableInserter inserter = + new BigQueryTableInserter(mockClient, ref); + + try { + inserter.getOrCreateTable(BigQueryIO.Write.WriteDisposition.WRITE_EMPTY, + BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null); + } finally { + verifyTableGet(); + verifyTabledataList(); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CloudMetricUtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CloudMetricUtilsTest.java new file mode 100644 index 0000000000000..31bc2f9241eaf --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CloudMetricUtilsTest.java @@ -0,0 +1,66 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.junit.Assert.assertEquals; + +import com.google.api.services.dataflow.model.MetricStructuredName; +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.cloud.dataflow.sdk.util.common.Metric; +import com.google.cloud.dataflow.sdk.util.common.Metric.DoubleMetric; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** Unit tests for {@link CloudMetricUtils}. */ +@RunWith(JUnit4.class) +public class CloudMetricUtilsTest { + private void addDoubleMetric(String name, double value, String workerId, + List> metrics, + List cloudMetrics) { + metrics.add(new DoubleMetric(name, value)); + MetricStructuredName structuredName = new MetricStructuredName(); + structuredName.setName(name); + Map context = new HashMap<>(); + context.put("workerId", workerId); + structuredName.setContext(context); + cloudMetrics.add(new MetricUpdate() + .setName(structuredName) + .setScalar(CloudObject.forFloat(value))); + } + + @Test + public void testExtractCloudMetrics() { + List> metrics = new ArrayList<>(); + List expected = new ArrayList<>(); + String workerId = "worker-id"; + + addDoubleMetric("m1", 3.14, workerId, metrics, expected); + addDoubleMetric("m2", 2.17, workerId, metrics, expected); + addDoubleMetric("m3", -66.666, workerId, metrics, expected); + + List actual = CloudMetricUtils.extractCloudMetrics(metrics, workerId); + + assertEquals(expected, actual); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtilsTest.java new file mode 100644 index 0000000000000..d813a103fabb2 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtilsTest.java @@ -0,0 +1,83 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; +import static com.google.cloud.dataflow.sdk.util.Structs.getString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import com.google.api.services.dataflow.model.Source; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Map; + +/** + * Tests for {@code CloudSourceUtils}. + */ +@RunWith(JUnit4.class) +public class CloudSourceUtilsTest { + @Test + public void testFlattenBaseSpecs() throws Exception { + // G = grandparent, P = parent, C = child. + CloudObject grandparent = CloudObject.forClassName("text"); + addString(grandparent, "G", "g_g"); + addString(grandparent, "GP", "gp_g"); + addString(grandparent, "GC", "gc_g"); + addString(grandparent, "GPC", "gpc_g"); + + CloudObject parent = CloudObject.forClassName("text"); + addString(parent, "P", "p_p"); + addString(parent, "PC", "pc_p"); + addString(parent, "GP", "gp_p"); + addString(parent, "GPC", "gpc_p"); + + CloudObject child = CloudObject.forClassName("text"); + addString(child, "C", "c_c"); + addString(child, "PC", "pc_c"); + addString(child, "GC", "gc_c"); + addString(child, "GPC", "gpc_c"); + + Source source = new Source(); + source.setBaseSpecs(new ArrayList>()); + source.getBaseSpecs().add(grandparent); + source.getBaseSpecs().add(parent); + source.setSpec(child); + source.setCodec(makeCloudEncoding(StringUtf8Coder.class.getName())); + + Source flat = CloudSourceUtils.flattenBaseSpecs(source); + assertNull(flat.getBaseSpecs()); + assertEquals( + StringUtf8Coder.class.getName(), + getString(flat.getCodec(), PropertyNames.OBJECT_TYPE_NAME)); + + CloudObject flatSpec = CloudObject.fromSpec(flat.getSpec()); + assertEquals("g_g", getString(flatSpec, "G")); + assertEquals("p_p", getString(flatSpec, "P")); + assertEquals("c_c", getString(flatSpec, "C")); + assertEquals("gp_p", getString(flatSpec, "GP")); + assertEquals("gc_c", getString(flatSpec, "GC")); + assertEquals("pc_c", getString(flatSpec, "PC")); + assertEquals("gpc_c", getString(flatSpec, "GPC")); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CoderUtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CoderUtilsTest.java new file mode 100644 index 0000000000000..92f9e7481558f --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CoderUtilsTest.java @@ -0,0 +1,158 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.Coder.Context; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.InputStream; +import java.io.OutputStream; + +/** + * Tests for CoderUtils. + */ +@RunWith(JUnit4.class) +public class CoderUtilsTest { + static class TestCoder extends AtomicCoder { + public static TestCoder of() { return new TestCoder(); } + + @Override + public void encode(Integer value, OutputStream outStream, Context context) { + throw new RuntimeException("not expecting to be called"); + } + + @Override + public Integer decode(InputStream inStream, Context context) { + throw new RuntimeException("not expecting to be called"); + } + + @Override + public boolean isDeterministic() { + return false; + } + } + + @Test + public void testCreateAtomicCoders() throws Exception { + Assert.assertEquals( + BigEndianIntegerCoder.of(), + Serializer.deserialize(makeCloudEncoding("BigEndianIntegerCoder"), Coder.class)); + Assert.assertEquals( + StringUtf8Coder.of(), + Serializer.deserialize( + makeCloudEncoding(StringUtf8Coder.class.getName()), Coder.class)); + Assert.assertEquals( + VoidCoder.of(), + Serializer.deserialize(makeCloudEncoding("VoidCoder"), Coder.class)); + Assert.assertEquals( + TestCoder.of(), + Serializer.deserialize(makeCloudEncoding(TestCoder.class.getName()), Coder.class)); + } + + @Test + public void testCreateCompositeCoders() throws Exception { + Assert.assertEquals( + IterableCoder.of(StringUtf8Coder.of()), + Serializer.deserialize( + makeCloudEncoding("IterableCoder", + makeCloudEncoding("StringUtf8Coder")), Coder.class)); + Assert.assertEquals( + KvCoder.of(BigEndianIntegerCoder.of(), VoidCoder.of()), + Serializer.deserialize( + makeCloudEncoding( + "KvCoder", + makeCloudEncoding(BigEndianIntegerCoder.class.getName()), + makeCloudEncoding("VoidCoder")), Coder.class)); + Assert.assertEquals( + IterableCoder.of( + KvCoder.of(IterableCoder.of(BigEndianIntegerCoder.of()), + KvCoder.of(VoidCoder.of(), + TestCoder.of()))), + Serializer.deserialize( + makeCloudEncoding( + IterableCoder.class.getName(), + makeCloudEncoding( + KvCoder.class.getName(), + makeCloudEncoding( + "IterableCoder", + makeCloudEncoding("BigEndianIntegerCoder")), + makeCloudEncoding( + "KvCoder", + makeCloudEncoding("VoidCoder"), + makeCloudEncoding(TestCoder.class.getName())))), Coder.class)); + } + + @Test + public void testCreateUntypedCoders() throws Exception { + Assert.assertEquals( + IterableCoder.of(StringUtf8Coder.of()), + Serializer.deserialize( + makeCloudEncoding( + "kind:stream", + makeCloudEncoding("StringUtf8Coder")), Coder.class)); + Assert.assertEquals( + KvCoder.of(BigEndianIntegerCoder.of(), VoidCoder.of()), + Serializer.deserialize( + makeCloudEncoding( + "kind:pair", + makeCloudEncoding(BigEndianIntegerCoder.class.getName()), + makeCloudEncoding("VoidCoder")), Coder.class)); + Assert.assertEquals( + IterableCoder.of( + KvCoder.of(IterableCoder.of(BigEndianIntegerCoder.of()), + KvCoder.of(VoidCoder.of(), + TestCoder.of()))), + Serializer.deserialize( + makeCloudEncoding( + "kind:stream", + makeCloudEncoding( + "kind:pair", + makeCloudEncoding( + "kind:stream", + makeCloudEncoding("BigEndianIntegerCoder")), + makeCloudEncoding( + "kind:pair", + makeCloudEncoding("VoidCoder"), + makeCloudEncoding(TestCoder.class.getName())))), Coder.class)); + } + + @Test + public void testCreateUnknownCoder() throws Exception { + try { + Serializer.deserialize(makeCloudEncoding("UnknownCoder"), Coder.class); + Assert.fail("should have thrown an exception"); + } catch (Exception exn) { + Assert.assertThat(exn.toString(), + CoreMatchers.containsString( + "Unable to convert coder ID UnknownCoder to class")); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GcsUtilTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GcsUtilTest.java new file mode 100644 index 0000000000000..cae705cea5a46 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GcsUtilTest.java @@ -0,0 +1,105 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +import com.google.api.client.auth.oauth2.Credential; +import com.google.api.client.util.Throwables; +import com.google.cloud.dataflow.sdk.options.GcsOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mockito; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +/** Test case for {@link GcsUtil}. */ +@RunWith(JUnit4.class) +public class GcsUtilTest { + @Test + public void testGlobTranslation() { + assertEquals("foo", GcsUtil.globToRegexp("foo")); + assertEquals("fo[^/]*o", GcsUtil.globToRegexp("fo*o")); + assertEquals("f[^/]*o\\.[^/]", GcsUtil.globToRegexp("f*o.?")); + assertEquals("foo-[0-9][^/]*", GcsUtil.globToRegexp("foo-[0-9]*")); + } + + @Test + public void testCreationWithDefaultOptions() { + GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class); + pipelineOptions.setGcpCredential(Mockito.mock(Credential.class)); + assertNotNull(pipelineOptions.getGcpCredential()); + } + + @Test + public void testCreationWithExecutorServiceProvided() { + GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class); + pipelineOptions.setGcpCredential(Mockito.mock(Credential.class)); + pipelineOptions.setExecutorService(Executors.newCachedThreadPool()); + assertSame(pipelineOptions.getExecutorService(), pipelineOptions.getGcsUtil().executorService); + } + + @Test + public void testCreationWithGcsUtilProvided() { + GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class); + GcsUtil gcsUtil = Mockito.mock(GcsUtil.class); + pipelineOptions.setGcsUtil(gcsUtil); + assertSame(gcsUtil, pipelineOptions.getGcsUtil()); + } + + @Test + public void testMultipleThreadsCanCompleteOutOfOrderWithDefaultThreadPool() throws Exception { + GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class); + ExecutorService executorService = pipelineOptions.getExecutorService(); + + int numThreads = 1000; + final CountDownLatch[] countDownLatches = new CountDownLatch[numThreads]; + for (int i = 0; i < numThreads; i++) { + final int currentLatch = i; + countDownLatches[i] = new CountDownLatch(1); + executorService.execute(new Runnable() { + @Override + public void run() { + // Wait for latch N and then release latch N - 1 + try { + countDownLatches[currentLatch].await(); + if (currentLatch > 0) { + countDownLatches[currentLatch - 1].countDown(); + } + } catch (InterruptedException e) { + throw Throwables.propagate(e); + } + } + }); + } + + // Release the last latch starting the chain reaction. + countDownLatches[countDownLatches.length - 1].countDown(); + executorService.shutdown(); + assertTrue("Expected tasks to complete", + executorService.awaitTermination(10, TimeUnit.SECONDS)); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java new file mode 100644 index 0000000000000..d482d2c4d345a --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java @@ -0,0 +1,231 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; +import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import org.hamcrest.Matchers; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** Unit tests for {@link GroupAlsoByWindowsDoFn}. */ +@RunWith(JUnit4.class) +public class GroupAlsoByWindowsDoFnTest { + ExecutionContext execContext; + CounterSet counters; + TupleTag>> outputTag; + + @Before public void setUp() { + execContext = new DirectModeExecutionContext(); + counters = new CounterSet(); + outputTag = new TupleTag<>(); + } + + @Test public void testEmpty() throws Exception { + DoFnRunner>>, + KV>, List> runner = + makeRunner(FixedWindows.of(Duration.millis(10))); + + runner.startBundle(); + + runner.finishBundle(); + + List>> result = runner.getReceiver(outputTag); + + assertEquals(0, result.size()); + } + + @Test public void testFixedWindows() throws Exception { + DoFnRunner>>, + KV>, List> runner = + makeRunner(FixedWindows.of(Duration.millis(10))); + + runner.startBundle(); + + runner.processElement(WindowedValue.valueInEmptyWindows( + KV.of("k", (Iterable>) Arrays.asList( + WindowedValue.of( + "v1", + new Instant(1), + Arrays.asList(window(0, 10))), + WindowedValue.of( + "v2", + new Instant(2), + Arrays.asList(window(0, 10))), + WindowedValue.of( + "v3", + new Instant(13), + Arrays.asList(window(10, 20))))))); + + runner.finishBundle(); + + List>>> result = runner.getReceiver(outputTag); + + assertEquals(2, result.size()); + + WindowedValue>> item0 = result.get(0); + assertEquals("k", item0.getValue().getKey()); + assertThat(item0.getValue().getValue(), Matchers.contains("v1", "v2")); + assertEquals(new Instant(9), item0.getTimestamp()); + assertThat(item0.getWindows(), + Matchers.contains(window(0, 10))); + + WindowedValue>> item1 = result.get(1); + assertEquals("k", item1.getValue().getKey()); + assertThat(item1.getValue().getValue(), Matchers.contains("v3")); + assertEquals(new Instant(19), item1.getTimestamp()); + assertThat(item1.getWindows(), + Matchers.contains(window(10, 20))); + } + + @Test public void testSlidingWindows() throws Exception { + DoFnRunner>>, + KV>, List> runner = + makeRunner(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10))); + + runner.startBundle(); + + runner.processElement(WindowedValue.valueInEmptyWindows( + KV.of("k", (Iterable>) Arrays.asList( + WindowedValue.of( + "v1", + new Instant(5), + Arrays.asList(window(-10, 10), window(0, 20))), + WindowedValue.of( + "v2", + new Instant(15), + Arrays.asList(window(0, 20), window(10, 30))))))); + + runner.finishBundle(); + + List>>> result = runner.getReceiver(outputTag); + + assertEquals(3, result.size()); + + WindowedValue>> item0 = result.get(0); + assertEquals("k", item0.getValue().getKey()); + assertThat(item0.getValue().getValue(), Matchers.contains("v1")); + assertEquals(new Instant(9), item0.getTimestamp()); + assertThat(item0.getWindows(), + Matchers.contains(window(-10, 10))); + + WindowedValue>> item1 = result.get(1); + assertEquals("k", item1.getValue().getKey()); + assertThat(item1.getValue().getValue(), Matchers.contains("v1", "v2")); + assertEquals(new Instant(19), item1.getTimestamp()); + assertThat(item1.getWindows(), + Matchers.contains(window(0, 20))); + + WindowedValue>> item2 = result.get(2); + assertEquals("k", item2.getValue().getKey()); + assertThat(item2.getValue().getValue(), Matchers.contains("v2")); + assertEquals(new Instant(29), item2.getTimestamp()); + assertThat(item2.getWindows(), + Matchers.contains(window(10, 30))); + } + + @Test public void testSessions() throws Exception { + DoFnRunner>>, + KV>, List> runner = + makeRunner(Sessions.withGapDuration(Duration.millis(10))); + + runner.startBundle(); + + runner.processElement(WindowedValue.valueInEmptyWindows( + KV.of("k", (Iterable>) Arrays.asList( + WindowedValue.of( + "v1", + new Instant(0), + Arrays.asList(window(0, 10))), + WindowedValue.of( + "v2", + new Instant(5), + Arrays.asList(window(5, 15))), + WindowedValue.of( + "v3", + new Instant(15), + Arrays.asList(window(15, 25))))))); + + runner.finishBundle(); + + List>>> result = runner.getReceiver(outputTag); + + assertEquals(2, result.size()); + + WindowedValue>> item0 = result.get(0); + assertEquals("k", item0.getValue().getKey()); + assertThat(item0.getValue().getValue(), Matchers.contains("v1", "v2")); + assertEquals(new Instant(14), item0.getTimestamp()); + assertThat(item0.getWindows(), + Matchers.contains(window(0, 15))); + + WindowedValue>> item1 = result.get(1); + assertEquals("k", item1.getValue().getKey()); + assertThat(item1.getValue().getValue(), Matchers.contains("v3")); + assertEquals(new Instant(24), item1.getTimestamp()); + assertThat(item1.getWindows(), + Matchers.contains(window(15, 25))); + } + + + private DoFnRunner>>, + KV>, List> makeRunner( + WindowingFn windowingFn) { + + GroupAlsoByWindowsDoFn fn = + new GroupAlsoByWindowsDoFn( + windowingFn, StringUtf8Coder.of()); + + DoFnRunner>>, + KV>, List> runner = + DoFnRunner.createWithListOutputs( + PipelineOptionsFactory.create(), + fn, + PTuple.empty(), + outputTag, + new ArrayList>(), + execContext.createStepContext("merge"), + counters.getAddCounterMutator()); + + return runner; + } + + private BoundedWindow window(long start, long end) { + return new IntervalWindow(new Instant(start), new Instant(end)); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOChannelUtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOChannelUtilsTest.java new file mode 100644 index 0000000000000..fe82972044d03 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOChannelUtilsTest.java @@ -0,0 +1,76 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.File; +import java.nio.channels.WritableByteChannel; + +/** + * Tests for IOChannelUtils. + */ +@RunWith(JUnit4.class) +public class IOChannelUtilsTest { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + @Test + public void testShardFormatExpansion() { + Assert.assertEquals("output-001-of-123.txt", + IOChannelUtils.constructName("output", "-SSS-of-NNN", + ".txt", + 1, 123)); + + Assert.assertEquals("out.txt/part-00042", + IOChannelUtils.constructName("out.txt", "/part-SSSSS", "", + 42, 100)); + + Assert.assertEquals("out.txt", + IOChannelUtils.constructName("ou", "t.t", "xt", 1, 1)); + + Assert.assertEquals("out0102shard.txt", + IOChannelUtils.constructName("out", "SSNNshard", ".txt", 1, 2)); + + Assert.assertEquals("out-2/1.part-1-of-2.txt", + IOChannelUtils.constructName("out", "-N/S.part-S-of-N", + ".txt", 1, 2)); + } + + @Test(expected = IllegalArgumentException.class) + public void testShardNameCollision() throws Exception { + File outFolder = tmpFolder.newFolder(); + String filename = outFolder.toPath().resolve("output").toString(); + + WritableByteChannel output = IOChannelUtils + .create(filename, "", "", 2, "text"); + Assert.fail("IOChannelUtils.create expected to fail due " + + "to filename collision"); + } + + @Test + public void testLargeShardCount() { + Assert.assertEquals("out-100-of-5000.txt", + IOChannelUtils.constructName("out", "-SS-of-NN", ".txt", + 100, 5000)); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java new file mode 100644 index 0000000000000..fbf2f70b22355 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java @@ -0,0 +1,99 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.runners.worker.TextSource; +import com.google.cloud.dataflow.sdk.util.common.worker.Source; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.File; +import java.io.FileOutputStream; +import java.util.Collection; +import java.util.Set; +import java.util.TreeSet; + +/** + * Tests for IOFactory. + */ +@RunWith(JUnit4.class) +public class IOFactoryTest { + + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + @Test + public void testLocalFileIO() throws Exception { + // Create some files to match against. + File foo1 = tmpFolder.newFile("foo1"); + foo1.createNewFile(); + File foo2 = tmpFolder.newFile("foo2"); + foo2.createNewFile(); + tmpFolder.newFile("barf").createNewFile(); + + FileIOChannelFactory factory = new FileIOChannelFactory(); + Collection paths = factory.match(tmpFolder.getRoot() + "/f*"); + + Assert.assertEquals(2, paths.size()); + Assert.assertTrue(paths.contains(foo1.getCanonicalPath())); + Assert.assertTrue(paths.contains(foo2.getCanonicalPath())); + } + + @Test + public void testMultiFileRead() throws Exception { + File file1 = tmpFolder.newFile("file1"); + FileOutputStream output = new FileOutputStream(file1); + output.write("1\n2".getBytes()); + output.close(); + + File file2 = tmpFolder.newFile("file2"); + output = new FileOutputStream(file2); + output.write("3\n4\n".getBytes()); + output.close(); + + File file3 = tmpFolder.newFile("file3"); + output = new FileOutputStream(file3); + output.write("5".getBytes()); + output.close(); + + + TextSource source = new TextSource<>( + tmpFolder.getRoot() + "/file*", + true /* strip newlines */, + null, null, StringUtf8Coder.of()); + + Set records = new TreeSet<>(); + try (Source.SourceIterator iterator = source.iterator()) { + while (iterator.hasNext()) { + records.add(iterator.next()); + } + } + + Assert.assertEquals(records.toString(), 5, records.size()); + Assert.assertTrue(records.contains("1")); + Assert.assertTrue(records.contains("2")); + Assert.assertTrue(records.contains("3")); + Assert.assertTrue(records.contains("4")); + Assert.assertTrue(records.contains("5")); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/InstanceBuilderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/InstanceBuilderTest.java new file mode 100644 index 0000000000000..18777b2aa3942 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/InstanceBuilderTest.java @@ -0,0 +1,114 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests of InstanceBuilder. + */ +@RunWith(JUnit4.class) +public class InstanceBuilderTest { + + @Rule + public ExpectedException expectedEx = ExpectedException.none(); + + @SuppressWarnings("unused") + private static TupleTag createTag(String id) { + return new TupleTag(id); + } + + @Test + public void testFullNameLookup() throws Exception { + TupleTag tag = InstanceBuilder.ofType(TupleTag.class) + .fromClassName(InstanceBuilderTest.class.getName()) + .fromFactoryMethod("createTag") + .withArg(String.class, "hello world!") + .build(); + + Assert.assertEquals("hello world!", tag.getId()); + } + + @Test + public void testConstructor() throws Exception { + TupleTag tag = InstanceBuilder.ofType(TupleTag.class) + .withArg(String.class, "hello world!") + .build(); + + Assert.assertEquals("hello world!", tag.getId()); + } + + @Test + public void testBadMethod() throws Exception { + expectedEx.expect(RuntimeException.class); + expectedEx.expectMessage( + Matchers.containsString("Unable to find factory method")); + + InstanceBuilder.ofType(String.class) + .fromClassName(InstanceBuilderTest.class.getName()) + .fromFactoryMethod("nonexistantFactoryMethod") + .withArg(String.class, "hello") + .withArg(String.class, " world!") + .build(); + } + + @Test + public void testBadArgs() throws Exception { + expectedEx.expect(RuntimeException.class); + expectedEx.expectMessage( + Matchers.containsString("Unable to find factory method")); + + InstanceBuilder.ofType(TupleTag.class) + .fromClassName(InstanceBuilderTest.class.getName()) + .fromFactoryMethod("createTag") + .withArg(String.class, "hello") + .withArg(Integer.class, 42) + .build(); + } + + @Test + public void testBadReturnType() throws Exception { + expectedEx.expect(RuntimeException.class); + expectedEx.expectMessage( + Matchers.containsString("must be assignable to String")); + + InstanceBuilder.ofType(String.class) + .fromClassName(InstanceBuilderTest.class.getName()) + .fromFactoryMethod("createTag") + .withArg(String.class, "hello") + .build(); + } + + @Test + public void testWrongType() throws Exception { + expectedEx.expect(RuntimeException.class); + expectedEx.expectMessage( + Matchers.containsString("must be assignable to TupleTag")); + + InstanceBuilder.ofType(TupleTag.class) + .fromClassName(InstanceBuilderTest.class.getName()) + .build(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/MonitoringUtilTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/MonitoringUtilTest.java new file mode 100644 index 0000000000000..8ec3012da4482 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/MonitoringUtilTest.java @@ -0,0 +1,90 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.JobMessage; +import com.google.api.services.dataflow.model.ListJobMessagesResponse; + +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Tests for MonitoringUtil. + */ +@RunWith(JUnit4.class) +public class MonitoringUtilTest { + private static final String PROJECT_ID = "someProject"; + private static final String JOB_ID = "1234"; + + @Test + public void testGetJobMessages() throws IOException { + Dataflow.V1b3.Projects.Jobs.Messages mockMessages = + mock(Dataflow.V1b3.Projects.Jobs.Messages.class); + + // Two requests are needed to get all the messages. + Dataflow.V1b3.Projects.Jobs.Messages.List firstRequest = + mock(Dataflow.V1b3.Projects.Jobs.Messages.List.class); + Dataflow.V1b3.Projects.Jobs.Messages.List secondRequest = + mock(Dataflow.V1b3.Projects.Jobs.Messages.List.class); + + when(mockMessages.list(PROJECT_ID, JOB_ID)) + .thenReturn(firstRequest) + .thenReturn(secondRequest); + + ListJobMessagesResponse firstResponse = new ListJobMessagesResponse(); + firstResponse.setJobMessages(new ArrayList()); + for (int i = 0; i < 100; ++i) { + JobMessage message = new JobMessage(); + message.setId("message_" + i); + message.setTime(TimeUtil.toCloudTime(new Instant(i))); + firstResponse.getJobMessages().add(message); + } + String pageToken = "page_token"; + firstResponse.setNextPageToken(pageToken); + + ListJobMessagesResponse secondResponse = new ListJobMessagesResponse(); + secondResponse.setJobMessages(new ArrayList()); + for (int i = 100; i < 150; ++i) { + JobMessage message = new JobMessage(); + message.setId("message_" + i); + message.setTime(TimeUtil.toCloudTime(new Instant(i))); + secondResponse.getJobMessages().add(message); + } + + when(firstRequest.execute()).thenReturn(firstResponse); + when(secondRequest.execute()).thenReturn(secondResponse); + + MonitoringUtil util = new MonitoringUtil(PROJECT_ID, mockMessages); + + List messages = util.getJobMessages(JOB_ID, -1); + + verify(secondRequest).setPageToken(pageToken); + + assertEquals(150, messages.size()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PTupleTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PTupleTest.java new file mode 100644 index 0000000000000..3692411a4a751 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PTupleTest.java @@ -0,0 +1,40 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link PTuple}. */ +@RunWith(JUnit4.class) +public final class PTupleTest { + @Test + public void accessingNullVoidValuesShouldNotCauseExceptions() { + TupleTag tag = new TupleTag() {}; + PTuple tuple = PTuple.of(tag, null); + assertTrue(tuple.has(tag)); + assertThat(tuple.get(tag), is(nullValue())); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java new file mode 100644 index 0000000000000..7d923c2fcdb7f --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java @@ -0,0 +1,342 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.io.Files; +import com.google.common.io.LineReader; + +import org.hamcrest.CoreMatchers; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.io.File; +import java.io.IOException; +import java.nio.channels.Channels; +import java.nio.channels.Pipe; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; + +/** Tests for PackageUtil. */ +@RunWith(JUnit4.class) +public class PackageUtilTest { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + @Rule + public FastNanoClockAndSleeper fastNanoClockAndSleeper = new FastNanoClockAndSleeper(); + + @Mock + GcsUtil mockGcsUtil; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + } + + @Test + public void testPackageNamingWithFileHavingExtension() throws Exception { + File tmpFile = tmpFolder.newFile("file.txt"); + Files.write("This is a test!", tmpFile, StandardCharsets.UTF_8); + GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); + + DataflowPackage target = PackageUtil.createPackage(tmpFile.getAbsolutePath(), gcsStaging, null); + + assertEquals("file-SAzzqSB2zmoIgNHC9A2G0A.txt", target.getName()); + assertEquals("storage.googleapis.com/somebucket/base/path/file-SAzzqSB2zmoIgNHC9A2G0A.txt", + target.getLocation()); + } + + @Test + public void testPackageNamingWithFileMissingExtension() throws Exception { + File tmpFile = tmpFolder.newFile("file"); + Files.write("This is a test!", tmpFile, StandardCharsets.UTF_8); + GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); + + DataflowPackage target = PackageUtil.createPackage(tmpFile.getAbsolutePath(), gcsStaging, null); + + assertEquals("file-SAzzqSB2zmoIgNHC9A2G0A", target.getName()); + assertEquals("storage.googleapis.com/somebucket/base/path/file-SAzzqSB2zmoIgNHC9A2G0A", + target.getLocation()); + } + + @Test + public void testPackageNamingWithDirectory() throws Exception { + File tmpDirectory = tmpFolder.newFolder("folder"); + File tmpFile = tmpFolder.newFile("folder/file.txt"); + Files.write("This is a test!", tmpFile, StandardCharsets.UTF_8); + GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); + + DataflowPackage target = + PackageUtil.createPackage(tmpDirectory.getAbsolutePath(), gcsStaging, null); + + assertEquals("folder-9MHI5fxducQ06t3IG9MC-g.zip", target.getName()); + assertEquals("storage.googleapis.com/somebucket/base/path/folder-9MHI5fxducQ06t3IG9MC-g.zip", + target.getLocation()); + } + + @Test + public void testPackageNamingWithFilesHavingSameContentsButDifferentNames() throws Exception { + tmpFolder.newFolder("folder1"); + File tmpDirectory1 = tmpFolder.newFolder("folder1/folderA"); + File tmpFile1 = tmpFolder.newFile("folder1/folderA/uniqueName1"); + Files.write("This is a test!", tmpFile1, StandardCharsets.UTF_8); + + tmpFolder.newFolder("folder2"); + File tmpDirectory2 = tmpFolder.newFolder("folder2/folderA"); + File tmpFile2 = tmpFolder.newFile("folder2/folderA/uniqueName2"); + Files.write("This is a test!", tmpFile2, StandardCharsets.UTF_8); + + GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); + + DataflowPackage target1 = + PackageUtil.createPackage(tmpDirectory1.getAbsolutePath(), gcsStaging, null); + DataflowPackage target2 = + PackageUtil.createPackage(tmpDirectory2.getAbsolutePath(), gcsStaging, null); + + assertFalse(target1.getName().equals(target2.getName())); + assertFalse(target1.getLocation().equals(target2.getLocation())); + } + + @Test + public void testPackageNamingWithDirectoriesHavingSameContentsButDifferentNames() + throws Exception { + tmpFolder.newFolder("folder1"); + File tmpDirectory1 = tmpFolder.newFolder("folder1/folderA"); + tmpFolder.newFolder("folder1/folderA/uniqueName1"); + + tmpFolder.newFolder("folder2"); + File tmpDirectory2 = tmpFolder.newFolder("folder2/folderA"); + tmpFolder.newFolder("folder2/folderA/uniqueName2"); + + GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); + + DataflowPackage target1 = + PackageUtil.createPackage(tmpDirectory1.getAbsolutePath(), gcsStaging, null); + DataflowPackage target2 = + PackageUtil.createPackage(tmpDirectory2.getAbsolutePath(), gcsStaging, null); + + assertFalse(target1.getName().equals(target2.getName())); + assertFalse(target1.getLocation().equals(target2.getLocation())); + } + + @Test + public void testPackageUploadWithFileSucceeds() throws Exception { + Pipe pipe = Pipe.open(); + File tmpFile = tmpFolder.newFile("file.txt"); + Files.write("This is a test!", tmpFile, StandardCharsets.UTF_8); + GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); + when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(-1L); + when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink()); + + List targets = PackageUtil.stageClasspathElementsToGcs(mockGcsUtil, + ImmutableList.of(tmpFile.getAbsolutePath()), gcsStaging); + DataflowPackage target = Iterables.getOnlyElement(targets); + + verify(mockGcsUtil).fileSize(any(GcsPath.class)); + verify(mockGcsUtil).create(any(GcsPath.class), anyString()); + verifyNoMoreInteractions(mockGcsUtil); + + assertEquals("file-SAzzqSB2zmoIgNHC9A2G0A.txt", target.getName()); + assertEquals("storage.googleapis.com/somebucket/base/path/file-SAzzqSB2zmoIgNHC9A2G0A.txt", + target.getLocation()); + assertEquals("This is a test!", + new LineReader(Channels.newReader(pipe.source(), "UTF-8")).readLine()); + } + + @Test + public void testPackageUploadWithDirectorySucceeds() throws Exception { + Pipe pipe = Pipe.open(); + File tmpDirectory = tmpFolder.newFolder("folder"); + tmpFolder.newFolder("folder/empty_directory"); + tmpFolder.newFolder("folder/directory"); + File tmpFile1 = tmpFolder.newFile("folder/file.txt"); + File tmpFile2 = tmpFolder.newFile("folder/directory/file.txt"); + Files.write("This is a test!", tmpFile1, StandardCharsets.UTF_8); + Files.write("This is also a test!", tmpFile2, StandardCharsets.UTF_8); + + GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); + when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(-1L); + when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink()); + + PackageUtil.stageClasspathElementsToGcs(mockGcsUtil, + ImmutableList.of(tmpDirectory.getAbsolutePath()), gcsStaging); + + verify(mockGcsUtil).fileSize(any(GcsPath.class)); + verify(mockGcsUtil).create(any(GcsPath.class), anyString()); + verifyNoMoreInteractions(mockGcsUtil); + + ZipInputStream inputStream = new ZipInputStream(Channels.newInputStream(pipe.source())); + List zipEntryNames = new ArrayList<>(); + for (ZipEntry entry = inputStream.getNextEntry(); entry != null; + entry = inputStream.getNextEntry()) { + zipEntryNames.add(entry.getName()); + } + assertTrue(CoreMatchers.hasItems("directory/file.txt", "empty_directory/", "file.txt").matches( + zipEntryNames)); + } + + @Test + public void testPackageUploadWithEmptyDirectorySucceeds() throws Exception { + Pipe pipe = Pipe.open(); + File tmpDirectory = tmpFolder.newFolder("folder"); + + GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); + when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(-1L); + when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink()); + + List targets = PackageUtil.stageClasspathElementsToGcs(mockGcsUtil, + ImmutableList.of(tmpDirectory.getAbsolutePath()), gcsStaging); + DataflowPackage target = Iterables.getOnlyElement(targets); + + verify(mockGcsUtil).fileSize(any(GcsPath.class)); + verify(mockGcsUtil).create(any(GcsPath.class), anyString()); + verifyNoMoreInteractions(mockGcsUtil); + + assertEquals("folder-wstW9MW_ZW-soJhufroDCA.zip", target.getName()); + assertEquals("storage.googleapis.com/somebucket/base/path/folder-wstW9MW_ZW-soJhufroDCA.zip", + target.getLocation()); + assertNull(new ZipInputStream(Channels.newInputStream(pipe.source())).getNextEntry()); + } + + @Test(expected = RuntimeException.class) + public void testPackageUploadFailsWhenIOExceptionThrown() throws Exception { + File tmpFile = tmpFolder.newFile("file.txt"); + Files.write("This is a test!", tmpFile, StandardCharsets.UTF_8); + GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); + when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(-1L); + when(mockGcsUtil.create(any(GcsPath.class), anyString())) + .thenThrow(new IOException("Upload error")); + + try { + PackageUtil.stageClasspathElementsToGcs(mockGcsUtil, + ImmutableList.of(tmpFile.getAbsolutePath()), gcsStaging, fastNanoClockAndSleeper); + } finally { + verify(mockGcsUtil).fileSize(any(GcsPath.class)); + verify(mockGcsUtil, times(5)).create(any(GcsPath.class), anyString()); + verifyNoMoreInteractions(mockGcsUtil); + } + } + + @Test + public void testPackageUploadEventuallySucceeds() throws Exception { + Pipe pipe = Pipe.open(); + File tmpFile = tmpFolder.newFile("file.txt"); + Files.write("This is a test!", tmpFile, StandardCharsets.UTF_8); + GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); + when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(-1L); + when(mockGcsUtil.create(any(GcsPath.class), anyString())) + .thenThrow(new IOException("410 Gone")) // First attempt fails + .thenReturn(pipe.sink()); // second attempt succeeds + + try { + PackageUtil.stageClasspathElementsToGcs(mockGcsUtil, + ImmutableList.of(tmpFile.getAbsolutePath()), + gcsStaging, + fastNanoClockAndSleeper); + } finally { + verify(mockGcsUtil).fileSize(any(GcsPath.class)); + verify(mockGcsUtil, times(2)).create(any(GcsPath.class), anyString()); + verifyNoMoreInteractions(mockGcsUtil); + } + } + + @Test + public void testPackageUploadIsSkippedWhenFileAlreadyExists() throws Exception { + File tmpFile = tmpFolder.newFile("file.txt"); + Files.write("This is a test!", tmpFile, StandardCharsets.UTF_8); + GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); + when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(tmpFile.length()); + + PackageUtil.stageClasspathElementsToGcs(mockGcsUtil, + ImmutableList.of(tmpFile.getAbsolutePath()), gcsStaging); + + verify(mockGcsUtil).fileSize(any(GcsPath.class)); + verifyNoMoreInteractions(mockGcsUtil); + } + + @Test + public void testPackageUploadIsNotSkippedWhenSizesAreDifferent() throws Exception { + Pipe pipe = Pipe.open(); + File tmpDirectory = tmpFolder.newFolder("folder"); + tmpFolder.newFolder("folder/empty_directory"); + tmpFolder.newFolder("folder/directory"); + File tmpFile1 = tmpFolder.newFile("folder/file.txt"); + File tmpFile2 = tmpFolder.newFile("folder/directory/file.txt"); + Files.write("This is a test!", tmpFile1, StandardCharsets.UTF_8); + Files.write("This is also a test!", tmpFile2, StandardCharsets.UTF_8); + GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); + when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(Long.MAX_VALUE); + when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink()); + + PackageUtil.stageClasspathElementsToGcs(mockGcsUtil, + ImmutableList.of(tmpDirectory.getAbsolutePath()), gcsStaging); + + verify(mockGcsUtil).fileSize(any(GcsPath.class)); + verify(mockGcsUtil).create(any(GcsPath.class), anyString()); + verifyNoMoreInteractions(mockGcsUtil); + } + + @Test + public void testPackageUploadWithExplicitPackageName() throws Exception { + Pipe pipe = Pipe.open(); + File tmpFile = tmpFolder.newFile("file.txt"); + Files.write("This is a test!", tmpFile, StandardCharsets.UTF_8); + GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); + final String overriddenName = "alias.txt"; + + when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(-1L); + when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink()); + + List targets = PackageUtil.stageClasspathElementsToGcs(mockGcsUtil, + ImmutableList.of(overriddenName + "=" + tmpFile.getAbsolutePath()), gcsStaging); + DataflowPackage target = Iterables.getOnlyElement(targets); + + verify(mockGcsUtil).fileSize(any(GcsPath.class)); + verify(mockGcsUtil).create(any(GcsPath.class), anyString()); + verifyNoMoreInteractions(mockGcsUtil); + + assertEquals(overriddenName, target.getName()); + assertEquals("storage.googleapis.com/somebucket/base/path/file-SAzzqSB2zmoIgNHC9A2G0A.txt", + target.getLocation()); + } + +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializerTest.java new file mode 100644 index 0000000000000..45924560630ba --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializerTest.java @@ -0,0 +1,234 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.junit.Assert.assertNotNull; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +import com.google.api.client.auth.oauth2.Credential; +import com.google.api.client.http.HttpRequest; +import com.google.api.client.http.HttpResponse; +import com.google.api.client.http.HttpResponseException; +import com.google.api.client.http.HttpTransport; +import com.google.api.client.http.LowLevelHttpRequest; +import com.google.api.client.http.LowLevelHttpResponse; +import com.google.api.client.json.JsonFactory; +import com.google.api.client.json.jackson2.JacksonFactory; +import com.google.api.client.util.NanoClock; +import com.google.api.client.util.Sleeper; +import com.google.api.services.storage.Storage; + +import org.hamcrest.Matchers; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.io.IOException; +import java.security.PrivateKey; + +/** + * Tests for RetryHttpRequestInitializer. + */ +@RunWith(JUnit4.class) +public class RetryHttpRequestInitializerTest { + + @Mock private Credential mockCredential; + @Mock private PrivateKey mockPrivateKey; + @Mock private LowLevelHttpRequest mockLowLevelRequest; + @Mock private LowLevelHttpResponse mockLowLevelResponse; + + private final JsonFactory jsonFactory = JacksonFactory.getDefaultInstance(); + private Storage storage; + + // Used to test retrying a request more than the default 10 times. + static class MockNanoClock implements NanoClock { + private int timesMs[] = {500, 750, 1125, 1688, 2531, 3797, 5695, 8543, + 12814, 19222, 28833, 43249, 64873, 97310, 145965, 218945, 328420}; + private int i = 0; + + @Override + public long nanoTime() { + return timesMs[i++ / 2] * 1000000; + } + } + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + + HttpTransport lowLevelTransport = new HttpTransport() { + @Override + protected LowLevelHttpRequest buildRequest(String method, String url) + throws IOException { + return mockLowLevelRequest; + } + }; + + // Retry initializer will pass through to credential, since we can have + // only a single HttpRequestInitializer, and we use multiple Credential + // types in the SDK, not all of which allow for retry configuration. + RetryHttpRequestInitializer initializer = new RetryHttpRequestInitializer( + mockCredential, new MockNanoClock(), new Sleeper() { + @Override + public void sleep(long millis) throws InterruptedException {} + }); + storage = new Storage.Builder(lowLevelTransport, jsonFactory, initializer) + .setApplicationName("test").build(); + } + + @After + public void tearDown() { + verifyNoMoreInteractions(mockPrivateKey); + verifyNoMoreInteractions(mockLowLevelRequest); + verifyNoMoreInteractions(mockCredential); + } + + @Test + public void testBasicOperation() throws IOException { + when(mockLowLevelRequest.execute()) + .thenReturn(mockLowLevelResponse); + when(mockLowLevelResponse.getStatusCode()) + .thenReturn(200); + + Storage.Buckets.Get result = storage.buckets().get("test"); + HttpResponse response = result.executeUnparsed(); + assertNotNull(response); + + verify(mockCredential).initialize(any(HttpRequest.class)); + verify(mockLowLevelRequest, atLeastOnce()) + .addHeader(anyString(), anyString()); + verify(mockLowLevelRequest).setTimeout(anyInt(), anyInt()); + verify(mockLowLevelRequest).execute(); + verify(mockLowLevelResponse).getStatusCode(); + } + + /** + * Tests that a non-retriable error is not retried. + */ + @Test + public void testErrorCodeForbidden() throws IOException { + when(mockLowLevelRequest.execute()) + .thenReturn(mockLowLevelResponse); + when(mockLowLevelResponse.getStatusCode()) + .thenReturn(403) // Non-retryable error. + .thenReturn(200); // Shouldn't happen. + + try { + Storage.Buckets.Get result = storage.buckets().get("test"); + HttpResponse response = result.executeUnparsed(); + assertNotNull(response); + } catch (HttpResponseException e) { + Assert.assertThat(e.getMessage(), Matchers.containsString("403")); + } + + verify(mockCredential).initialize(any(HttpRequest.class)); + verify(mockLowLevelRequest, atLeastOnce()) + .addHeader(anyString(), anyString()); + verify(mockLowLevelRequest).setTimeout(anyInt(), anyInt()); + verify(mockLowLevelRequest).execute(); + verify(mockLowLevelResponse).getStatusCode(); + } + + /** + * Tests that a retriable error is retried. + */ + @Test + public void testRetryableError() throws IOException { + when(mockLowLevelRequest.execute()) + .thenReturn(mockLowLevelResponse) + .thenReturn(mockLowLevelResponse) + .thenReturn(mockLowLevelResponse); + when(mockLowLevelResponse.getStatusCode()) + .thenReturn(503) // Retryable + .thenReturn(429) // We also retry on 429 Too Many Requests. + .thenReturn(200); + + Storage.Buckets.Get result = storage.buckets().get("test"); + HttpResponse response = result.executeUnparsed(); + assertNotNull(response); + + verify(mockCredential).initialize(any(HttpRequest.class)); + verify(mockLowLevelRequest, atLeastOnce()) + .addHeader(anyString(), anyString()); + verify(mockLowLevelRequest, times(3)).setTimeout(anyInt(), anyInt()); + verify(mockLowLevelRequest, times(3)).execute(); + verify(mockLowLevelResponse, times(3)).getStatusCode(); + } + + /** + * Tests that an IOException is retried. + */ + @Test + public void testThrowIOException() throws IOException { + when(mockLowLevelRequest.execute()) + .thenThrow(new IOException("Fake Error")) + .thenReturn(mockLowLevelResponse); + when(mockLowLevelResponse.getStatusCode()) + .thenReturn(200); + + Storage.Buckets.Get result = storage.buckets().get("test"); + HttpResponse response = result.executeUnparsed(); + assertNotNull(response); + + verify(mockCredential).initialize(any(HttpRequest.class)); + verify(mockLowLevelRequest, atLeastOnce()) + .addHeader(anyString(), anyString()); + verify(mockLowLevelRequest, times(2)).setTimeout(anyInt(), anyInt()); + verify(mockLowLevelRequest, times(2)).execute(); + verify(mockLowLevelResponse).getStatusCode(); + } + + /** + * Tests that a retryable error is retried enough times. + */ + @Test + public void testRetryableErrorRetryEnoughTimes() throws IOException { + when(mockLowLevelRequest.execute()).thenReturn(mockLowLevelResponse); + final int retries = 10; + when(mockLowLevelResponse.getStatusCode()).thenAnswer(new Answer(){ + int n = 0; + @Override + public Integer answer(InvocationOnMock invocation) { + return (n++ < retries - 1) ? 503 : 200; + }}); + + Storage.Buckets.Get result = storage.buckets().get("test"); + HttpResponse response = result.executeUnparsed(); + assertNotNull(response); + + verify(mockCredential).initialize(any(HttpRequest.class)); + verify(mockLowLevelRequest, atLeastOnce()).addHeader(anyString(), + anyString()); + verify(mockLowLevelRequest, times(retries)).setTimeout(anyInt(), anyInt()); + verify(mockLowLevelRequest, times(retries)).execute(); + verify(mockLowLevelResponse, times(retries)).getStatusCode(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializableUtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializableUtilsTest.java new file mode 100644 index 0000000000000..90f10cdc97134 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializableUtilsTest.java @@ -0,0 +1,75 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import org.hamcrest.CoreMatchers; +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; + +/** + * Tests for SerializableUtils. + */ +@RunWith(JUnit4.class) +public class SerializableUtilsTest { + static class TestClass implements Serializable { + final String stringValue; + final int intValue; + + public TestClass(String stringValue, int intValue) { + this.stringValue = stringValue; + this.intValue = intValue; + } + } + + @Test + public void testTranscode() { + String stringValue = "hi bob"; + int intValue = 42; + + TestClass testObject = new TestClass(stringValue, intValue); + + Object copy = + SerializableUtils.deserializeFromByteArray( + SerializableUtils.serializeToByteArray(testObject), + "a TestObject"); + + Assert.assertThat(copy, new IsInstanceOf(TestClass.class)); + TestClass testCopy = (TestClass) copy; + + Assert.assertEquals(stringValue, testCopy.stringValue); + Assert.assertEquals(intValue, testCopy.intValue); + } + + @Test + public void testDeserializationError() { + try { + SerializableUtils.deserializeFromByteArray( + "this isn't legal".getBytes(), + "a bogus string"); + Assert.fail("should have thrown an exception"); + } catch (Exception exn) { + Assert.assertThat(exn.toString(), + CoreMatchers.containsString( + "unable to deserialize a bogus string")); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializerTest.java new file mode 100644 index 0000000000000..40e2cc00f650d --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializerTest.java @@ -0,0 +1,163 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean; +import static com.google.cloud.dataflow.sdk.util.Structs.addDouble; +import static com.google.cloud.dataflow.sdk.util.Structs.addLong; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests Serializer implementation. + */ +@RunWith(JUnit4.class) +@Ignore +public class SerializerTest { + /** + * A POJO to use for testing serialization. + */ + @JsonTypeInfo(use = JsonTypeInfo.Id.CLASS, include = JsonTypeInfo.As.PROPERTY, + property = PropertyNames.OBJECT_TYPE_NAME) + public static class TestRecord { + // TODO: When we apply property name typing to all non-final classes, the + // annotation on this class should be removed. + public String name; + public boolean ok; + public int value; + public double dValue; + } + + @Test + public void testStatefulDeserialization() { + CloudObject object = CloudObject.forClass(TestRecord.class); + + addString(object, "name", "foobar"); + addBoolean(object, "ok", true); + addLong(object, "value", 42L); + addDouble(object, "dValue", .25); + + TestRecord record = Serializer.deserialize(object, TestRecord.class); + Assert.assertEquals("foobar", record.name); + Assert.assertEquals(true, record.ok); + Assert.assertEquals(42L, record.value); + Assert.assertEquals(0.25, record.dValue, 0.0001); + } + + private static class InjectedTestRecord { + private final String n; + private final int v; + + public InjectedTestRecord( + @JsonProperty("name") String name, + @JsonProperty("value") int value) { + this.n = name; + this.v = value; + } + + public String getName() { + return n; + } + public int getValue() { + return v; + } + } + + @Test + public void testDeserializationInjection() { + CloudObject object = CloudObject.forClass(InjectedTestRecord.class); + addString(object, "name", "foobar"); + addLong(object, "value", 42L); + + InjectedTestRecord record = + Serializer.deserialize(object, InjectedTestRecord.class); + + Assert.assertEquals("foobar", record.getName()); + Assert.assertEquals(42L, record.getValue()); + } + + private static class FactoryInjectedTestRecord { + @JsonCreator + public static FactoryInjectedTestRecord of( + @JsonProperty("name") String name, + @JsonProperty("value") int value) { + return new FactoryInjectedTestRecord(name, value); + } + + private final String n; + private final int v; + + private FactoryInjectedTestRecord(String name, int value) { + this.n = name; + this.v = value; + } + + public String getName() { + return n; + } + public int getValue() { + return v; + } + } + + @Test + public void testDeserializationFactoryInjection() { + CloudObject object = CloudObject.forClass(FactoryInjectedTestRecord.class); + addString(object, "name", "foobar"); + addLong(object, "value", 42L); + + FactoryInjectedTestRecord record = + Serializer.deserialize(object, FactoryInjectedTestRecord.class); + Assert.assertEquals("foobar", record.getName()); + Assert.assertEquals(42L, record.getValue()); + } + + private static class DerivedTestRecord extends TestRecord { + public String derived; + } + + @Test + public void testSubclassDeserialization() { + CloudObject object = CloudObject.forClass(DerivedTestRecord.class); + + addString(object, "name", "foobar"); + addBoolean(object, "ok", true); + addLong(object, "value", 42L); + addDouble(object, "dValue", .25); + addString(object, "derived", "baz"); + + TestRecord result = Serializer.deserialize(object, TestRecord.class); + Assert.assertThat(result, Matchers.instanceOf(DerivedTestRecord.class)); + + DerivedTestRecord record = (DerivedTestRecord) result; + Assert.assertEquals("foobar", record.name); + Assert.assertEquals(true, record.ok); + Assert.assertEquals(42L, record.value); + Assert.assertEquals(0.25, record.dValue, 0.0001); + Assert.assertEquals("baz", record.derived); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java new file mode 100644 index 0000000000000..94c44c707d0f3 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java @@ -0,0 +1,282 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.WindowUtils.windowToString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; +import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import org.hamcrest.Matchers; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** Unit tests for {@link StreamingGroupAlsoByWindowsDoFn}. */ +@RunWith(JUnit4.class) +public class StreamingGroupAlsoByWindowsDoFnTest { + ExecutionContext execContext; + CounterSet counters; + TupleTag>> outputTag; + + @Before public void setUp() { + execContext = new DirectModeExecutionContext(); + counters = new CounterSet(); + outputTag = new TupleTag<>(); + } + + @Test public void testEmpty() throws Exception { + DoFnRunner>, + KV>, List> runner = + makeRunner(FixedWindows.of(Duration.millis(10))); + + runner.startBundle(); + + runner.finishBundle(); + + List>> result = runner.getReceiver(outputTag); + + assertEquals(0, result.size()); + } + + @Test public void testFixedWindows() throws Exception { + DoFnRunner>, + KV>, List> runner = + makeRunner(FixedWindows.of(Duration.millis(10))); + + Coder windowCoder = FixedWindows.of(Duration.millis(10)).windowCoder(); + + runner.startBundle(); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", "v1")), + new Instant(1), + Arrays.asList(window(0, 10)))); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", "v2")), + new Instant(2), + Arrays.asList(window(0, 10)))); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", "v0")), + new Instant(0), + Arrays.asList(window(0, 10)))); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", "v3")), + new Instant(13), + Arrays.asList(window(10, 20)))); + + runner.processElement(WindowedValue.valueInEmptyWindows( + TimerOrElement.>timer( + windowToString((IntervalWindow) window(0, 10), windowCoder), + new Instant(9), "k"))); + + runner.processElement(WindowedValue.valueInEmptyWindows( + TimerOrElement.>timer( + windowToString((IntervalWindow) window(10, 20), windowCoder), + new Instant(19), "k"))); + + runner.finishBundle(); + + List>>> result = runner.getReceiver(outputTag); + + assertEquals(2, result.size()); + + WindowedValue>> item0 = result.get(0); + assertEquals("k", item0.getValue().getKey()); + assertThat(item0.getValue().getValue(), Matchers.containsInAnyOrder("v0", "v1", "v2")); + assertEquals(new Instant(9), item0.getTimestamp()); + assertThat(item0.getWindows(), Matchers.contains(window(0, 10))); + + WindowedValue>> item1 = result.get(1); + assertEquals("k", item1.getValue().getKey()); + assertThat(item1.getValue().getValue(), Matchers.containsInAnyOrder("v3")); + assertEquals(new Instant(19), item1.getTimestamp()); + assertThat(item1.getWindows(), Matchers.contains(window(10, 20))); + } + + @Test public void testSlidingWindows() throws Exception { + DoFnRunner>, + KV>, List> runner = + makeRunner(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10))); + + Coder windowCoder = + SlidingWindows.of(Duration.millis(10)).every(Duration.millis(10)).windowCoder(); + + runner.startBundle(); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", "v1")), + new Instant(5), + Arrays.asList(window(-10, 10), window(0, 20)))); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", "v0")), + new Instant(2), + Arrays.asList(window(-10, 10), window(0, 20)))); + + runner.processElement(WindowedValue.valueInEmptyWindows( + TimerOrElement.>timer( + windowToString((IntervalWindow) window(-10, 10), windowCoder), + new Instant(9), "k"))); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", "v2")), + new Instant(5), + Arrays.asList(window(0, 20), window(10, 30)))); + + runner.processElement(WindowedValue.valueInEmptyWindows( + TimerOrElement.>timer( + windowToString((IntervalWindow) window(0, 20), windowCoder), + new Instant(19), "k"))); + + runner.processElement(WindowedValue.valueInEmptyWindows( + TimerOrElement.>timer( + windowToString((IntervalWindow) window(10, 30), windowCoder), + new Instant(29), "k"))); + + runner.finishBundle(); + + List>>> result = runner.getReceiver(outputTag); + + assertEquals(3, result.size()); + + WindowedValue>> item0 = result.get(0); + assertEquals("k", item0.getValue().getKey()); + assertThat(item0.getValue().getValue(), Matchers.containsInAnyOrder("v0", "v1")); + assertEquals(new Instant(9), item0.getTimestamp()); + assertThat(item0.getWindows(), Matchers.contains(window(-10, 10))); + + WindowedValue>> item1 = result.get(1); + assertEquals("k", item1.getValue().getKey()); + assertThat(item1.getValue().getValue(), Matchers.containsInAnyOrder("v0", "v1", "v2")); + assertEquals(new Instant(19), item1.getTimestamp()); + assertThat(item1.getWindows(), Matchers.contains(window(0, 20))); + + WindowedValue>> item2 = result.get(2); + assertEquals("k", item2.getValue().getKey()); + assertThat(item2.getValue().getValue(), Matchers.containsInAnyOrder("v2")); + assertEquals(new Instant(29), item2.getTimestamp()); + assertThat(item2.getWindows(), Matchers.contains(window(10, 30))); + } + + @Test public void testSessions() throws Exception { + DoFnRunner>, + KV>, List> runner = + makeRunner(Sessions.withGapDuration(Duration.millis(10))); + + Coder windowCoder = + Sessions.withGapDuration(Duration.millis(10)).windowCoder(); + + runner.startBundle(); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", "v1")), + new Instant(0), + Arrays.asList(window(0, 10)))); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", "v2")), + new Instant(5), + Arrays.asList(window(5, 15)))); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", "v3")), + new Instant(15), + Arrays.asList(window(15, 25)))); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", "v0")), + new Instant(3), + Arrays.asList(window(3, 13)))); + + runner.processElement(WindowedValue.valueInEmptyWindows( + TimerOrElement.>timer( + windowToString((IntervalWindow) window(0, 15), windowCoder), + new Instant(14), "k"))); + + runner.processElement(WindowedValue.valueInEmptyWindows( + TimerOrElement.>timer( + windowToString((IntervalWindow) window(15, 25), windowCoder), + new Instant(24), "k"))); + + runner.finishBundle(); + + List>>> result = runner.getReceiver(outputTag); + + assertEquals(2, result.size()); + + WindowedValue>> item0 = result.get(0); + assertEquals("k", item0.getValue().getKey()); + assertThat(item0.getValue().getValue(), Matchers.containsInAnyOrder("v0", "v1", "v2")); + assertEquals(new Instant(14), item0.getTimestamp()); + assertThat(item0.getWindows(), Matchers.contains(window(0, 15))); + + WindowedValue>> item1 = result.get(1); + assertEquals("k", item1.getValue().getKey()); + assertThat(item1.getValue().getValue(), Matchers.containsInAnyOrder("v3")); + assertEquals(new Instant(24), item1.getTimestamp()); + assertThat(item1.getWindows(), Matchers.contains(window(15, 25))); + } + + + private DoFnRunner>, + KV>, List> makeRunner( + WindowingFn windowingStrategy) { + + StreamingGroupAlsoByWindowsDoFn, IntervalWindow> fn = + StreamingGroupAlsoByWindowsDoFn.create(windowingStrategy, StringUtf8Coder.of()); + + DoFnRunner>, + KV>, List> runner = + DoFnRunner.createWithListOutputs( + PipelineOptionsFactory.create(), + fn, + PTuple.empty(), + outputTag, + new ArrayList>(), + execContext.createStepContext("merge"), + counters.getAddCounterMutator()); + + return runner; + } + + private BoundedWindow window(long start, long end) { + return new IntervalWindow(new Instant(start), new Instant(end)); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StringUtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StringUtilsTest.java new file mode 100644 index 0000000000000..bf1a3193b7e3c --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StringUtilsTest.java @@ -0,0 +1,88 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.hamcrest.core.Is.is; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Tests for StringUtils. + */ +@RunWith(JUnit4.class) +public class StringUtilsTest { + @Test + public void testTranscodeEmptyByteArray() { + byte[] bytes = { }; + String string = ""; + assertEquals(string, StringUtils.byteArrayToJsonString(bytes)); + assertArrayEquals(bytes, StringUtils.jsonStringToByteArray(string)); + } + + @Test + public void testTranscodeMixedByteArray() { + byte[] bytes = { + 0, 5, 12, 16, 31, 32, 65, 66, 126, 127, (byte) 128, (byte) 255, 67, 0 }; + String string = "%00%05%0c%10%1f AB~%7f%80%ffC%00"; + assertEquals(string, StringUtils.byteArrayToJsonString(bytes)); + assertArrayEquals(bytes, StringUtils.jsonStringToByteArray(string)); + } + + /** + * Inner class for simple name test. + */ + private class EmbeddedDoFn { + // Returns an anonymous inner class. + private EmbeddedDoFn getEmbedded() { + return new EmbeddedDoFn(){}; + } + } + + @Test + public void testSimpleName() { + assertEquals("Embedded", + StringUtils.approximateSimpleName(EmbeddedDoFn.class)); + } + + @Test + public void testAnonSimpleName() { + EmbeddedDoFn anon = new EmbeddedDoFn(){}; + + Pattern p = Pattern.compile("StringUtilsTest\\$[0-9]+"); + Matcher m = p.matcher(StringUtils.approximateSimpleName(anon.getClass())); + assertThat(m.matches(), is(true)); + } + + @Test + public void testNestedSimpleName() { + EmbeddedDoFn fn = new EmbeddedDoFn(); + EmbeddedDoFn anon = fn.getEmbedded(); + + // Expect to find "Embedded$1" + Pattern p = Pattern.compile("Embedded\\$[0-9]+"); + Matcher m = p.matcher(StringUtils.approximateSimpleName(anon.getClass())); + assertThat(m.matches(), is(true)); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StructsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StructsTest.java new file mode 100644 index 0000000000000..9b8cc208fca9b --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StructsTest.java @@ -0,0 +1,177 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean; +import static com.google.cloud.dataflow.sdk.util.Structs.addDouble; +import static com.google.cloud.dataflow.sdk.util.Structs.addList; +import static com.google.cloud.dataflow.sdk.util.Structs.addLong; +import static com.google.cloud.dataflow.sdk.util.Structs.addLongs; +import static com.google.cloud.dataflow.sdk.util.Structs.addNull; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; +import static com.google.cloud.dataflow.sdk.util.Structs.addStringList; +import static com.google.cloud.dataflow.sdk.util.Structs.getBoolean; +import static com.google.cloud.dataflow.sdk.util.Structs.getDictionary; +import static com.google.cloud.dataflow.sdk.util.Structs.getLong; +import static com.google.cloud.dataflow.sdk.util.Structs.getObject; +import static com.google.cloud.dataflow.sdk.util.Structs.getString; +import static com.google.cloud.dataflow.sdk.util.Structs.getStrings; + +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Tests for Structs. + */ +@RunWith(JUnit4.class) +public class StructsTest { + private List> makeCloudObjects() { + List> objects = new ArrayList<>(); + { + CloudObject o = CloudObject.forClassName("string"); + addString(o, "singletonStringKey", "stringValue"); + objects.add(o); + } + { + CloudObject o = CloudObject.forClassName("long"); + addLong(o, "singletonLongKey", 42L); + objects.add(o); + } + return objects; + } + + private Map makeCloudDictionary() { + Map o = new HashMap<>(); + addList(o, "emptyKey", Collections.>emptyList()); + addNull(o, "noStringsKey"); + addString(o, "singletonStringKey", "stringValue"); + addStringList(o, "multipleStringsKey", Arrays.asList("hi", "there", "bob")); + addLongs(o, "multipleLongsKey", 47L, 1L << 42, -5L); + addLong(o, "singletonLongKey", 42L); + addDouble(o, "singletonDoubleKey", 3.14); + addBoolean(o, "singletonBooleanKey", true); + addNull(o, "noObjectsKey"); + addList(o, "multipleObjectsKey", makeCloudObjects()); + return o; + } + + @Test + public void testGetStringParameter() throws Exception { + Map o = makeCloudDictionary(); + + Assert.assertEquals( + "stringValue", + getString(o, "singletonStringKey")); + Assert.assertEquals( + "stringValue", + getString(o, "singletonStringKey", "defaultValue")); + Assert.assertEquals( + "defaultValue", + getString(o, "missingKey", "defaultValue")); + + try { + getString(o, "missingKey"); + Assert.fail("should have thrown an exception"); + } catch (Exception exn) { + Assert.assertThat(exn.toString(), + Matchers.containsString( + "didn't find required parameter missingKey")); + } + + try { + getString(o, "noStringsKey"); + Assert.fail("should have thrown an exception"); + } catch (Exception exn) { + Assert.assertThat(exn.toString(), + Matchers.containsString("not a string")); + } + + Assert.assertThat(getStrings(o, "noStringsKey", null), Matchers.emptyIterable()); + Assert.assertThat(getObject(o, "noStringsKey").keySet(), Matchers.emptyIterable()); + Assert.assertThat(getDictionary(o, "noStringsKey").keySet(), Matchers.emptyIterable()); + Assert.assertThat(getDictionary(o, "noStringsKey", null).keySet(), + Matchers.emptyIterable()); + + try { + getString(o, "multipleStringsKey"); + Assert.fail("should have thrown an exception"); + } catch (Exception exn) { + Assert.assertThat(exn.toString(), + Matchers.containsString("not a string")); + } + + try { + getString(o, "emptyKey"); + Assert.fail("should have thrown an exception"); + } catch (Exception exn) { + Assert.assertThat(exn.toString(), + Matchers.containsString("not a string")); + } + } + + @Test + public void testGetBooleanParameter() throws Exception { + Map o = makeCloudDictionary(); + + Assert.assertEquals( + true, + getBoolean(o, "singletonBooleanKey", false)); + Assert.assertEquals( + false, + getBoolean(o, "missingKey", false)); + + try { + getBoolean(o, "emptyKey", false); + Assert.fail("should have thrown an exception"); + } catch (Exception exn) { + Assert.assertThat(exn.toString(), + Matchers.containsString("not a boolean")); + } + } + + @Test + public void testGetLongParameter() throws Exception { + Map o = makeCloudDictionary(); + + Assert.assertEquals( + (Long) 42L, + getLong(o, "singletonLongKey", 666L)); + Assert.assertEquals( + (Long) 666L, + getLong(o, "missingKey", 666L)); + + try { + getLong(o, "emptyKey", 666L); + Assert.fail("should have thrown an exception"); + } catch (Exception exn) { + Assert.assertThat(exn.toString(), + Matchers.containsString("not an int")); + } + } + + // TODO: Test builder operations. +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TimeUtilTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TimeUtilTest.java new file mode 100644 index 0000000000000..1faebeba7c0e4 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TimeUtilTest.java @@ -0,0 +1,73 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudDuration; +import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime; +import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; +import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudTime; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link TimeUtil}. */ +@RunWith(JUnit4.class) +public final class TimeUtilTest { + @Test + public void toCloudTimeShouldPrintTimeStrings() { + assertEquals("1970-01-01T00:00:00Z", toCloudTime(new Instant(0))); + assertEquals("1970-01-01T00:00:00.001Z", toCloudTime(new Instant(1))); + } + + @Test + public void fromCloudTimeShouldParseTimeStrings() { + assertEquals(new Instant(0), fromCloudTime("1970-01-01T00:00:00Z")); + assertEquals(new Instant(1), fromCloudTime("1970-01-01T00:00:00.001Z")); + assertEquals(new Instant(1), fromCloudTime("1970-01-01T00:00:00.001000Z")); + assertEquals(new Instant(1), fromCloudTime("1970-01-01T00:00:00.001001Z")); + assertEquals(new Instant(1), fromCloudTime("1970-01-01T00:00:00.001000000Z")); + assertEquals(new Instant(1), fromCloudTime("1970-01-01T00:00:00.001000001Z")); + assertNull(fromCloudTime("")); + assertNull(fromCloudTime("1970-01-01T00:00:00")); + } + + @Test + public void toCloudDurationShouldPrintDurationStrings() { + assertEquals("0s", toCloudDuration(Duration.ZERO)); + assertEquals("4s", toCloudDuration(Duration.millis(4000))); + assertEquals("4.001s", toCloudDuration(Duration.millis(4001))); + } + + @Test + public void fromCloudDurationShouldParseDurationStrings() { + assertEquals(Duration.millis(4000), fromCloudDuration("4s")); + assertEquals(Duration.millis(4001), fromCloudDuration("4.001s")); + assertEquals(Duration.millis(4001), fromCloudDuration("4.001000s")); + assertEquals(Duration.millis(4001), fromCloudDuration("4.001001s")); + assertEquals(Duration.millis(4001), fromCloudDuration("4.001000000s")); + assertEquals(Duration.millis(4001), fromCloudDuration("4.001000001s")); + assertNull(fromCloudDuration("")); + assertNull(fromCloudDuration("4")); + assertNull(fromCloudDuration("4.1")); + assertNull(fromCloudDuration("4.1s")); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/VarIntTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/VarIntTest.java new file mode 100644 index 0000000000000..d6b771bd0512b --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/VarIntTest.java @@ -0,0 +1,281 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.EOFException; +import java.io.IOException; + +/** Unit tests for {@link VarInt}. */ +@RunWith(JUnit4.class) +public class VarIntTest { + @Rule public final ExpectedException thrown = ExpectedException.none(); + + // Long values to check for boundary cases. + private static final long[] LONG_VALUES = { + 0, + 1, + 127, + 128, + 16383, + 16384, + 2097151, + 2097152, + 268435455, + 268435456, + 34359738367L, + 34359738368L, + 9223372036854775807L, + -9223372036854775808L, + -1, + }; + + // VarInt encoding of the above VALUES. + private static final byte[][] LONG_ENCODED = { + // 0 + { 0x00 }, + // 1 + { 0x01 }, + // 127 + { 0x7f }, + // 128 + { (byte) 0x80, 0x01 }, + // 16383 + { (byte) 0xff, 0x7f }, + // 16834 + { (byte) 0x80, (byte) 0x80, 0x01 }, + // 2097151 + { (byte) 0xff, (byte) 0xff, 0x7f }, + // 2097152 + { (byte) 0x80, (byte) 0x80, (byte) 0x80, 0x01 }, + // 268435455 + { (byte) 0xff, (byte) 0xff, (byte) 0xff, 0x7f }, + // 268435456 + { (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, 0x01 }, + // 34359738367 + { (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, 0x7f }, + // 34359738368 + { (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, + 0x01 }, + // 9223372036854775807 + { (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, + (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0x7f }, + // -9223372036854775808L + { (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, + (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, 0x01 }, + // -1 + { (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, + (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, 0x01 } + }; + + // Integer values to check for boundary cases. + private static final int[] INT_VALUES = { + 0, + 1, + 127, + 128, + 16383, + 16384, + 2097151, + 2097152, + 268435455, + 268435456, + 2147483647, + -2147483648, + -1, + }; + + // VarInt encoding of the above VALUES. + private static final byte[][] INT_ENCODED = { + // 0 + { (byte) 0x00 }, + // 1 + { (byte) 0x01 }, + // 127 + { (byte) 0x7f }, + // 128 + { (byte) 0x80, (byte) 0x01 }, + // 16383 + { (byte) 0xff, (byte) 0x7f }, + // 16834 + { (byte) 0x80, (byte) 0x80, (byte) 0x01 }, + // 2097151 + { (byte) 0xff, (byte) 0xff, (byte) 0x7f }, + // 2097152 + { (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x01 }, + // 268435455 + { (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0x7f }, + // 268435456 + { (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x01 }, + // 2147483647 + { (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0x07 }, + // -2147483648 + { (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x80, (byte) 0x08 }, + // -1 + { (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0x0f } + }; + + private static byte[] encodeInt(int v) throws IOException { + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + VarInt.encode(v, stream); + return stream.toByteArray(); + } + + private static byte[] encodeLong(long v) throws IOException { + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + VarInt.encode(v, stream); + return stream.toByteArray(); + } + + private static int decodeInt(byte[] encoded) throws IOException { + ByteArrayInputStream stream = new ByteArrayInputStream(encoded); + return VarInt.decodeInt(stream); + } + + private static long decodeLong(byte[] encoded) throws IOException { + ByteArrayInputStream stream = new ByteArrayInputStream(encoded); + return VarInt.decodeLong(stream); + } + + @Test + public void decodeValues() throws IOException { + assertEquals(LONG_VALUES.length, LONG_ENCODED.length); + for (int i = 0; i < LONG_ENCODED.length; ++i) { + ByteArrayInputStream stream = new ByteArrayInputStream(LONG_ENCODED[i]); + long parsed = VarInt.decodeLong(stream); + assertEquals(LONG_VALUES[i], parsed); + assertEquals(-1, stream.read()); + } + + assertEquals(INT_VALUES.length, INT_ENCODED.length); + for (int i = 0; i < INT_ENCODED.length; ++i) { + ByteArrayInputStream stream = new ByteArrayInputStream(INT_ENCODED[i]); + int parsed = VarInt.decodeInt(stream); + assertEquals(INT_VALUES[i], parsed); + assertEquals(-1, stream.read()); + } + } + + @Test + public void encodeValuesAndGetLength() throws IOException { + assertEquals(LONG_VALUES.length, LONG_ENCODED.length); + for (int i = 0; i < LONG_VALUES.length; ++i) { + byte[] encoded = encodeLong(LONG_VALUES[i]); + assertThat(encoded, equalTo(LONG_ENCODED[i])); + assertEquals(LONG_ENCODED[i].length, VarInt.getLength(LONG_VALUES[i])); + } + + assertEquals(INT_VALUES.length, INT_ENCODED.length); + for (int i = 0; i < INT_VALUES.length; ++i) { + byte[] encoded = encodeInt(INT_VALUES[i]); + assertThat(encoded, equalTo(INT_ENCODED[i])); + assertEquals(INT_ENCODED[i].length, VarInt.getLength(INT_VALUES[i])); + } + } + + @Test + public void decodeThrowsExceptionForOverflow() throws IOException { + final byte[] tooLargeNumber = + { (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, + (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, 0x02 }; + + thrown.expect(IOException.class); + + long parsed = decodeLong(tooLargeNumber); + } + + @Test + public void decodeThrowsExceptionForIntOverflow() throws IOException { + byte[] encoded = encodeLong(1L << 32); + + thrown.expect(IOException.class); + + int parsed = decodeInt(encoded); + } + + @Test + public void decodeThrowsExceptionForIntUnderflow() throws IOException { + byte[] encoded = encodeLong(-1); + + thrown.expect(IOException.class); + + int parsed = decodeInt(encoded); + } + + @Test + public void decodeThrowsExceptionForNonterminated() throws IOException { + final byte[] nonTerminatedNumber = + { (byte) 0xff, (byte) 0xff }; + + thrown.expect(IOException.class); + + long parsed = decodeLong(nonTerminatedNumber); + } + + @Test + public void decodeParsesEncodedValues() throws IOException { + ByteArrayOutputStream outStream = new ByteArrayOutputStream(); + for (int i = 10; i < Integer.MAX_VALUE; i = (int) (i * 1.1)) { + VarInt.encode(i, outStream); + VarInt.encode(-i, outStream); + } + for (long i = 10; i < Long.MAX_VALUE; i = (long) (i * 1.1)) { + VarInt.encode(i, outStream); + VarInt.encode(-i, outStream); + } + + ByteArrayInputStream inStream = + new ByteArrayInputStream(outStream.toByteArray()); + for (int i = 10; i < Integer.MAX_VALUE; i = (int) (i * 1.1)) { + assertEquals(i, VarInt.decodeInt(inStream)); + assertEquals(-i, VarInt.decodeInt(inStream)); + } + for (long i = 10; i < Long.MAX_VALUE; i = (long) (i * 1.1)) { + assertEquals(i, VarInt.decodeLong(inStream)); + assertEquals(-i, VarInt.decodeLong(inStream)); + } + } + + @Test + public void endOfFileThrowsException() throws Exception { + ByteArrayInputStream inStream = + new ByteArrayInputStream(new byte[0]); + thrown.expect(EOFException.class); + VarInt.decodeInt(inStream); + } + + @Test + public void unterminatedThrowsException() throws Exception { + byte[] e = encodeLong(Long.MAX_VALUE); + byte[] s = new byte[1]; + s[0] = e[0]; + ByteArrayInputStream inStream = new ByteArrayInputStream(s); + thrown.expect(IOException.class); + VarInt.decodeInt(inStream); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/WindowedValueTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/WindowedValueTest.java new file mode 100644 index 0000000000000..67f21f5490928 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/WindowedValueTest.java @@ -0,0 +1,55 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; + +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; + +/** Test case for {@link WindowedValue}. */ +@RunWith(JUnit4.class) +public class WindowedValueTest { + @Test + public void testWindowedValueCoder() throws CoderException { + Instant timestamp = new Instant(1234); + WindowedValue value = WindowedValue.of( + "abc", + new Instant(1234), + Arrays.asList(new IntervalWindow(timestamp, timestamp.plus(1000)), + new IntervalWindow(timestamp.plus(1000), timestamp.plus(2000)))); + + Coder> windowedValueCoder = + WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); + + byte[] encodedValue = CoderUtils.encodeToByteArray(windowedValueCoder, value); + WindowedValue decodedValue = + CoderUtils.decodeFromByteArray(windowedValueCoder, encodedValue); + + Assert.assertEquals(value.getValue(), decodedValue.getValue()); + Assert.assertEquals(value.getTimestamp(), decodedValue.getTimestamp()); + Assert.assertArrayEquals(value.getWindows().toArray(), decodedValue.getWindows().toArray()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterSetTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterSetTest.java new file mode 100644 index 0000000000000..c8fa6c2fab5a0 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterSetTest.java @@ -0,0 +1,75 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MAX; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SET; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Unit tests for {@link CounterSet}. + */ +@RunWith(JUnit4.class) +public class CounterSetTest { + @Test + public void testSet() { + CounterSet set = new CounterSet(); + assertTrue(set.add(Counter.longs("c1", SUM))); + assertFalse(set.add(Counter.longs("c1", SUM))); + assertTrue(set.add(Counter.longs("c2", MAX))); + assertEquals(2, set.size()); + } + + @Test + public void testAddCounterMutator() { + CounterSet set = new CounterSet(); + Counter c1 = Counter.longs("c1", SUM); + Counter c1SecondInstance = Counter.longs("c1", SUM); + Counter c1IncompatibleInstance = Counter.longs("c1", SET); + Counter c2 = Counter.longs("c2", MAX); + Counter c2IncompatibleInstance = Counter.doubles("c2", MAX); + + assertEquals(c1, set.getAddCounterMutator().addCounter(c1)); + assertEquals(c2, set.getAddCounterMutator().addCounter(c2)); + + assertEquals(c1, set.getAddCounterMutator().addCounter(c1SecondInstance)); + + try { + set.getAddCounterMutator().addCounter(c1IncompatibleInstance); + fail("should have failed"); + } catch (IllegalArgumentException exn) { + // Expected. + } + + try { + set.getAddCounterMutator().addCounter(c2IncompatibleInstance); + fail("should have failed"); + } catch (IllegalArgumentException exn) { + // Expected. + } + + assertEquals(2, set.size()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTest.java new file mode 100644 index 0000000000000..ff40e0d06f182 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTest.java @@ -0,0 +1,743 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import static com.google.cloud.dataflow.sdk.util.Values.asBoolean; +import static com.google.cloud.dataflow.sdk.util.Values.asDouble; +import static com.google.cloud.dataflow.sdk.util.Values.asLong; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.AND; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MAX; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MIN; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.OR; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SET; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.cloud.dataflow.sdk.util.CloudCounterUtils; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.common.collect.Sets; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Unit tests for the {@link Counter} API. + */ +@RunWith(JUnit4.class) +public class CounterTest { + + private static MetricUpdate flush(Counter c) { + // TODO: Move this out into a separate Counter test. + return CounterTestUtils.extractCounterUpdate(c, true); + } + + private static final double EPSILON = 0.00000000001; + + @Test + public void testNameKindAndCloudCounterRepresentation() { + Counter c1 = Counter.longs("c1", SUM); + Counter c2 = Counter.doubles("c2", MAX); + Counter c3 = Counter.strings("c3", SET); + Counter c4 = Counter.doubles("c4", MEAN); + Counter c5 = Counter.ints("c5", MIN); + Counter c6 = Counter.booleans("c6", AND); + Counter c7 = Counter.booleans("c7", OR); + + assertEquals("c1", c1.getName()); + assertEquals(SUM, c1.getKind()); + MetricUpdate cc = flush(c1); + assertEquals("c1", cc.getName().getName()); + assertEquals("SUM", cc.getKind()); + assertEquals(0L, asLong(cc.getScalar()).longValue()); + c1.addValue(123L).addValue(-13L); + cc = flush(c1); + assertEquals(110L, asLong(cc.getScalar()).longValue()); + + assertEquals("c2", c2.getName()); + assertEquals(MAX, c2.getKind()); + cc = flush(c2); + assertEquals("c2", cc.getName().getName()); + assertEquals("MAX", cc.getKind()); + assertEquals(Double.MIN_VALUE, asDouble(cc.getScalar()), EPSILON); + c2.resetToValue(0.0).addValue(Math.PI).addValue(Math.E); + cc = flush(c2); + assertEquals(Math.PI, asDouble(cc.getScalar()), EPSILON); + + assertEquals("c3", c3.getName()); + assertEquals(SET, c3.getKind()); + cc = flush(c3); // empty sets are not sent to the service + assertEquals(null, cc); + c3.addValue("abc").addValue("e").addValue("abc"); + cc = flush(c3); + assertEquals("c3", cc.getName().getName()); + assertEquals("SET", cc.getKind()); + Set s = (Set) cc.getSet(); + assertEquals(2, s.size()); + assertTrue(s.containsAll(Arrays.asList( + CloudObject.forString("e"), + CloudObject.forString("abc")))); + + assertEquals("c4", c4.getName()); + assertEquals(MEAN, c4.getKind()); + cc = flush(c4); // zero-count means are not sent to the service + assertEquals(null, cc); + c4.addValue(Math.PI).addValue(Math.E).addValue(Math.sqrt(2)); + cc = flush(c4); + assertEquals("c4", cc.getName().getName()); + assertEquals("MEAN", cc.getKind()); + Object ms = cc.getMeanSum(); + Object mc = cc.getMeanCount(); + assertEquals(Math.PI + Math.E + Math.sqrt(2), asDouble(ms), EPSILON); + assertEquals(3, asLong(mc).longValue()); + c4.addValue(2.0).addValue(5.0); + cc = flush(c4); + ms = cc.getMeanSum(); + mc = cc.getMeanCount(); + assertEquals(7.0, asDouble(ms), EPSILON); + assertEquals(2L, asLong(mc).longValue()); + + assertEquals("c5", c5.getName()); + assertEquals(MIN, c5.getKind()); + cc = flush(c5); + assertEquals("c5", cc.getName().getName()); + assertEquals("MIN", cc.getKind()); + assertEquals(Integer.MAX_VALUE, asLong(cc.getScalar()).longValue()); + c5.addValue(123).addValue(-13); + cc = flush(c5); + assertEquals(-13, asLong(cc.getScalar()).longValue()); + + assertEquals("c6", c6.getName()); + assertEquals(AND, c6.getKind()); + cc = flush(c6); + assertEquals("c6", cc.getName().getName()); + assertEquals("AND", cc.getKind()); + assertEquals(true, asBoolean(cc.getScalar())); + c6.addValue(false); + cc = flush(c6); + assertEquals(false, asBoolean(cc.getScalar())); + + assertEquals("c7", c7.getName()); + assertEquals(OR, c7.getKind()); + cc = flush(c7); + assertEquals("c7", cc.getName().getName()); + assertEquals("OR", cc.getKind()); + assertEquals(false, asBoolean(cc.getScalar())); + c7.addValue(true); + cc = flush(c7); + assertEquals(true, asBoolean(cc.getScalar())); + } + + @Test + public void testCompatibility() { + // Equal counters are compatible, of all kinds. + assertTrue( + Counter.longs("c", SUM).isCompatibleWith(Counter.longs("c", SUM))); + assertTrue( + Counter.ints("c", SUM).isCompatibleWith(Counter.ints("c", SUM))); + assertTrue( + Counter.doubles("c", SUM).isCompatibleWith(Counter.doubles("c", SUM))); + assertTrue( + Counter.strings("c", SET).isCompatibleWith(Counter.strings("c", SET))); + assertTrue( + Counter.booleans("c", OR).isCompatibleWith( + Counter.booleans("c", OR))); + + // The name, kind, and type of the counter must match. + assertFalse( + Counter.longs("c", SUM).isCompatibleWith(Counter.longs("c2", SUM))); + assertFalse( + Counter.longs("c", SUM).isCompatibleWith(Counter.longs("c", MAX))); + assertFalse( + Counter.longs("c", SUM).isCompatibleWith(Counter.ints("c", SUM))); + + // The value of the counters are ignored. + assertTrue( + Counter.longs("c", SUM).resetToValue(666L).isCompatibleWith( + Counter.longs("c", SUM).resetToValue(42L))); + } + + + private void assertOK(long total, long delta, Counter c) { + assertEquals(total, c.getTotalAggregate().longValue()); + assertEquals(delta, c.getDeltaAggregate().longValue()); + } + + private void assertOK(double total, double delta, Counter c) { + assertEquals(total, asDouble(c.getTotalAggregate()), EPSILON); + assertEquals(delta, asDouble(c.getDeltaAggregate()), EPSILON); + } + + + // Tests for SUM. + + @Test + public void testSumLong() { + Counter c = Counter.longs("sum-long", SUM); + long expectedTotal = 0; + long expectedDelta = 0; + assertOK(expectedTotal, expectedDelta, c); + + c.addValue(13L).addValue(42L).addValue(0L); + expectedTotal += 55; + expectedDelta += 55; + assertOK(expectedTotal, expectedDelta, c); + + c.resetToValue(120L).addValue(17L).addValue(37L); + expectedTotal = expectedDelta = 174; + assertOK(expectedTotal, expectedDelta, c); + + flush(c); + expectedDelta = 0; + assertOK(expectedTotal, expectedDelta, c); + + c.addValue(15L).addValue(42L); + expectedTotal += 57; + expectedDelta += 57; + assertOK(expectedTotal, expectedDelta, c); + + c.resetToValue(100L).addValue(17L).addValue(49L); + expectedTotal = expectedDelta = 166; + assertOK(expectedTotal, expectedDelta, c); + } + + @Test + public void testSumDouble() { + Counter c = Counter.doubles("sum-double", SUM); + double expectedTotal = 0.0; + double expectedDelta = 0.0; + assertOK(expectedTotal, expectedDelta, c); + + c.addValue(Math.E).addValue(Math.PI).addValue(0.0); + expectedTotal += Math.E + Math.PI; + expectedDelta += Math.E + Math.PI; + assertOK(expectedTotal, expectedDelta, c); + + c.resetToValue(Math.sqrt(2)).addValue(2 * Math.PI).addValue(3 * Math.E); + expectedTotal = expectedDelta = Math.sqrt(2) + 2 * Math.PI + 3 * Math.E; + assertOK(expectedTotal, expectedDelta, c); + + flush(c); + expectedDelta = 0.0; + assertOK(expectedTotal, expectedDelta, c); + + c.addValue(7 * Math.PI).addValue(5 * Math.E); + expectedTotal += 7 * Math.PI + 5 * Math.E; + expectedDelta += 7 * Math.PI + 5 * Math.E; + assertOK(expectedTotal, expectedDelta, c); + + c.resetToValue(Math.sqrt(17)).addValue(17.0).addValue(49.0); + expectedTotal = expectedDelta = Math.sqrt(17.0) + 17.0 + 49.0; + assertOK(expectedTotal, expectedDelta, c); + } + + + // Tests for MAX. + + @Test + public void testMaxLong() { + Counter c = Counter.longs("max-long", MAX); + long expectedTotal = Long.MIN_VALUE; + long expectedDelta = Long.MIN_VALUE; + assertOK(expectedTotal, expectedDelta, c); + + c.addValue(13L).addValue(42L).addValue(Long.MIN_VALUE); + expectedTotal = expectedDelta = 42; + assertOK(expectedTotal, expectedDelta, c); + + c.resetToValue(120L).addValue(17L).addValue(37L); + expectedTotal = expectedDelta = 120; + assertOK(expectedTotal, expectedDelta, c); + + flush(c); + expectedDelta = Long.MIN_VALUE; + assertOK(expectedTotal, expectedDelta, c); + + c.addValue(42L).addValue(15L); + expectedDelta = 42; + assertOK(expectedTotal, expectedDelta, c); + + c.resetToValue(100L).addValue(171L).addValue(49L); + expectedTotal = expectedDelta = 171; + assertOK(expectedTotal, expectedDelta, c); + } + + @Test + public void testMaxDouble() { + Counter c = Counter.doubles("max-double", MAX); + double expectedTotal = Double.MIN_VALUE; + double expectedDelta = Double.MIN_VALUE; + assertOK(expectedTotal, expectedDelta, c); + + c.addValue(Math.E).addValue(Math.PI).addValue(Double.MIN_VALUE); + expectedTotal = expectedDelta = Math.PI; + assertOK(expectedTotal, expectedDelta, c); + + c.resetToValue(Math.sqrt(12345)).addValue(2 * Math.PI).addValue(3 * Math.E); + expectedTotal = expectedDelta = Math.sqrt(12345); + assertOK(expectedTotal, expectedDelta, c); + + flush(c); + expectedDelta = Double.MIN_VALUE; + assertOK(expectedTotal, expectedDelta, c); + + c.addValue(7 * Math.PI).addValue(5 * Math.E); + expectedDelta = 7 * Math.PI; + assertOK(expectedTotal, expectedDelta, c); + + c.resetToValue(Math.sqrt(17)).addValue(171.0).addValue(49.0); + expectedTotal = expectedDelta = 171.0; + assertOK(expectedTotal, expectedDelta, c); + } + + + // Tests for MIN. + + @Test + public void testMinLong() { + Counter c = Counter.longs("min-long", MIN); + long expectedTotal = Long.MAX_VALUE; + long expectedDelta = Long.MAX_VALUE; + assertOK(expectedTotal, expectedDelta, c); + + c.addValue(13L).addValue(42L).addValue(Long.MAX_VALUE); + expectedTotal = expectedDelta = 13; + assertOK(expectedTotal, expectedDelta, c); + + c.resetToValue(120L).addValue(17L).addValue(37L); + expectedTotal = expectedDelta = 17; + assertOK(expectedTotal, expectedDelta, c); + + flush(c); + expectedDelta = Long.MAX_VALUE; + assertOK(expectedTotal, expectedDelta, c); + + c.addValue(42L).addValue(18L); + expectedDelta = 18; + assertOK(expectedTotal, expectedDelta, c); + + c.resetToValue(100L).addValue(171L).addValue(49L); + expectedTotal = expectedDelta = 49; + assertOK(expectedTotal, expectedDelta, c); + } + + @Test + public void testMinDouble() { + Counter c = Counter.doubles("min-double", MIN); + double expectedTotal = Double.MAX_VALUE; + double expectedDelta = Double.MAX_VALUE; + assertOK(expectedTotal, expectedDelta, c); + + c.addValue(Math.E).addValue(Math.PI).addValue(Double.MAX_VALUE); + expectedTotal = expectedDelta = Math.E; + assertOK(expectedTotal, expectedDelta, c); + + c.resetToValue(Math.sqrt(12345)).addValue(2 * Math.PI).addValue(3 * Math.E); + expectedTotal = expectedDelta = 2 * Math.PI; + assertOK(expectedTotal, expectedDelta, c); + + flush(c); + expectedDelta = Double.MAX_VALUE; + assertOK(expectedTotal, expectedDelta, c); + + c.addValue(7 * Math.PI).addValue(5 * Math.E); + expectedDelta = 5 * Math.E; + assertOK(expectedTotal, expectedDelta, c); + + c.resetToValue(Math.sqrt(17)).addValue(171.0).addValue(0.0); + expectedTotal = expectedDelta = 0.0; + assertOK(expectedTotal, expectedDelta, c); + } + + + // Tests for MEAN. + + private void assertMean(long s, long sd, long c, long cd, Counter cn) { + assertEquals(s, cn.getTotalAggregate().longValue()); + assertEquals(sd, cn.getDeltaAggregate().longValue()); + assertEquals(c, cn.getTotalCount()); + assertEquals(cd, cn.getDeltaCount()); + } + + private void assertMean(double s, double sd, long c, long cd, + Counter cn) { + assertEquals(s, cn.getTotalAggregate().doubleValue(), EPSILON); + assertEquals(sd, cn.getDeltaAggregate().doubleValue(), EPSILON); + assertEquals(c, cn.getTotalCount()); + assertEquals(cd, cn.getDeltaCount()); + } + + @Test + public void testMeanLong() { + Counter c = Counter.longs("mean-long", MEAN); + long expTotal = 0; + long expDelta = 0; + long expCountTotal = 0; + long expCountDelta = 0; + assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); + + c.addValue(13L).addValue(42L).addValue(0L); + expTotal += 55; + expDelta += 55; + expCountTotal += 3; + expCountDelta += 3; + assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); + + c.resetToValue(1L, 120L).addValue(17L).addValue(37L); + expTotal = expDelta = 174; + assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); + + flush(c); + expDelta = 0; + expCountDelta = 0; + assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); + + c.addValue(15L).addValue(42L); + expTotal += 57; + expDelta += 57; + expCountTotal += 2; + expCountDelta += 2; + assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); + + c.resetToValue(3L, 100L).addValue(17L).addValue(49L); + expTotal = expDelta = 166; + expCountTotal = expCountDelta = 5; + assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); + } + + @Test + public void testMeanDouble() { + Counter c = Counter.doubles("mean-double", MEAN); + double expTotal = 0.0; + double expDelta = 0.0; + long expCountTotal = 0; + long expCountDelta = 0; + assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); + + c.addValue(Math.E).addValue(Math.PI).addValue(0.0); + expTotal += Math.E + Math.PI; + expDelta += Math.E + Math.PI; + expCountTotal += 3; + expCountDelta += 3; + assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); + + c.resetToValue(1L, Math.sqrt(2)).addValue(2 * Math.PI).addValue(3 * Math.E); + expTotal = expDelta = Math.sqrt(2) + 2 * Math.PI + 3 * Math.E; + assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); + + flush(c); + expDelta = 0.0; + expCountDelta = 0; + assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); + + c.addValue(7 * Math.PI).addValue(5 * Math.E); + expTotal += 7 * Math.PI + 5 * Math.E; + expDelta += 7 * Math.PI + 5 * Math.E; + expCountTotal += 2; + expCountDelta += 2; + assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); + + c.resetToValue(3L, Math.sqrt(17)).addValue(17.0).addValue(49.0); + expTotal = expDelta = Math.sqrt(17.0) + 17.0 + 49.0; + expCountTotal = expCountDelta = 5; + assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); + } + + + // Tests for SET. + + private void assertSet(Set total, Set delta, Counter c) { + assertTrue(total.containsAll(c.getTotalSet())); + assertTrue(c.getTotalSet().containsAll(total)); + assertTrue(delta.containsAll(c.getDeltaSet())); + assertTrue(c.getDeltaSet().containsAll(delta)); + } + + @Test + public void testSetLong() { + Counter c = Counter.longs("set-long", SET); + HashSet expectedTotal = new HashSet<>(); + HashSet expectedDelta = new HashSet<>(); + assertSet(expectedTotal, expectedDelta, c); + + c.addValue(13L).addValue(42L).addValue(13L); + expectedTotal = expectedDelta = Sets.newHashSet(13L, 42L); + assertSet(expectedTotal, expectedDelta, c); + + c.resetToValue(120L).addValue(17L).addValue(37L); + expectedTotal = expectedDelta = Sets.newHashSet(120L, 17L, 37L); + assertSet(expectedTotal, expectedDelta, c); + + flush(c); + expectedDelta = new HashSet<>(); + assertSet(expectedTotal, expectedDelta, c); + + c.addValue(42L).addValue(18L); + expectedTotal.addAll(Arrays.asList(42L, 18L)); + expectedDelta = Sets.newHashSet(42L, 18L); + assertSet(expectedTotal, expectedDelta, c); + + c.resetToValue(100L).addValue(171L).addValue(49L); + expectedTotal = expectedDelta = Sets.newHashSet(100L, 171L, 49L); + assertSet(expectedTotal, expectedDelta, c); + } + + @Test + public void testSetDouble() { + Counter c = Counter.doubles("set-double", SET); + HashSet expectedTotal = new HashSet<>(); + HashSet expectedDelta = new HashSet<>(); + assertSet(expectedTotal, expectedDelta, c); + + c.addValue(Math.E).addValue(Math.PI); + expectedTotal = expectedDelta = Sets.newHashSet(Math.E, Math.PI); + assertSet(expectedTotal, expectedDelta, c); + + c.resetToValue(Math.sqrt(12345)).addValue(2 * Math.PI).addValue(3 * Math.E); + expectedTotal = + expectedDelta = Sets.newHashSet(Math.sqrt(12345), 2 * Math.PI, 3 * Math.E); + assertSet(expectedTotal, expectedDelta, c); + + flush(c); + expectedDelta = new HashSet<>(); + assertSet(expectedTotal, expectedDelta, c); + + c.addValue(7 * Math.PI).addValue(5 * Math.E); + expectedTotal.addAll(Arrays.asList(7 * Math.PI, 5 * Math.E)); + expectedDelta = Sets.newHashSet(7 * Math.PI, 5 * Math.E); + assertSet(expectedTotal, expectedDelta, c); + + c.resetToValue(Math.sqrt(17)).addValue(171.0).addValue(0.0); + expectedTotal = expectedDelta = Sets.newHashSet(Math.sqrt(17), 171.0, 0.0); + assertSet(expectedTotal, expectedDelta, c); + } + + @Test + public void testSetString() { + Counter c = Counter.strings("set-string", SET); + HashSet expectedTotal = new HashSet<>(); + HashSet expectedDelta = new HashSet<>(); + assertSet(expectedTotal, expectedDelta, c); + + c.addValue("a").addValue("b").addValue("a"); + expectedTotal = expectedDelta = Sets.newHashSet("a", "b"); + assertSet(expectedTotal, expectedDelta, c); + + c.resetToValue("c").addValue("d").addValue("e"); + expectedTotal = expectedDelta = Sets.newHashSet("c", "d", "e"); + assertSet(expectedTotal, expectedDelta, c); + + flush(c); + expectedDelta = new HashSet<>(); + assertSet(expectedTotal, expectedDelta, c); + + c.addValue("b").addValue("f"); + expectedTotal.addAll(Arrays.asList("b", "f")); + expectedDelta = Sets.newHashSet("b", "f"); + assertSet(expectedTotal, expectedDelta, c); + + c.resetToValue("g").addValue("h").addValue("i"); + expectedTotal = expectedDelta = Sets.newHashSet("g", "h", "i"); + assertSet(expectedTotal, expectedDelta, c); + } + + + // Test for AND and OR. + + private void assertBool(boolean total, boolean delta, Counter c) { + assertEquals(total, c.getTotalAggregate().booleanValue()); + assertEquals(delta, c.getDeltaAggregate().booleanValue()); + } + + @Test + public void testBoolAnd() { + Counter c = Counter.booleans("bool-and", AND); + boolean expectedTotal = true; + boolean expectedDelta = true; + assertBool(expectedTotal, expectedDelta, c); + + c.addValue(true); + assertBool(expectedTotal, expectedDelta, c); + + c.addValue(false); + expectedTotal = expectedDelta = false; + assertBool(expectedTotal, expectedDelta, c); + + c.resetToValue(true).addValue(true); + expectedTotal = expectedDelta = true; + assertBool(expectedTotal, expectedDelta, c); + + c.addValue(false); + expectedTotal = expectedDelta = false; + assertBool(expectedTotal, expectedDelta, c); + + flush(c); + expectedDelta = true; + assertBool(expectedTotal, expectedDelta, c); + + c.addValue(false); + expectedDelta = false; + assertBool(expectedTotal, expectedDelta, c); + + c.addValue(true); + assertBool(expectedTotal, expectedDelta, c); + } + + @Test + public void testBoolOr() { + Counter c = Counter.booleans("bool-or", OR); + boolean expectedTotal = false; + boolean expectedDelta = false; + assertBool(expectedTotal, expectedDelta, c); + + c.addValue(false); + assertBool(expectedTotal, expectedDelta, c); + + c.addValue(true); + expectedTotal = expectedDelta = true; + assertBool(expectedTotal, expectedDelta, c); + + c.resetToValue(false).addValue(false); + expectedTotal = expectedDelta = false; + assertBool(expectedTotal, expectedDelta, c); + + c.addValue(true); + expectedTotal = expectedDelta = true; + assertBool(expectedTotal, expectedDelta, c); + + flush(c); + expectedDelta = false; + assertBool(expectedTotal, expectedDelta, c); + + c.addValue(true); + expectedDelta = true; + assertBool(expectedTotal, expectedDelta, c); + + c.addValue(false); + assertBool(expectedTotal, expectedDelta, c); + } + + + // Incompatibility tests. + + @Test(expected = IllegalArgumentException.class) + public void testSumBool() { + Counter.booleans("counter", SUM); + } + + @Test(expected = IllegalArgumentException.class) + public void testSumString() { + Counter.strings("counter", SUM); + } + + @Test(expected = IllegalArgumentException.class) + public void testMinBool() { + Counter.booleans("counter", MIN); + } + + @Test(expected = IllegalArgumentException.class) + public void testMinString() { + Counter.strings("counter", MIN); + } + + @Test(expected = IllegalArgumentException.class) + public void testMaxBool() { + Counter.booleans("counter", MAX); + } + + @Test(expected = IllegalArgumentException.class) + public void testMaxString() { + Counter.strings("counter", MAX); + } + + @Test(expected = IllegalArgumentException.class) + public void testMeanBool() { + Counter.booleans("counter", MEAN); + } + + @Test(expected = IllegalArgumentException.class) + public void testMeanString() { + Counter.strings("counter", MEAN); + } + + @Test(expected = IllegalArgumentException.class) + public void testSetBool() { + Counter.booleans("counter", SET); + } + + @Test(expected = IllegalArgumentException.class) + public void testAndLong() { + Counter.longs("counter", AND); + } + + @Test(expected = IllegalArgumentException.class) + public void testAndDouble() { + Counter.doubles("counter", AND); + } + + @Test(expected = IllegalArgumentException.class) + public void testAndString() { + Counter.strings("counter", AND); + } + + @Test(expected = IllegalArgumentException.class) + public void testOrLong() { + Counter.longs("counter", OR); + } + + @Test(expected = IllegalArgumentException.class) + public void testOrDouble() { + Counter.doubles("counter", OR); + } + + @Test(expected = IllegalArgumentException.class) + public void testOrString() { + Counter.strings("counter", OR); + } + + @Test + public void testExtraction() { + Counter[] counters = {Counter.longs("c1", SUM), + Counter.doubles("c2", MAX), + Counter.strings("c3", SET)}; + CounterSet set = new CounterSet(); + for (Counter c : counters) { + set.addCounter(c); + } + + List cloudCountersFromSet = CloudCounterUtils.extractCounters(set, true); + + List cloudCountersFromArray = + CounterTestUtils.extractCounterUpdates(Arrays.asList(counters), true); + + assertEquals(cloudCountersFromArray.size(), cloudCountersFromSet.size()); + for (int i = 0; i < cloudCountersFromArray.size(); i++) { + assertEquals(cloudCountersFromArray.get(i), cloudCountersFromSet.get(i)); + } + + assertEquals(2, cloudCountersFromSet.size()); // empty set was ignored + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTestUtils.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTestUtils.java new file mode 100644 index 0000000000000..9c428476e28f0 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTestUtils.java @@ -0,0 +1,123 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN; + +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.util.CloudCounterUtils; + +import org.junit.Assert; + +import java.io.ByteArrayOutputStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Set; + +/** + * Utilities for testing {@link Counter}s. + */ +public class CounterTestUtils { + + /** + * Extracts a MetricUpdate update from the given counter. This is used mainly + * for testing. + * + * @param extractDelta specifies whether or not to extract the cumulative + * aggregate value or the delta since the last extraction. + */ + public static MetricUpdate extractCounterUpdate(Counter counter, + boolean extractDelta) { + // This may be invoked asynchronously with regular counter updates but + // access to counter data is synchronized, so this is safe. + return CloudCounterUtils.extractCounter(counter, extractDelta); + } + + /** + * Extracts MetricUpdate updates from the given counters. This is used mainly + * for testing. + * + * @param extractDelta specifies whether or not to extract the cumulative + * aggregate values or the deltas since the last extraction. + */ + public static List extractCounterUpdates( + Collection> counters, boolean extractDelta) { + // This may be invoked asynchronously with regular counter updates but + // access to counter data is synchronized, so this is safe. Note however + // that the result is NOT an atomic snapshot across all given counters. + List cloudCounters = new ArrayList<>(counters.size()); + for (Counter counter : counters) { + MetricUpdate cloudCounter = extractCounterUpdate(counter, extractDelta); + if (null != cloudCounter) { + cloudCounters.add(cloudCounter); + } + } + return cloudCounters; + } + + + // These methods expose a counter's values for testing. + + public static T getTotalAggregate(Counter counter) { + return counter.getTotalAggregate(); + } + + public static T getDeltaAggregate(Counter counter) { + return counter.getDeltaAggregate(); + } + + public static long getTotalCount(Counter counter) { + return counter.getTotalCount(); + } + + public static long getDeltaCount(Counter counter) { + return counter.getDeltaCount(); + } + + public static Set getTotalSet(Counter counter) { + return counter.getTotalSet(); + } + + public static Set getDeltaSet(Counter counter) { + return counter.getDeltaSet(); + } + + /** + * A utility method that passes the given (unencoded) elements through + * coder's registerByteSizeObserver() and encode() methods, and confirms + * they are mutually consistent. This is useful for testing coder + * implementations. + */ + public static void testByteCount(Coder coder, Coder.Context context, Object[] elements) + throws Exception { + Counter meanByteCount = Counter.longs("meanByteCount", MEAN); + ElementByteSizeObserver observer = new ElementByteSizeObserver(meanByteCount); + + ByteArrayOutputStream os = new ByteArrayOutputStream(); + for (Object elem : elements) { + coder.registerByteSizeObserver(elem, observer, context); + coder.encode(elem, os, context); + observer.advance(); + } + long expectedLength = os.toByteArray().length; + + Assert.assertEquals(expectedLength, (long) getTotalAggregate(meanByteCount)); + Assert.assertEquals(elements.length, (long) getTotalCount(meanByteCount)); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/MetricTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/MetricTest.java new file mode 100644 index 0000000000000..0c60901ca0a69 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/MetricTest.java @@ -0,0 +1,40 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import static org.junit.Assert.assertEquals; + +import com.google.cloud.dataflow.sdk.util.common.Metric.DoubleMetric; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link Metric}. */ +@RunWith(JUnit4.class) +public class MetricTest { + @Test + public void testDoubleMetric() { + String name = "metric-name"; + double value = 3.14; + + DoubleMetric doubleMetric = new DoubleMetric(name, value); + + assertEquals(name, doubleMetric.getName()); + assertEquals((Double) value, doubleMetric.getValue()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/BatchingShuffleEntryReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/BatchingShuffleEntryReaderTest.java new file mode 100644 index 0000000000000..5a41494717280 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/BatchingShuffleEntryReaderTest.java @@ -0,0 +1,138 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.api.client.util.Lists.newArrayList; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +import com.google.cloud.dataflow.sdk.runners.worker.ByteArrayShufflePosition; +import com.google.cloud.dataflow.sdk.util.common.Reiterator; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** Unit tests for {@link BatchingShuffleEntryReader}. */ +@RunWith(JUnit4.class) +public final class BatchingShuffleEntryReaderTest { + private static final byte[] KEY = {0xA}; + private static final byte[] SKEY = {0xB}; + private static final byte[] VALUE = {0xC}; + private static final ShufflePosition START_POSITION = + ByteArrayShufflePosition.of("aaa".getBytes()); + private static final ShufflePosition END_POSITION = + ByteArrayShufflePosition.of("zzz".getBytes()); + private static final ShufflePosition NEXT_START_POSITION = + ByteArrayShufflePosition.of("next".getBytes()); + private static final ShufflePosition SECOND_NEXT_START_POSITION = + ByteArrayShufflePosition.of("next-second".getBytes()); + + @Mock private ShuffleBatchReader batchReader; + private ShuffleEntryReader reader; + + @Before + public void initMocksAndReader() { + MockitoAnnotations.initMocks(this); + reader = new BatchingShuffleEntryReader(batchReader); + } + + @Test + public void readerCanRead() throws Exception { + ShuffleEntry e1 = new ShuffleEntry(KEY, SKEY, VALUE); + ShuffleEntry e2 = new ShuffleEntry(KEY, SKEY, VALUE); + ArrayList entries = new ArrayList<>(); + entries.add(e1); + entries.add(e2); + when(batchReader.read(START_POSITION, END_POSITION)) + .thenReturn(new ShuffleBatchReader.Batch(entries, null)); + List results = newArrayList(reader.read(START_POSITION, END_POSITION)); + assertThat(results, contains(e1, e2)); + } + + @Test + public void readerIteratorCanBeCopied() throws Exception { + ShuffleEntry e1 = new ShuffleEntry(KEY, SKEY, VALUE); + ShuffleEntry e2 = new ShuffleEntry(KEY, SKEY, VALUE); + ArrayList entries = new ArrayList<>(); + entries.add(e1); + entries.add(e2); + when(batchReader.read(START_POSITION, END_POSITION)) + .thenReturn(new ShuffleBatchReader.Batch(entries, null)); + Reiterator it = reader.read(START_POSITION, END_POSITION); + assertThat(it.hasNext(), equalTo(Boolean.TRUE)); + assertThat(it.next(), equalTo(e1)); + Reiterator copy = it.copy(); + assertThat(it.hasNext(), equalTo(Boolean.TRUE)); + assertThat(it.next(), equalTo(e2)); + assertThat(it.hasNext(), equalTo(Boolean.FALSE)); + assertThat(copy.hasNext(), equalTo(Boolean.TRUE)); + assertThat(copy.next(), equalTo(e2)); + assertThat(copy.hasNext(), equalTo(Boolean.FALSE)); + } + + @Test + public void readerShouldMergeMultipleBatchResults() throws Exception { + ShuffleEntry e1 = new ShuffleEntry(KEY, SKEY, VALUE); + List e1s = Collections.singletonList(e1); + ShuffleEntry e2 = new ShuffleEntry(KEY, SKEY, VALUE); + List e2s = Collections.singletonList(e2); + when(batchReader.read(START_POSITION, END_POSITION)) + .thenReturn(new ShuffleBatchReader.Batch(e1s, NEXT_START_POSITION)); + when(batchReader.read(NEXT_START_POSITION, END_POSITION)) + .thenReturn(new ShuffleBatchReader.Batch(e2s, null)); + List results = newArrayList(reader.read(START_POSITION, END_POSITION)); + assertThat(results, contains(e1, e2)); + + verify(batchReader).read(START_POSITION, END_POSITION); + verify(batchReader).read(NEXT_START_POSITION, END_POSITION); + verifyNoMoreInteractions(batchReader); + } + + @Test + public void readerShouldMergeMultipleBatchResultsIncludingEmptyShards() + throws Exception { + List e1s = new ArrayList<>(); + List e2s = new ArrayList<>(); + ShuffleEntry e3 = new ShuffleEntry(KEY, SKEY, VALUE); + List e3s = Collections.singletonList(e3); + when(batchReader.read(START_POSITION, END_POSITION)) + .thenReturn(new ShuffleBatchReader.Batch(e1s, NEXT_START_POSITION)); + when(batchReader.read(NEXT_START_POSITION, END_POSITION)) + .thenReturn(new ShuffleBatchReader.Batch(e2s, SECOND_NEXT_START_POSITION)); + when(batchReader.read(SECOND_NEXT_START_POSITION, END_POSITION)) + .thenReturn(new ShuffleBatchReader.Batch(e3s, null)); + List results = newArrayList(reader.read(START_POSITION, END_POSITION)); + assertThat(results, contains(e3)); + + verify(batchReader).read(START_POSITION, END_POSITION); + verify(batchReader).read(NEXT_START_POSITION, END_POSITION); + verify(batchReader).read(SECOND_NEXT_START_POSITION, END_POSITION); + verifyNoMoreInteractions(batchReader); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/CachingShuffleBatchReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/CachingShuffleBatchReaderTest.java new file mode 100644 index 0000000000000..4175c91505963 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/CachingShuffleBatchReaderTest.java @@ -0,0 +1,95 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.util.ArrayList; + +/** Unit tests for {@link CachingShuffleBatchReader}. */ +@RunWith(JUnit4.class) +public final class CachingShuffleBatchReaderTest { + + private final ShuffleBatchReader.Batch testBatch = + new ShuffleBatchReader.Batch(new ArrayList(), null); + + @Test + public void readerShouldCacheReads() throws IOException { + ShuffleBatchReader base = mock(ShuffleBatchReader.class); + CachingShuffleBatchReader reader = new CachingShuffleBatchReader(base); + when(base.read(null, null)).thenReturn(testBatch); + // N.B. We need to capture the result of reader.read() in order to ensure + // that there's a strong reference to it, preventing it from being + // collected. Not that this should be an issue in tests, but it's good to + // be solid. + ShuffleBatchReader.Batch read = reader.read(null, null); + assertThat(read, equalTo(testBatch)); + assertThat(reader.read(null, null), equalTo(testBatch)); + assertThat(reader.read(null, null), equalTo(testBatch)); + assertThat(reader.read(null, null), equalTo(testBatch)); + assertThat(reader.read(null, null), equalTo(testBatch)); + verify(base, times(1)).read(null, null); + } + + @Test + public void readerShouldNotCacheExceptions() throws IOException { + ShuffleBatchReader base = mock(ShuffleBatchReader.class); + CachingShuffleBatchReader reader = new CachingShuffleBatchReader(base); + when(base.read(null, null)) + .thenThrow(new IOException("test")) + .thenReturn(testBatch); + try { + reader.read(null, null); + fail("expected an IOException"); + } catch (IOException e) { + // Nothing to do -- exception is expected. + } + assertThat(reader.read(null, null), equalTo(testBatch)); + verify(base, times(2)).read(null, null); + } + + @Test + public void readerShouldRereadClearedBatches() throws IOException { + ShuffleBatchReader base = mock(ShuffleBatchReader.class); + CachingShuffleBatchReader reader = new CachingShuffleBatchReader(base); + when(base.read(null, null)).thenReturn(testBatch); + ShuffleBatchReader.Batch read = reader.read(null, null); + assertThat(read, equalTo(testBatch)); + verify(base, times(1)).read(null, null); + CachingShuffleBatchReader.BatchRange range = + new CachingShuffleBatchReader.BatchRange(null, null); + CachingShuffleBatchReader.RangeReadReference ref = + reader.cache.get(range); + assertThat(ref, notNullValue()); + ref.clear(); + read = reader.read(null, null); + assertThat(read, equalTo(testBatch)); + verify(base, times(2)).read(null, null); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ExecutorTestUtils.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ExecutorTestUtils.java new file mode 100644 index 0000000000000..0c678abe75d55 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ExecutorTestUtils.java @@ -0,0 +1,238 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.runners.worker.MapTaskExecutorFactory.ElementByteSizeObservableCoder; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObservable; + +import org.junit.Assert; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Observable; +import java.util.Observer; + +/** + * Utilities for tests. + */ +public class ExecutorTestUtils { + // Do not instantiate. + private ExecutorTestUtils() { } + + /** An Operation with a specified number of outputs. */ + public static class TestOperation extends Operation { + public TestOperation(int numOutputs) { + this(numOutputs, new CounterSet()); + } + + TestOperation(int numOutputs, CounterSet counters) { + this(numOutputs, counters, "test-"); + } + + TestOperation(int numOutputs, CounterSet counters, String counterPrefix) { + this(numOutputs, counterPrefix, counters.getAddCounterMutator(), + new StateSampler(counterPrefix, counters.getAddCounterMutator())); + } + + TestOperation(int numOutputs, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) { + super("TestOperation", + createOutputReceivers(numOutputs, counterPrefix, + addCounterMutator, stateSampler), + counterPrefix, + addCounterMutator, + stateSampler); + } + + private static OutputReceiver[] createOutputReceivers( + int numOutputs, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler) { + OutputReceiver[] receivers = new OutputReceiver[numOutputs]; + for (int i = 0; i < numOutputs; i++) { + receivers[i] = new OutputReceiver( + "out_" + i, + new ElementByteSizeObservableCoder(StringUtf8Coder.of()), + counterPrefix, + addCounterMutator); + } + return receivers; + } + } + + /** An OutputReceiver that allows the output elements to be retrieved. */ + public static class TestReceiver extends OutputReceiver { + List outputElems = new ArrayList<>(); + + public TestReceiver(CounterSet counterSet) { + this("test_receiver_out", counterSet); + } + + public TestReceiver(Coder coder) { + this(coder, new CounterSet()); + } + + public TestReceiver(Coder coder, CounterSet counterSet) { + this("test_receiver_out", + new ElementByteSizeObservableCoder(coder), + counterSet, + "test-"); + } + + public TestReceiver(CounterSet counterSet, String counterPrefix) { + this("test_receiver_out", counterSet, counterPrefix); + } + + public TestReceiver(String outputName, CounterSet counterSet) { + this(outputName, counterSet, "test-"); + } + + public TestReceiver(String outputName, + CounterSet counterSet, String counterPrefix) { + this(outputName, + new ElementByteSizeObservableCoder(StringUtf8Coder.of()), + counterSet, + counterPrefix); + } + + public TestReceiver(ElementByteSizeObservable elementByteSizeObservable, + CounterSet counterSet, String counterPrefix) { + this("test_receiver_out", elementByteSizeObservable, + counterSet, counterPrefix); + } + + public TestReceiver(String outputName, + ElementByteSizeObservable elementByteSizeObservable, + CounterSet counterSet, String counterPrefix) { + super(outputName, + elementByteSizeObservable, + counterPrefix, + counterSet.getAddCounterMutator()); + } + + @Override + public void process(Object elem) throws Exception { + super.process(elem); + outputElems.add(elem); + } + + @Override + protected boolean sampleElement() { + return true; + } + } + + /** A {@code Source} that yields a specified set of values. */ + public static class TestSource extends Source { + List inputs = new ArrayList<>(); + + public void addInput(String... inputs) { + this.inputs.addAll(Arrays.asList(inputs)); + } + + @Override + public SourceIterator iterator() { + return new TestSourceIterator(inputs); + } + + class TestSourceIterator extends AbstractSourceIterator { + Iterator iter; + boolean closed = false; + + public TestSourceIterator(List inputs) { + iter = inputs.iterator(); + } + + @Override + public boolean hasNext() { return iter.hasNext(); } + + @Override + public String next() { + String next = iter.next(); + notifyElementRead(next.length()); + return next; + } + + @Override + public void close() { + Assert.assertFalse(closed); + closed = true; + } + } + } + + /** + * An Observer that stores all sizes into an ArrayList, to compare + * against the gold standard during testing. + */ + public static class TestSourceObserver implements Observer { + private final Source source; + private final List sizes; + + public TestSourceObserver(Source source) { + this(source, new ArrayList()); + } + + public TestSourceObserver(Source source, List sizes) { + this.source = source; + this.sizes = sizes; + source.addObserver(this); + } + + @Override + public void update(Observable obs, Object obj) { + sizes.add((int) (long) obj); + } + + public List getActualSizes() { + return sizes; + } + } + + /** A {@code Sink} that allows the output elements to be retrieved. */ + public static class TestSink extends Sink { + List outputElems = new ArrayList<>(); + boolean closed = false; + + @Override + public SinkWriter writer() { + return new TestSinkWriter(); + } + + class TestSinkWriter implements SinkWriter { + @Override + public long add(String outputElem) { + outputElems.add(outputElem); + return outputElem.length(); + } + + @Override + public void close() { + Assert.assertFalse(closed); + closed = true; + } + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperationTest.java new file mode 100644 index 0000000000000..d0f8e747de7eb --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperationTest.java @@ -0,0 +1,79 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; + +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for FlattenOperation. + */ +@RunWith(JUnit4.class) +public class FlattenOperationTest { + @Test + public void testRunFlattenOperation() throws Exception { + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler( + counterPrefix, counterSet.getAddCounterMutator()); + ExecutorTestUtils.TestReceiver receiver = + new ExecutorTestUtils.TestReceiver(counterSet, counterPrefix); + + FlattenOperation flattenOperation = + new FlattenOperation(receiver, + counterPrefix, counterSet.getAddCounterMutator(), + stateSampler); + + flattenOperation.start(); + + flattenOperation.process("hi"); + flattenOperation.process("there"); + flattenOperation.process(""); + flattenOperation.process("bob"); + + flattenOperation.finish(); + + Assert.assertThat(receiver.outputElems, + CoreMatchers.hasItems("hi", "there", "", "bob")); + + Assert.assertEquals( + new CounterSet( + Counter.longs("test-FlattenOperation-start-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-FlattenOperation-start-msecs")).getAggregate(false)), + Counter.longs("test-FlattenOperation-process-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-FlattenOperation-process-msecs")).getAggregate(false)), + Counter.longs("test-FlattenOperation-finish-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-FlattenOperation-finish-msecs")).getAggregate(false)), + Counter.longs("test_receiver_out-ElementCount", SUM) + .resetToValue(4L), + Counter.longs("test_receiver_out-MeanByteCount", MEAN) + .resetToValue(4, 10L)), + counterSet); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java new file mode 100644 index 0000000000000..27017962ccc5f --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java @@ -0,0 +1,290 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToSourcePosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; + +import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.CounterSet.AddCounterMutator; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils.TestReceiver; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils.TestSource; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Tests for MapTaskExecutor. + */ +@RunWith(JUnit4.class) +public class MapTaskExecutorTest { + static class TestOperation extends Operation { + String label; + List log; + + private static CounterSet counterSet = new CounterSet(); + private static String counterPrefix = "test-"; + private static StateSampler stateSampler = new StateSampler( + counterPrefix, counterSet.getAddCounterMutator()); + + TestOperation(String label, List log) { + super(label, + new OutputReceiver[]{}, + counterPrefix, + counterSet.getAddCounterMutator(), + stateSampler); + this.label = label; + this.log = log; + } + + TestOperation(String outputName, + String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler, + long outputCount) { + super(outputName, new OutputReceiver[]{}, + counterPrefix, addCounterMutator, stateSampler); + addCounterMutator.addCounter( + Counter.longs(outputName + "-ElementCount", SUM) + .resetToValue(outputCount)); + } + + @Override + public void start() throws Exception { + super.start(); + log.add(label + " started"); + } + + @Override + public void finish() throws Exception { + log.add(label + " finished"); + super.finish(); + } + } + + // A mock ReadOperation fed to a MapTaskExecutor in test. + static class TestReadOperation extends ReadOperation { + private ApproximateProgress progress = null; + + TestReadOperation(OutputReceiver outputReceiver, + String counterPrefix, + AddCounterMutator addCounterMutator, + StateSampler stateSampler) { + super(new TestSource(), outputReceiver, + counterPrefix, addCounterMutator, stateSampler); + } + + @Override + public Source.Progress getProgress() { + return cloudProgressToSourceProgress(progress); + } + + @Override + public Source.Position proposeStopPosition( + Source.Progress proposedStopPosition) { + // Fakes the return with the same position as proposed. + return cloudPositionToSourcePosition( + sourceProgressToCloudProgress(proposedStopPosition) + .getPosition()); + } + + public void setProgress(ApproximateProgress progress) { + this.progress = progress; + } + } + + @Test + public void testExecuteMapTaskExecutor() throws Exception { + List log = new ArrayList<>(); + + List operations = Arrays.asList(new Operation[]{ + new TestOperation("o1", log), + new TestOperation("o2", log), + new TestOperation("o3", log)}); + + CounterSet counters = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler( + counterPrefix, counters.getAddCounterMutator()); + MapTaskExecutor executor = + new MapTaskExecutor(operations, counters, stateSampler); + + executor.execute(); + + Assert.assertThat(log, CoreMatchers.hasItems( + "o3 started", + "o2 started", + "o1 started", + "o1 finished", + "o2 finished", + "o3 finished")); + + executor.close(); + } + + @Test + public void testGetOutputCounters() throws Exception { + CounterSet counters = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler( + counterPrefix, counters.getAddCounterMutator()); + List operations = Arrays.asList(new Operation[]{ + new TestOperation( + "o1", counterPrefix, counters.getAddCounterMutator(), + stateSampler, 1), + new TestOperation( + "o2", counterPrefix, counters.getAddCounterMutator(), + stateSampler, 2), + new TestOperation( + "o3", counterPrefix, counters.getAddCounterMutator(), + stateSampler, 3)}); + + MapTaskExecutor executor = + new MapTaskExecutor(operations, counters, stateSampler); + + CounterSet counterSet = executor.getOutputCounters(); + Assert.assertEquals( + new CounterSet( + Counter.longs("o1-ElementCount", SUM).resetToValue(1L), + Counter.longs("test-o1-start-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-o1-start-msecs")).getAggregate(false)), + Counter.longs("test-o1-process-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-o1-process-msecs")).getAggregate(false)), + Counter.longs("test-o1-finish-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-o1-finish-msecs")).getAggregate(false)), + Counter.longs("o2-ElementCount", SUM).resetToValue(2L), + Counter.longs("test-o2-start-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-o2-start-msecs")).getAggregate(false)), + Counter.longs("test-o2-process-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-o2-process-msecs")).getAggregate(false)), + Counter.longs("test-o2-finish-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-o2-finish-msecs")).getAggregate(false)), + Counter.longs("o3-ElementCount", SUM).resetToValue(3L), + Counter.longs("test-o3-start-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-o3-start-msecs")).getAggregate(false)), + Counter.longs("test-o3-process-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-o3-process-msecs")).getAggregate(false)), + Counter.longs("test-o3-finish-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-o3-finish-msecs")).getAggregate(false))), + counterSet); + + executor.close(); + } + + @Test + public void testGetReadOperation() throws Exception { + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler( + counterPrefix, counterSet.getAddCounterMutator()); + // Test MapTaskExecutor without a single operation. + MapTaskExecutor executor = + new MapTaskExecutor(new ArrayList(), + counterSet, stateSampler); + + try { + ReadOperation readOperation = executor.getReadOperation(); + Assert.fail("Expected IllegalStateException."); + } catch (IllegalStateException e) { + // Exception expected + } + + List operations = Arrays.asList(new Operation[]{ + new TestOperation("o1", + counterPrefix, counterSet.getAddCounterMutator(), + stateSampler, 1), + new TestOperation("o2", + counterPrefix, counterSet.getAddCounterMutator(), + stateSampler, 2)}); + // Test MapTaskExecutor without ReadOperation. + executor = new MapTaskExecutor(operations, counterSet, stateSampler); + + try { + ReadOperation readOperation = executor.getReadOperation(); + Assert.fail("Expected IllegalStateException."); + } catch (IllegalStateException e) { + // Exception expected + } + + executor.close(); + + TestReceiver receiver = new TestReceiver(counterSet, counterPrefix); + operations = Arrays.asList(new Operation[]{ + new TestReadOperation( + receiver, counterPrefix, counterSet.getAddCounterMutator(), + stateSampler)}); + executor = new MapTaskExecutor(operations, counterSet, stateSampler); + Assert.assertEquals(operations.get(0), executor.getReadOperation()); + executor.close(); + } + + @Test + public void testGetProgressAndRequestSplit() throws Exception { + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler( + counterPrefix, counterSet.getAddCounterMutator()); + TestReceiver receiver = new TestReceiver(counterSet, counterPrefix); + TestReadOperation operation = + new TestReadOperation(receiver, + counterPrefix, counterSet.getAddCounterMutator(), + stateSampler); + MapTaskExecutor executor = new MapTaskExecutor( + Arrays.asList(new Operation[]{operation}), counterSet, stateSampler); + + operation.setProgress(new ApproximateProgress().setPosition(makePosition(1L))); + Assert.assertEquals( + makePosition(1L), + sourceProgressToCloudProgress(executor.getWorkerProgress()).getPosition()); + Assert.assertEquals( + makePosition(1L), + sourcePositionToCloudPosition( + executor.proposeStopPosition( + cloudProgressToSourceProgress( + new ApproximateProgress().setPosition(makePosition(1L)))))); + + executor.close(); + } + + private com.google.api.services.dataflow.model.Position makePosition(long index) { + com.google.api.services.dataflow.model.Position position = + new com.google.api.services.dataflow.model.Position(); + position.setRecordIndex(index); + return position; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiverTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiverTest.java new file mode 100644 index 0000000000000..08955ac564d7c --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiverTest.java @@ -0,0 +1,135 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; + +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.runners.worker.MapTaskExecutorFactory.ElementByteSizeObservableCoder; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.CounterTestUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils.TestReceiver; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for OutputReceiver. + */ +@RunWith(JUnit4.class) +public class OutputReceiverTest { + // We test OutputReceiver where every element is sampled. + static class TestOutputReceiver extends OutputReceiver { + public TestOutputReceiver() { + this(new CounterSet()); + } + + public TestOutputReceiver(CounterSet counters) { + super("output_name", + new ElementByteSizeObservableCoder(StringUtf8Coder.of()), + "test-", + counters.getAddCounterMutator()); + } + + @Override + protected boolean sampleElement() { + return true; + } + } + + @Test + public void testEmptyOutputReceiver() throws Exception { + TestOutputReceiver fanOut = new TestOutputReceiver(); + fanOut.process("hi"); + fanOut.process("bob"); + + Assert.assertEquals("output_name", fanOut.getName()); + Assert.assertEquals( + 2, + (long) CounterTestUtils.getTotalAggregate(fanOut.getElementCount())); + Assert.assertEquals( + 5, + (long) CounterTestUtils.getTotalAggregate(fanOut.getMeanByteCount())); + Assert.assertEquals( + 2, + (long) CounterTestUtils.getTotalCount(fanOut.getMeanByteCount())); + } + + @Test + public void testMultipleOutputReceiver() throws Exception { + TestOutputReceiver fanOut = new TestOutputReceiver(); + + CounterSet counters = new CounterSet(); + String counterPrefix = "test-"; + + TestReceiver receiver1 = new TestReceiver(counters, counterPrefix); + fanOut.addOutput(receiver1); + + TestReceiver receiver2 = new TestReceiver(counters, counterPrefix); + fanOut.addOutput(receiver2); + + fanOut.process("hi"); + fanOut.process("bob"); + + Assert.assertEquals("output_name", fanOut.getName()); + Assert.assertEquals( + 2, + (long) CounterTestUtils.getTotalAggregate(fanOut.getElementCount())); + Assert.assertEquals( + 5, + (long) CounterTestUtils.getTotalAggregate(fanOut.getMeanByteCount())); + Assert.assertEquals( + 2, + (long) CounterTestUtils.getTotalCount(fanOut.getMeanByteCount())); + Assert.assertThat(receiver1.outputElems, + CoreMatchers.hasItems("hi", "bob")); + Assert.assertThat(receiver2.outputElems, + CoreMatchers.hasItems("hi", "bob")); + } + + @Test(expected = ClassCastException.class) + public void testIncorrectType() throws Exception { + TestOutputReceiver fanOut = new TestOutputReceiver(); + fanOut.process(5); + } + + @Test(expected = CoderException.class) + public void testNullArgument() throws Exception { + TestOutputReceiver fanOut = new TestOutputReceiver(); + fanOut.process(null); + } + + @Test + public void testAddingCountersIntoCounterSet() throws Exception { + CounterSet counters = new CounterSet(); + TestOutputReceiver receiver = new TestOutputReceiver(counters); + + Assert.assertEquals( + new CounterSet( + Counter.longs("output_name-ElementCount", SUM) + .resetToValue(0L), + Counter.longs("output_name-MeanByteCount", MEAN) + .resetToValue(0, 0L)), + counters); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperationTest.java new file mode 100644 index 0000000000000..b08266cbb4d82 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperationTest.java @@ -0,0 +1,116 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; + +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for ParDoOperation. + */ +@RunWith(JUnit4.class) +public class ParDoOperationTest { + static class TestParDoFn extends ParDoFn { + final OutputReceiver outputReceiver; + + public TestParDoFn(OutputReceiver outputReceiver) { + this.outputReceiver = outputReceiver; + } + + @Override + public void startBundle(final Receiver... receivers) throws Exception { + if (receivers.length != 1) { + throw new AssertionError( + "unexpected number of receivers for DoFn"); + } + + outputReceiver.process("x-start"); + } + + @Override + public void processElement(Object elem) throws Exception { + outputReceiver.process("y-" + elem); + } + + @Override + public void finishBundle() throws Exception { + outputReceiver.process("z-finish"); + } + } + + @Test + public void testRunParDoOperation() throws Exception { + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler( + counterPrefix, counterSet.getAddCounterMutator()); + ExecutorTestUtils.TestReceiver receiver = + new ExecutorTestUtils.TestReceiver(counterSet); + + ParDoOperation parDoOperation = + new ParDoOperation( + "ParDoOperation", + new TestParDoFn(receiver), + new OutputReceiver[]{ receiver }, + counterPrefix, + counterSet.getAddCounterMutator(), + stateSampler); + + parDoOperation.start(); + + parDoOperation.process("hi"); + parDoOperation.process("there"); + parDoOperation.process(""); + parDoOperation.process("bob"); + + parDoOperation.finish(); + + Assert.assertThat( + receiver.outputElems, + CoreMatchers.hasItems( + "x-start", "y-hi", "y-there", "y-", "y-bob", "z-finish")); + + Assert.assertEquals( + new CounterSet( + Counter.longs("test-ParDoOperation-start-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-ParDoOperation-start-msecs")).getAggregate(false)), + Counter.longs("test-ParDoOperation-process-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-ParDoOperation-process-msecs")).getAggregate(false)), + Counter.longs("test-ParDoOperation-finish-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-ParDoOperation-finish-msecs")).getAggregate(false)), + Counter.longs("test_receiver_out-ElementCount", SUM) + .resetToValue(6L), + Counter.longs("test_receiver_out-MeanByteCount", MEAN) + .resetToValue(6, 33L)), + counterSet); + } + + // TODO: Test side inputs. + // TODO: Test side outputs. +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperationTest.java new file mode 100644 index 0000000000000..620ac0c89894f --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperationTest.java @@ -0,0 +1,397 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; +import static org.hamcrest.Matchers.anyOf; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.isIn; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.runners.worker.MapTaskExecutorFactory.CoderGroupingKeyCreator; +import com.google.cloud.dataflow.sdk.runners.worker.MapTaskExecutorFactory.CoderSizeEstimator; +import com.google.cloud.dataflow.sdk.runners.worker.MapTaskExecutorFactory.ElementByteSizeObservableCoder; +import com.google.cloud.dataflow.sdk.runners.worker.MapTaskExecutorFactory.PairInfo; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils.TestReceiver; +import com.google.cloud.dataflow.sdk.util.common.worker.PartialGroupByKeyOperation.BufferingGroupingTable; +import com.google.cloud.dataflow.sdk.util.common.worker.PartialGroupByKeyOperation.Combiner; +import com.google.cloud.dataflow.sdk.util.common.worker.PartialGroupByKeyOperation.CombiningGroupingTable; +import com.google.cloud.dataflow.sdk.util.common.worker.PartialGroupByKeyOperation.GroupingKeyCreator; +import com.google.cloud.dataflow.sdk.util.common.worker.PartialGroupByKeyOperation.SamplingSizeEstimator; +import com.google.cloud.dataflow.sdk.util.common.worker.PartialGroupByKeyOperation.SizeEstimator; +import com.google.cloud.dataflow.sdk.values.KV; + +import org.hamcrest.Description; +import org.hamcrest.TypeSafeDiagnosingMatcher; +import org.hamcrest.collection.IsIterableContainingInAnyOrder; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; + +/** + * Tests for PartialGroupByKeyOperation. + */ +@RunWith(JUnit4.class) +public class PartialGroupByKeyOperationTest { + @Test + public void testRunPartialGroupByKeyOperation() throws Exception { + Coder keyCoder = StringUtf8Coder.of(); + Coder valueCoder = BigEndianIntegerCoder.of(); + + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler( + counterPrefix, counterSet.getAddCounterMutator()); + TestReceiver receiver = + new TestReceiver( + new ElementByteSizeObservableCoder( + WindowedValue.getValueOnlyCoder( + KvCoder.of(keyCoder, IterableCoder.of(valueCoder)))), + counterSet, counterPrefix); + + PartialGroupByKeyOperation pgbkOperation = + new PartialGroupByKeyOperation(new CoderGroupingKeyCreator(keyCoder), + new CoderSizeEstimator(keyCoder), + new CoderSizeEstimator(valueCoder), + PairInfo.create(), + receiver, + counterPrefix, + counterSet.getAddCounterMutator(), + stateSampler); + + pgbkOperation.start(); + + pgbkOperation.process(WindowedValue.valueInEmptyWindows(KV.of("hi", 4))); + pgbkOperation.process(WindowedValue.valueInEmptyWindows(KV.of("there", 5))); + pgbkOperation.process(WindowedValue.valueInEmptyWindows(KV.of("hi", 6))); + pgbkOperation.process(WindowedValue.valueInEmptyWindows(KV.of("joe", 7))); + pgbkOperation.process(WindowedValue.valueInEmptyWindows(KV.of("there", 8))); + pgbkOperation.process(WindowedValue.valueInEmptyWindows(KV.of("hi", 9))); + + pgbkOperation.finish(); + + assertThat(receiver.outputElems, + IsIterableContainingInAnyOrder.containsInAnyOrder( + WindowedValue.valueInEmptyWindows(KV.of("hi", Arrays.asList(4, 6, 9))), + WindowedValue.valueInEmptyWindows(KV.of("there", Arrays.asList(5, 8))), + WindowedValue.valueInEmptyWindows(KV.of("joe", Arrays.asList(7))))); + + // Exact counter values depend on size of encoded data. If encoding + // changes, then these expected counters should change to match. + assertEquals( + new CounterSet( + Counter.longs("test-PartialGroupByKeyOperation-start-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-PartialGroupByKeyOperation-start-msecs")).getAggregate(false)), + Counter.longs("test-PartialGroupByKeyOperation-process-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-PartialGroupByKeyOperation-process-msecs")).getAggregate(false)), + Counter.longs("test-PartialGroupByKeyOperation-finish-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-PartialGroupByKeyOperation-finish-msecs")).getAggregate(false)), + Counter.longs("test_receiver_out-ElementCount", SUM) + .resetToValue(3L), + Counter.longs("test_receiver_out-MeanByteCount", MEAN) + .resetToValue(3, 49L)), + counterSet); + } + + // TODO: Add tests about early flushing when the table fills. + + //////////////////////////////////////////////////////////////////////////// + // Tests for PartialGroupByKey internals. + + /** + * Return the key as its grouping key. + */ + public static class IdentityGroupingKeyCreator implements GroupingKeyCreator { + @Override + public Object createGroupingKey(Object key) { + return key; + } + } + + /** + * "Estimate" the size of longs by looking at their value. + */ + private static class IdentitySizeEstimator implements SizeEstimator { + public int calls = 0; + @Override + public long estimateSize(Long element) { + calls++; + return element; + } + } + + /** + * "Estimate" the size of strings by taking the tenth power of their length. + */ + private static class StringPowerSizeEstimator implements SizeEstimator { + @Override + public long estimateSize(String element) { + return (long) Math.pow(10, element.length()); + } + } + + @Test + public void testBufferingGroupingTable() throws Exception { + BufferingGroupingTable table = + new BufferingGroupingTable<>( + 1000, new IdentityGroupingKeyCreator(), PairInfo.create(), + new StringPowerSizeEstimator(), new StringPowerSizeEstimator()); + TestReceiver receiver = new TestReceiver( + WindowedValue.getValueOnlyCoder( + KvCoder.of(StringUtf8Coder.of(), IterableCoder.of(StringUtf8Coder.of())))); + + table.put("A", "a", receiver); + table.put("B", "b1", receiver); + table.put("B", "b2", receiver); + table.put("C", "c", receiver); + assertThat(unwindowed(receiver.outputElems), empty()); + + table.put("C", "cccc", receiver); + assertThat(unwindowed(receiver.outputElems), + hasItem((Object) KV.of("C", Arrays.asList("c", "cccc")))); + + table.put("DDDD", "d", receiver); + assertThat(unwindowed(receiver.outputElems), + hasItem((Object) KV.of("DDDD", Arrays.asList("d")))); + + table.flush(receiver); + assertThat(unwindowed(receiver.outputElems), + IsIterableContainingInAnyOrder.containsInAnyOrder( + KV.of("A", Arrays.asList("a")), + KV.of("B", Arrays.asList("b1", "b2")), + KV.of("C", Arrays.asList("c", "cccc")), + KV.of("DDDD", Arrays.asList("d")))); + } + + @Test + public void testCombiningGroupingTable() throws Exception { + Combiner summingCombineFn = + new Combiner() { + public Long createAccumulator(Object key) { + return 0L; + } + public Long add(Object key, Long accumulator, Integer value) { + return accumulator + value; + } + public Long merge(Object key, Iterable accumulators) { + long sum = 0; + for (Long part : accumulators) { sum += part; } + return sum; + } + public Long extract(Object key, Long accumulator) { + return accumulator; + } + }; + + CombiningGroupingTable table = + new CombiningGroupingTable( + 1000, new IdentityGroupingKeyCreator(), PairInfo.create(), + summingCombineFn, + new StringPowerSizeEstimator(), new IdentitySizeEstimator()); + + TestReceiver receiver = new TestReceiver( + WindowedValue.getValueOnlyCoder( + KvCoder.of(StringUtf8Coder.of(), BigEndianLongCoder.of()))); + + table.put("A", 1, receiver); + table.put("B", 2, receiver); + table.put("B", 3, receiver); + table.put("C", 4, receiver); + assertThat(unwindowed(receiver.outputElems), empty()); + + table.put("C", 5000, receiver); + assertThat(unwindowed(receiver.outputElems), hasItem((Object) KV.of("C", 5004L))); + + table.put("DDDD", 6, receiver); + assertThat(unwindowed(receiver.outputElems), hasItem((Object) KV.of("DDDD", 6L))); + + table.flush(receiver); + assertThat(unwindowed(receiver.outputElems), + IsIterableContainingInAnyOrder.containsInAnyOrder( + KV.of("A", 1L), + KV.of("B", 2L + 3), + KV.of("C", 5000L + 4), + KV.of("DDDD", 6L))); + } + + private List unwindowed(Iterable windowed) { + List unwindowed = new ArrayList<>(); + for (Object withWindow : windowed) { + unwindowed.add(((WindowedValue) withWindow).getValue()); + } + return unwindowed; + } + + + //////////////////////////////////////////////////////////////////////////// + // Tests for the sampling size estimator. + + @Test + public void testSampleFlatSizes() throws Exception { + IdentitySizeEstimator underlying = new IdentitySizeEstimator(); + SizeEstimator estimator = + new SamplingSizeEstimator(underlying, 0.05, 1.0, 10, new Random(1)); + // First 10 elements are always sampled. + for (int k = 0; k < 10; k++) { + assertEquals(100, estimator.estimateSize(100L)); + assertEquals(k + 1, underlying.calls); + } + // Next 10 are sometimes sampled. + for (int k = 10; k < 20; k++) { + assertEquals(100, estimator.estimateSize(100L)); + } + assertThat(underlying.calls, between(11, 19)); + int initialCalls = underlying.calls; + // Next 1000 are sampled at about 5%. + for (int k = 20; k < 1020; k++) { + assertEquals(100, estimator.estimateSize(100L)); + } + assertThat(underlying.calls - initialCalls, between(40, 60)); + } + + @Test + public void testSampleBoringSizes() throws Exception { + IdentitySizeEstimator underlying = new IdentitySizeEstimator(); + SizeEstimator estimator = + new SamplingSizeEstimator(underlying, 0.05, 1.0, 10, new Random(1)); + // First 10 elements are always sampled. + for (int k = 0; k < 10; k += 2) { + assertEquals(100, estimator.estimateSize(100L)); + assertEquals(102, estimator.estimateSize(102L)); + assertEquals(k + 2, underlying.calls); + } + // Next 10 are sometimes sampled. + for (int k = 10; k < 20; k += 2) { + assertThat(estimator.estimateSize(100L), between(100L, 102L)); + assertThat(estimator.estimateSize(102L), between(100L, 102L)); + } + assertThat(underlying.calls, between(11, 19)); + int initialCalls = underlying.calls; + // Next 1000 are sampled at about 5%. + for (int k = 20; k < 1020; k += 2) { + assertThat(estimator.estimateSize(100L), between(100L, 102L)); + assertThat(estimator.estimateSize(102L), between(100L, 102L)); + } + assertThat(underlying.calls - initialCalls, between(40, 60)); + } + + @Test + public void testSampleHighVarianceSizes() throws Exception { + // The largest element is much larger than the average. + List sizes = Arrays.asList(1L, 10L, 100L, 1000L); + IdentitySizeEstimator underlying = new IdentitySizeEstimator(); + SizeEstimator estimator = + new SamplingSizeEstimator(underlying, 0.1, 0.2, 10, new Random(1)); + // First 10 elements are always sampled. + for (int k = 0; k < 10; k++) { + long size = sizes.get(k % sizes.size()); + assertEquals(size, estimator.estimateSize(size)); + assertEquals(k + 1, underlying.calls); + } + // We're still not out of the woods; sample every element. + for (int k = 10; k < 20; k++) { + long size = sizes.get(k % sizes.size()); + assertEquals(size, estimator.estimateSize(size)); + assertEquals(k + 1, underlying.calls); + } + // Sample some more to let things settle down. + for (int k = 20; k < 500; k++) { + estimator.estimateSize(sizes.get(k % sizes.size())); + } + // Next 1000 are sampled at about 20% (maxSampleRate). + int initialCalls = underlying.calls; + for (int k = 500; k < 1500; k++) { + long size = sizes.get(k % sizes.size()); + assertThat(estimator.estimateSize(size), + anyOf(isIn(sizes), between(250L, 350L))); + } + assertThat(underlying.calls - initialCalls, between(180, 220)); + // Sample some more to let things settle down. + for (int k = 1500; k < 3000; k++) { + estimator.estimateSize(sizes.get(k % sizes.size())); + } + // Next 1000 are sampled at about 10% (minSampleRate). + initialCalls = underlying.calls; + for (int k = 3000; k < 4000; k++) { + long size = sizes.get(k % sizes.size()); + assertThat(estimator.estimateSize(size), + anyOf(isIn(sizes), between(250L, 350L))); + } + assertThat(underlying.calls - initialCalls, between(90, 110)); + } + + @Test + public void testSampleChangingSizes() throws Exception { + IdentitySizeEstimator underlying = new IdentitySizeEstimator(); + SizeEstimator estimator = + new SamplingSizeEstimator(underlying, 0.05, 1.0, 10, new Random(1)); + // First 10 elements are always sampled. + for (int k = 0; k < 10; k++) { + assertEquals(100, estimator.estimateSize(100L)); + assertEquals(k + 1, underlying.calls); + } + // Next 10 are sometimes sampled. + for (int k = 10; k < 20; k++) { + assertEquals(100, estimator.estimateSize(100L)); + } + assertThat(underlying.calls, between(11, 19)); + int initialCalls = underlying.calls; + // Next 1000 are sampled at about 5%. + for (int k = 20; k < 1020; k++) { + assertEquals(100, estimator.estimateSize(100L)); + } + assertThat(underlying.calls - initialCalls, between(40, 60)); + // Inject a big element until it is sampled. + while (estimator.estimateSize(1000000L) == 100) { } + // Check that we have started sampling more regularly again. + assertEquals(99, estimator.estimateSize(99L)); + } + + private static > TypeSafeDiagnosingMatcher + between(final T min, final T max) { + return new TypeSafeDiagnosingMatcher() { + @Override + public void describeTo(Description description) { + description.appendText("is between " + min + " and " + max); + } + @Override + protected boolean matchesSafely(T item, Description mismatchDescription) { + return min.compareTo(item) <= 0 && item.compareTo(max) <= 0; + } + }; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java new file mode 100644 index 0000000000000..b3e29f8e5cf55 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java @@ -0,0 +1,303 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN; +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.everyItem; +import static org.hamcrest.collection.IsIterableContainingInOrder.contains; + +import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.Position; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils.TestReceiver; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils.TestSource; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +/** + * Tests for ReadOperation. + */ +@RunWith(JUnit4.class) +public class ReadOperationTest { + private static final long ITERATIONS = 3L; + + /** + * The test Source for testing updating stop position and progress report. + * The number of read iterations is controlled by ITERATIONS. + */ + static class TestTextSource extends Source { + @Override + public SourceIterator iterator() { + return new TestTextSourceIterator(); + } + + class TestTextSourceIterator extends AbstractSourceIterator { + long offset = 0L; + List proposedPositions = + new ArrayList<>(); + + @Override + public boolean hasNext() { + return offset < ITERATIONS; + } + + @Override + public String next() { + if (hasNext()) { + offset++; + return "hi"; + } else { + throw new AssertionError("No next Element."); + } + } + + @Override + public Progress getProgress() { + com.google.api.services.dataflow.model.Position currentPosition = + new com.google.api.services.dataflow.model.Position(); + currentPosition.setByteOffset(offset); + + ApproximateProgress progress = new ApproximateProgress(); + progress.setPosition(currentPosition); + + return cloudProgressToSourceProgress(progress); + } + + @Override + public Position updateStopPosition(Progress proposedStopPosition) { + proposedPositions.add(sourceProgressToCloudProgress(proposedStopPosition).getPosition()); + // Actually no update happens, returns null. + return null; + } + } + } + + /** + * The OutputReceiver for testing updating stop position and progress report. + * The offset of the Source (iterator) will be advanced each time this + * Receiver processes a record. + */ + static class TestTextReceiver extends OutputReceiver { + ReadOperation readOperation = null; + com.google.api.services.dataflow.model.Position proposedStopPosition = null; + List progresses = new ArrayList<>(); + + public TestTextReceiver(CounterSet counterSet, String counterPrefix) { + super("test_receiver_out", counterPrefix, counterSet.getAddCounterMutator()); + } + + public void setReadOperation(ReadOperation readOp) { + this.readOperation = readOp; + } + + public void setProposedStopPosition(com.google.api.services.dataflow.model.Position position) { + this.proposedStopPosition = position; + } + + @Override + public void process(Object outputElem) throws Exception { + // Calls getProgress() and proposeStopPosition() in each iteration. + progresses.add(sourceProgressToCloudProgress(readOperation.getProgress())); + // We expect that call to proposeStopPosition is a no-op that does not + // update the stop position for every iteration. We will verify it is + // delegated to SourceIterator after ReadOperation finishes. + Assert.assertNull( + readOperation.proposeStopPosition( + cloudProgressToSourceProgress(makeApproximateProgress(proposedStopPosition)))); + } + } + + @Test + public void testRunReadOperation() throws Exception { + TestSource source = new TestSource(); + source.addInput("hi", "there", "", "bob"); + + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); + TestReceiver receiver = new TestReceiver(counterSet, counterPrefix); + + ReadOperation readOperation = new ReadOperation( + source, receiver, counterPrefix, counterSet.getAddCounterMutator(), stateSampler); + + readOperation.start(); + readOperation.finish(); + + Assert.assertThat( + receiver.outputElems, CoreMatchers.hasItems("hi", "there", "", "bob")); + + Assert.assertEquals( + new CounterSet( + Counter.longs("ReadOperation-ByteCount", SUM).resetToValue(2L + 5 + 0 + 3), + Counter.longs("test_receiver_out-ElementCount", SUM).resetToValue(4L), + Counter.longs("test_receiver_out-MeanByteCount", MEAN).resetToValue(4, 10L), + Counter.longs("test-ReadOperation-start-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-ReadOperation-start-msecs")).getAggregate(false)), + Counter.longs("test-ReadOperation-read-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-ReadOperation-read-msecs")).getAggregate(false)), + Counter.longs("test-ReadOperation-process-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-ReadOperation-process-msecs")).getAggregate(false)), + Counter.longs("test-ReadOperation-finish-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-ReadOperation-finish-msecs")).getAggregate(false))), + counterSet); + } + + @Test + public void testGetProgressAndProposeStopPosition() throws Exception { + TestTextSource testSource = new TestTextSource(); + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); + TestTextReceiver receiver = new TestTextReceiver(counterSet, counterPrefix); + ReadOperation readOperation = new ReadOperation( + testSource, receiver, counterPrefix, counterSet.getAddCounterMutator(), stateSampler); + readOperation.setProgressUpdatePeriodMs(0); + receiver.setReadOperation(readOperation); + + Position proposedStopPosition = makePosition(3L); + receiver.setProposedStopPosition(proposedStopPosition); + + Assert.assertNull(readOperation.getProgress()); + Assert.assertNull(readOperation.proposeStopPosition( + cloudProgressToSourceProgress( + makeApproximateProgress(proposedStopPosition)))); + + readOperation.start(); + readOperation.finish(); + + TestTextSource.TestTextSourceIterator testIterator = + (TestTextSource.TestTextSourceIterator) readOperation.sourceIterator; + + Assert.assertEquals(sourceProgressToCloudProgress(testIterator.getProgress()), + sourceProgressToCloudProgress(readOperation.getProgress())); + Assert.assertEquals(sourcePositionToCloudPosition(testIterator.updateStopPosition( + cloudProgressToSourceProgress( + makeApproximateProgress(proposedStopPosition)))), + sourcePositionToCloudPosition(readOperation.proposeStopPosition( + cloudProgressToSourceProgress( + makeApproximateProgress(proposedStopPosition))))); + + // Verifies progress report and stop position updates. + Assert.assertEquals(testIterator.proposedPositions.size(), ITERATIONS + 2); + Assert.assertThat( + testIterator.proposedPositions, everyItem(equalTo(makePosition(3L)))); + Assert.assertThat( + receiver.progresses, contains(makeApproximateProgress(1L), makeApproximateProgress(2L), + makeApproximateProgress(3L))); + } + + @Test + public void testGetProgressDoesNotBlock() throws Exception { + final BlockingQueue queue = new LinkedBlockingQueue<>(); + final Source.SourceIterator iterator = new Source.AbstractSourceIterator() { + private int itemsReturned = 0; + + @Override + public boolean hasNext() throws IOException { + return itemsReturned < 5; + } + + @Override + public Integer next() throws IOException { + ++itemsReturned; + try { + return queue.take(); + } catch (InterruptedException e) { + throw new NoSuchElementException("interrupted"); + } + } + + @Override + public Source.Progress getProgress() { + return cloudProgressToSourceProgress(new ApproximateProgress().setPosition( + new Position().setRecordIndex((long) itemsReturned))); + } + }; + + Source source = new Source() { + @Override + public SourceIterator iterator() throws IOException { + return iterator; + } + }; + + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); + TestTextReceiver receiver = new TestTextReceiver(counterSet, counterPrefix); + final ReadOperation readOperation = new ReadOperation( + source, receiver, counterPrefix, counterSet.getAddCounterMutator(), stateSampler); + // Update progress not continuously, but so that it's never more than 1 record stale. + readOperation.setProgressUpdatePeriodMs(150); + receiver.setReadOperation(readOperation); + + new Thread() { + @Override + public void run() { + try { + readOperation.start(); + readOperation.finish(); + } catch (Exception e) { + e.printStackTrace(); + } + } + }.start(); + + for (int i = 0; i < 5; ++i) { + Thread.sleep(100); // Wait for the operation to start and block. + // Ensure that getProgress() doesn't block. + ApproximateProgress progress = sourceProgressToCloudProgress(readOperation.getProgress()); + long observedIndex = progress.getPosition().getRecordIndex().longValue(); + Assert.assertTrue("Actual: " + observedIndex, i == observedIndex || i == observedIndex + 1); + queue.offer(i); + } + } + + private static Position makePosition(long offset) { + return new Position().setByteOffset(offset); + } + + private static ApproximateProgress makeApproximateProgress(long offset) { + return makeApproximateProgress(makePosition(offset)); + } + + private static ApproximateProgress makeApproximateProgress( + com.google.api.services.dataflow.model.Position position) { + return new ApproximateProgress().setPosition(position); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleEntryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleEntryTest.java new file mode 100644 index 0000000000000..10e3b4da63f4c --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ShuffleEntryTest.java @@ -0,0 +1,145 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link ShuffleEntry}. */ +@RunWith(JUnit4.class) +public class ShuffleEntryTest { + private static final byte[] KEY = {0xA}; + private static final byte[] SKEY = {0xB}; + private static final byte[] VALUE = {0xC}; + + @Test + public void accessors() { + ShuffleEntry entry = new ShuffleEntry(KEY, SKEY, VALUE); + assertThat(entry.getKey(), equalTo(KEY)); + assertThat(entry.getSecondaryKey(), equalTo(SKEY)); + assertThat(entry.getValue(), equalTo(VALUE)); + } + + @Test + public void equalsToItself() { + ShuffleEntry entry = new ShuffleEntry(KEY, SKEY, VALUE); + assertTrue(entry.equals(entry)); + } + + @Test + public void equalsForEqualEntries() { + ShuffleEntry entry0 = new ShuffleEntry(KEY, SKEY, VALUE); + ShuffleEntry entry1 = new ShuffleEntry( + KEY.clone(), SKEY.clone(), VALUE.clone()); + + assertTrue(entry0.equals(entry1)); + assertTrue(entry1.equals(entry0)); + assertEquals(entry0.hashCode(), entry1.hashCode()); + } + + @Test + public void equalsForEqualNullEntries() { + ShuffleEntry entry0 = new ShuffleEntry(null, null, null); + ShuffleEntry entry1 = new ShuffleEntry(null, null, null); + + assertTrue(entry0.equals(entry1)); + assertTrue(entry1.equals(entry0)); + assertEquals(entry0.hashCode(), entry1.hashCode()); + } + + @Test + public void notEqualsWhenKeysDiffer() { + final byte[] otherKey = {0x1}; + ShuffleEntry entry0 = new ShuffleEntry(KEY, SKEY, VALUE); + ShuffleEntry entry1 = new ShuffleEntry(otherKey, SKEY, VALUE); + + assertFalse(entry0.equals(entry1)); + assertFalse(entry1.equals(entry0)); + assertThat(entry0.hashCode(), not(equalTo(entry1.hashCode()))); + } + + @Test + public void notEqualsWhenKeysDifferOneNull() { + ShuffleEntry entry0 = new ShuffleEntry(KEY, SKEY, VALUE); + ShuffleEntry entry1 = new ShuffleEntry(null, SKEY, VALUE); + + assertFalse(entry0.equals(entry1)); + assertFalse(entry1.equals(entry0)); + assertThat(entry0.hashCode(), not(equalTo(entry1.hashCode()))); + } + + @Test + public void notEqualsWhenSecondaryKeysDiffer() { + final byte[] otherSKey = {0x2}; + ShuffleEntry entry0 = new ShuffleEntry(KEY, SKEY, VALUE); + ShuffleEntry entry1 = new ShuffleEntry(KEY, otherSKey, VALUE); + + assertFalse(entry0.equals(entry1)); + assertFalse(entry1.equals(entry0)); + assertThat(entry0.hashCode(), not(equalTo(entry1.hashCode()))); + } + + @Test + public void notEqualsWhenSecondaryKeysDifferOneNull() { + ShuffleEntry entry0 = new ShuffleEntry(KEY, SKEY, VALUE); + ShuffleEntry entry1 = new ShuffleEntry(KEY, null, VALUE); + + assertFalse(entry0.equals(entry1)); + assertFalse(entry1.equals(entry0)); + assertThat(entry0.hashCode(), not(equalTo(entry1.hashCode()))); + } + + @Test + public void notEqualsWhenValuesDiffer() { + final byte[] otherValue = {0x2}; + ShuffleEntry entry0 = new ShuffleEntry(KEY, SKEY, VALUE); + ShuffleEntry entry1 = new ShuffleEntry(KEY, SKEY, otherValue); + + assertFalse(entry0.equals(entry1)); + assertFalse(entry1.equals(entry0)); + assertThat(entry0.hashCode(), not(equalTo(entry1.hashCode()))); + } + + @Test + public void notEqualsWhenValuesDifferOneNull() { + ShuffleEntry entry0 = new ShuffleEntry(KEY, SKEY, VALUE); + ShuffleEntry entry1 = new ShuffleEntry(KEY, SKEY, null); + + assertFalse(entry0.equals(entry1)); + assertFalse(entry1.equals(entry0)); + assertThat(entry0.hashCode(), not(equalTo(entry1.hashCode()))); + } + + @Test + public void emptyNotTheSameAsNull() { + final byte[] empty = {}; + ShuffleEntry entry0 = new ShuffleEntry(null, null, null); + ShuffleEntry entry1 = new ShuffleEntry(empty, empty, empty); + + assertFalse(entry0.equals(entry1)); + assertFalse(entry1.equals(entry0)); + assertThat(entry0.hashCode(), not(equalTo(entry1.hashCode()))); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java new file mode 100644 index 0000000000000..d350db1798bfe --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java @@ -0,0 +1,139 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.AbstractMap.SimpleEntry; +import java.util.Map; + +/** + * Unit tests for the {@link Counter} API. + */ +@RunWith(JUnit4.class) +public class StateSamplerTest { + + @Test + public void basicTest() throws InterruptedException { + CounterSet counters = new CounterSet(); + long periodMs = 50; + StateSampler stateSampler = new StateSampler("test-", + counters.getAddCounterMutator(), periodMs); + + int state1 = stateSampler.stateForName("1"); + int state2 = stateSampler.stateForName("2"); + + assertEquals(new SimpleEntry<>("", 0L), + stateSampler.getCurrentStateAndDuration()); + + try (StateSampler.ScopedState s1 = + stateSampler.scopedState(state1)) { + Thread.sleep(2 * periodMs); + } + + try (StateSampler.ScopedState s2 = + stateSampler.scopedState(state2)) { + Thread.sleep(3 * periodMs); + } + + long s1 = stateSampler.getStateDuration(state1); + long s2 = stateSampler.getStateDuration(state2); + + System.out.println("basic s1: " + s1); + System.out.println("basic s2: " + s2); + + long toleranceMs = periodMs; + assertTrue(s1 + s2 >= 4 * periodMs - toleranceMs); + assertTrue(s1 + s2 <= 10 * periodMs + toleranceMs); + } + + @Test + public void nestingTest() throws InterruptedException { + CounterSet counters = new CounterSet(); + long periodMs = 50; + StateSampler stateSampler = new StateSampler("test-", + counters.getAddCounterMutator(), periodMs); + + int state1 = stateSampler.stateForName("1"); + int state2 = stateSampler.stateForName("2"); + int state3 = stateSampler.stateForName("3"); + + assertEquals(new SimpleEntry<>("", 0L), + stateSampler.getCurrentStateAndDuration()); + + try (StateSampler.ScopedState s1 = + stateSampler.scopedState(state1)) { + Thread.sleep(2 * periodMs); + + try (StateSampler.ScopedState s2 = + stateSampler.scopedState(state2)) { + Thread.sleep(2 * periodMs); + + try (StateSampler.ScopedState s3 = + stateSampler.scopedState(state3)) { + Thread.sleep(2 * periodMs); + } + Thread.sleep(periodMs); + } + Thread.sleep(periodMs); + } + + long s1 = stateSampler.getStateDuration(state1); + long s2 = stateSampler.getStateDuration(state2); + long s3 = stateSampler.getStateDuration(state3); + + System.out.println("s1: " + s1); + System.out.println("s2: " + s2); + System.out.println("s3: " + s3); + + long toleranceMs = periodMs; + assertTrue(s1 + s2 + s3 >= 4 * periodMs - toleranceMs); + assertTrue(s1 + s2 + s3 <= 16 * periodMs + toleranceMs); + } + + @Test + public void nonScopedTest() throws InterruptedException { + CounterSet counters = new CounterSet(); + long periodMs = 50; + StateSampler stateSampler = new StateSampler("test-", + counters.getAddCounterMutator(), periodMs); + + int state1 = stateSampler.stateForName("1"); + int previousState = stateSampler.setState(state1); + Thread.sleep(2 * periodMs); + Map.Entry currentStateAndDuration = + stateSampler.getCurrentStateAndDuration(); + stateSampler.setState(previousState); + assertEquals("test-1-msecs", currentStateAndDuration.getKey()); + long tolerance = periodMs; + long s = currentStateAndDuration.getValue(); + System.out.println("s: " + s); + assertTrue(s >= periodMs - tolerance); + assertTrue(s <= 4 * periodMs + tolerance); + + assertTrue(stateSampler.getCurrentStateAndDuration() + .getKey().isEmpty()); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutorTest.java new file mode 100644 index 0000000000000..ecce00d68b76c --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutorTest.java @@ -0,0 +1,58 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; + +import com.google.cloud.dataflow.sdk.util.common.Metric; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Collection; + +/** + * Unit tests for {@link WorkExecutor}. + */ +@RunWith(JUnit4.class) +public class WorkExecutorTest { + private WorkExecutor mapWorker; + private WorkExecutor seqMapWorker; + + @Before + public void setUp() { + mapWorker = new MapTaskExecutor(null, null, null); + } + + @Test + public void testMapTaskGetOutputMetrics() { + Collection> metrics = mapWorker.getOutputMetrics(); + verifyOutputMetrics(metrics); + } + + private void verifyOutputMetrics(Collection> metrics) { + Collection metricNames = new ArrayList<>(); + for (Metric metric : metrics) { + metricNames.add(metric.getName()); + } + Assert.assertThat(metricNames, containsInAnyOrder("CPU")); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperationTest.java new file mode 100644 index 0000000000000..6b51bc603531f --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperationTest.java @@ -0,0 +1,73 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common.worker; + +import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; + +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for WriteOperation. + */ +@RunWith(JUnit4.class) +public class WriteOperationTest { + @Test + public void testRunWriteOperation() throws Exception { + ExecutorTestUtils.TestSink sink = new ExecutorTestUtils.TestSink(); + CounterSet counterSet = new CounterSet(); + String counterPrefix = "test-"; + StateSampler stateSampler = new StateSampler( + counterPrefix, counterSet.getAddCounterMutator()); + + WriteOperation writeOperation = new WriteOperation( + sink, counterPrefix, counterSet.getAddCounterMutator(), stateSampler); + + writeOperation.start(); + + writeOperation.process("hi"); + writeOperation.process("there"); + writeOperation.process(""); + writeOperation.process("bob"); + + writeOperation.finish(); + + Assert.assertThat(sink.outputElems, + CoreMatchers.hasItems("hi", "there", "", "bob")); + + Assert.assertEquals( + new CounterSet( + Counter.longs("WriteOperation-ByteCount", SUM) + .resetToValue(2L + 5 + 0 + 3), + Counter.longs("test-WriteOperation-start-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-WriteOperation-start-msecs")).getAggregate(false)), + Counter.longs("test-WriteOperation-process-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-WriteOperation-process-msecs")).getAggregate(false)), + Counter.longs("test-WriteOperation-finish-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-WriteOperation-finish-msecs")).getAggregate(false))), + counterSet); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPathTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPathTest.java new file mode 100644 index 0000000000000..9904bd5a2428e --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPathTest.java @@ -0,0 +1,334 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.gcsfs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.net.URI; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +/** + * Tests of GcsPath. + */ +@RunWith(JUnit4.class) +public class GcsPathTest { + + /** + * Test case, which tests parsing and building of GcsPaths. + */ + static final class TestCase { + + final String uri; + final String expectedBucket; + final String expectedObject; + final String[] namedComponents; + + TestCase(String uri, String... namedComponents) { + this.uri = uri; + this.expectedBucket = namedComponents[0]; + this.namedComponents = namedComponents; + this.expectedObject = uri.substring(expectedBucket.length() + 6); + } + } + + // Each test case is an expected URL, then the components used to build it. + // Empty components result in a double slash. + static final List PATH_TEST_CASES = Arrays.asList( + new TestCase("gs://bucket/then/object", "bucket", "then", "object"), + new TestCase("gs://bucket//then/object", "bucket", "", "then", "object"), + new TestCase("gs://bucket/then//object", "bucket", "then", "", "object"), + new TestCase("gs://bucket/then///object", "bucket", "then", "", "", "object"), + new TestCase("gs://bucket/then/object/", "bucket", "then", "object/"), + new TestCase("gs://bucket/then/object/", "bucket", "then/", "object/"), + new TestCase("gs://bucket/then/object//", "bucket", "then", "object", ""), + new TestCase("gs://bucket/then/object//", "bucket", "then", "object/", ""), + new TestCase("gs://bucket/", "bucket") + ); + + @Test + public void testGcsPathParsing() throws IOException { + for (TestCase testCase : PATH_TEST_CASES) { + String uriString = testCase.uri; + + GcsPath path = GcsPath.fromUri(URI.create(uriString)); + // Deconstruction - check bucket, object, and components. + assertEquals(testCase.expectedBucket, path.getBucket()); + assertEquals(testCase.expectedObject, path.getObject()); + assertEquals(testCase.uri, + testCase.namedComponents.length, path.getNameCount()); + + // Construction - check that the path can be built from components. + GcsPath built = GcsPath.fromComponents(null, null); + for (String component : testCase.namedComponents) { + built = built.resolve(component); + } + assertEquals(testCase.uri, built.toString()); + } + } + + @Test + public void testParentRelationship() throws IOException { + GcsPath path = GcsPath.fromComponents("bucket", "then/object"); + assertEquals("bucket", path.getBucket()); + assertEquals("then/object", path.getObject()); + assertEquals(3, path.getNameCount()); + assertTrue(path.endsWith("object")); + assertTrue(path.startsWith("bucket/then")); + + GcsPath parent = path.getParent(); // gs://bucket/then/ + assertEquals("bucket", parent.getBucket()); + assertEquals("then/", parent.getObject()); + assertEquals(2, parent.getNameCount()); + assertThat(path, Matchers.not(Matchers.equalTo(parent))); + assertTrue(path.startsWith(parent)); + assertFalse(parent.startsWith(path)); + assertTrue(parent.endsWith("then/")); + assertTrue(parent.startsWith("bucket/then")); + assertTrue(parent.isAbsolute()); + + GcsPath root = path.getRoot(); + assertEquals(0, root.getNameCount()); + assertEquals("gs://", root.toString()); + assertEquals("", root.getBucket()); + assertEquals("", root.getObject()); + assertTrue(root.isAbsolute()); + assertThat(root, Matchers.equalTo(parent.getRoot())); + + GcsPath grandParent = parent.getParent(); // gs://bucket/ + assertEquals(1, grandParent.getNameCount()); + assertEquals("gs://bucket/", grandParent.toString()); + assertTrue(grandParent.isAbsolute()); + assertThat(root, Matchers.equalTo(grandParent.getParent())); + assertThat(root.getParent(), Matchers.nullValue()); + + assertTrue(path.startsWith(path.getRoot())); + assertTrue(parent.startsWith(path.getRoot())); + } + + @Test + public void testRelativeParent() throws IOException { + GcsPath path = GcsPath.fromComponents(null, "a/b"); + GcsPath parent = path.getParent(); + assertEquals("a/", parent.toString()); + + GcsPath grandParent = parent.getParent(); + assertNull(grandParent); + } + + @Test + public void testUriSupport() throws IOException { + URI uri = URI.create("gs://bucket/some/path"); + + GcsPath path = GcsPath.fromUri(uri); + assertEquals("bucket", path.getBucket()); + assertEquals("some/path", path.getObject()); + + URI reconstructed = path.toUri(); + assertEquals(uri, reconstructed); + + path = GcsPath.fromUri("gs://bucket"); + assertEquals("gs://bucket/", path.toString()); + } + + @Test + public void testBucketParsing() throws IOException { + GcsPath path = GcsPath.fromUri("gs://bucket"); + GcsPath path2 = GcsPath.fromUri("gs://bucket/"); + + assertEquals(path, path2); + assertEquals(path.toString(), path2.toString()); + assertEquals(path.toUri(), path2.toUri()); + } + + @Test + public void testGcsPathToString() throws Exception { + String filename = "gs://some_bucket/some/file.txt"; + GcsPath path = GcsPath.fromUri(filename); + assertEquals(filename, path.toString()); + } + + @Test + public void testEquals() { + GcsPath a = GcsPath.fromComponents(null, "a/b/c"); + GcsPath a2 = GcsPath.fromComponents(null, "a/b/c"); + assertFalse(a.isAbsolute()); + assertFalse(a2.isAbsolute()); + + GcsPath b = GcsPath.fromComponents("bucket", "a/b/c"); + GcsPath b2 = GcsPath.fromComponents("bucket", "a/b/c"); + assertTrue(b.isAbsolute()); + assertTrue(b2.isAbsolute()); + + assertEquals(a, a); + assertThat(a, Matchers.not(Matchers.equalTo(b))); + assertThat(b, Matchers.not(Matchers.equalTo(a))); + + assertEquals(a, a2); + assertEquals(a2, a); + assertEquals(b, b2); + assertEquals(b2, b); + + assertThat(a, Matchers.not(Matchers.equalTo(Paths.get("/tmp/foo")))); + assertTrue(a != null); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidGcsPath() { + @SuppressWarnings("unused") + GcsPath filename = + GcsPath.fromUri("file://invalid/gcs/path"); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidBucket() { + GcsPath.fromComponents("invalid/", ""); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidObject_newline() { + GcsPath.fromComponents(null, "a\nb"); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidObject_cr() { + GcsPath.fromComponents(null, "a\rb"); + } + + @Test + public void testResolveUri() { + GcsPath path = GcsPath.fromComponents("bucket", "a/b/c"); + GcsPath d = path.resolve("gs://bucket2/d"); + assertEquals("gs://bucket2/d", d.toString()); + } + + @Test + public void testResolveOther() { + GcsPath a = GcsPath.fromComponents("bucket", "a"); + GcsPath b = a.resolve(Paths.get("b")); + assertEquals("a/b", b.getObject()); + } + + @Test + public void testCompareTo() { + GcsPath a = GcsPath.fromComponents("bucket", "a"); + GcsPath b = GcsPath.fromComponents("bucket", "b"); + GcsPath b2 = GcsPath.fromComponents("bucket2", "b"); + GcsPath brel = GcsPath.fromComponents(null, "b"); + GcsPath a2 = GcsPath.fromComponents("bucket", "a"); + GcsPath arel = GcsPath.fromComponents(null, "a"); + + assertThat(a.compareTo(b), Matchers.lessThan(0)); + assertThat(b.compareTo(a), Matchers.greaterThan(0)); + assertThat(a.compareTo(a2), Matchers.equalTo(0)); + + assertThat(a.hashCode(), Matchers.equalTo(a2.hashCode())); + assertThat(a.hashCode(), Matchers.not(Matchers.equalTo(b.hashCode()))); + assertThat(b.hashCode(), Matchers.not(Matchers.equalTo(brel.hashCode()))); + + assertThat(brel.compareTo(b), Matchers.lessThan(0)); + assertThat(b.compareTo(brel), Matchers.greaterThan(0)); + assertThat(arel.compareTo(brel), Matchers.lessThan(0)); + assertThat(brel.compareTo(arel), Matchers.greaterThan(0)); + + assertThat(b.compareTo(b2), Matchers.lessThan(0)); + assertThat(b2.compareTo(b), Matchers.greaterThan(0)); + } + + @Test + public void testCompareTo_ordering() { + GcsPath ab = GcsPath.fromComponents("bucket", "a/b"); + GcsPath abc = GcsPath.fromComponents("bucket", "a/b/c"); + GcsPath a1b = GcsPath.fromComponents("bucket", "a-1/b"); + + assertThat(ab.compareTo(a1b), Matchers.lessThan(0)); + assertThat(a1b.compareTo(ab), Matchers.greaterThan(0)); + + assertThat(ab.compareTo(abc), Matchers.lessThan(0)); + assertThat(abc.compareTo(ab), Matchers.greaterThan(0)); + } + + @Test + public void testCompareTo_buckets() { + GcsPath a = GcsPath.fromComponents(null, "a/b/c"); + GcsPath b = GcsPath.fromComponents("bucket", "a/b/c"); + + assertThat(a.compareTo(b), Matchers.lessThan(0)); + assertThat(b.compareTo(a), Matchers.greaterThan(0)); + } + + @Test + public void testIterator() { + GcsPath a = GcsPath.fromComponents("bucket", "a/b/c"); + Iterator it = a.iterator(); + + assertTrue(it.hasNext()); + assertEquals("gs://bucket/", it.next().toString()); + assertTrue(it.hasNext()); + assertEquals("a", it.next().toString()); + assertTrue(it.hasNext()); + assertEquals("b", it.next().toString()); + assertTrue(it.hasNext()); + assertEquals("c", it.next().toString()); + assertFalse(it.hasNext()); + } + + @Test + public void testSubpath() { + GcsPath a = GcsPath.fromComponents("bucket", "a/b/c/d"); + assertThat(a.subpath(0, 1).toString(), Matchers.equalTo("gs://bucket/")); + assertThat(a.subpath(0, 2).toString(), Matchers.equalTo("gs://bucket/a")); + assertThat(a.subpath(0, 3).toString(), Matchers.equalTo("gs://bucket/a/b")); + assertThat(a.subpath(0, 4).toString(), Matchers.equalTo("gs://bucket/a/b/c")); + assertThat(a.subpath(1, 2).toString(), Matchers.equalTo("a")); + assertThat(a.subpath(2, 3).toString(), Matchers.equalTo("b")); + assertThat(a.subpath(2, 4).toString(), Matchers.equalTo("b/c")); + assertThat(a.subpath(2, 5).toString(), Matchers.equalTo("b/c/d")); + } + + @Test + public void testGetName() { + GcsPath a = GcsPath.fromComponents("bucket", "a/b/c/d"); + assertEquals(5, a.getNameCount()); + assertThat(a.getName(0).toString(), Matchers.equalTo("gs://bucket/")); + assertThat(a.getName(1).toString(), Matchers.equalTo("a")); + assertThat(a.getName(2).toString(), Matchers.equalTo("b")); + assertThat(a.getName(3).toString(), Matchers.equalTo("c")); + assertThat(a.getName(4).toString(), Matchers.equalTo("d")); + } + + @Test(expected = IllegalArgumentException.class) + public void testSubPathError() { + GcsPath a = GcsPath.fromComponents("bucket", "a/b/c/d"); + a.subpath(1, 1); // throws IllegalArgumentException + Assert.fail(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListenerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListenerTest.java new file mode 100644 index 0000000000000..96e5bf6b49883 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListenerTest.java @@ -0,0 +1,83 @@ +/** + * Copyright 2013 Google Inc. All Rights Reserved. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.gcsio; + +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.verifyZeroInteractions; + +import com.google.api.client.googleapis.media.MediaHttpUploader.UploadState; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.slf4j.Logger; + +/** Unit tests for {@link LoggingMediaHttpUploaderProgressListener}. */ +@RunWith(JUnit4.class) +public class LoggingMediaHttpUploaderProgressListenerTest { + @Mock + private Logger mockLogger; + private LoggingMediaHttpUploaderProgressListener listener; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + listener = new LoggingMediaHttpUploaderProgressListener("NAME", 60000L); + } + + @Test + public void testLoggingInitiation() { + listener.progressChanged(mockLogger, UploadState.INITIATION_STARTED, 0L, 0L); + verify(mockLogger).info("Uploading: {}", "NAME"); + verifyNoMoreInteractions(mockLogger); + } + + @Test + public void testLoggingProgressAfterSixtySeconds() { + listener.progressChanged(mockLogger, UploadState.MEDIA_IN_PROGRESS, 10485760L, 60001L); + listener.progressChanged(mockLogger, UploadState.MEDIA_IN_PROGRESS, 104857600L, 120002L); + verify(mockLogger).info( + "Uploading: NAME Average Rate: 0.167 MiB/s, Current Rate: 0.167 MiB/s, Total: 10.000 MiB"); + verify(mockLogger).info( + "Uploading: NAME Average Rate: 0.833 MiB/s, Current Rate: 1.500 MiB/s, Total: 100.000 MiB"); + verifyNoMoreInteractions(mockLogger); + } + + @Test + public void testSkippingLoggingAnInProgressUpdate() { + listener.progressChanged(mockLogger, UploadState.MEDIA_IN_PROGRESS, 104857600L, 60000L); + verifyZeroInteractions(mockLogger); + } + + @Test + public void testLoggingCompletion() { + listener.progressChanged(mockLogger, UploadState.MEDIA_COMPLETE, 104857600L, 60000L); + verify(mockLogger).info("Finished Uploading: {}", "NAME"); + verifyNoMoreInteractions(mockLogger); + } + + @Test + public void testOtherUpdatesIgnored() { + listener.progressChanged(mockLogger, UploadState.NOT_STARTED, 0L, 60001L); + listener.progressChanged(mockLogger, UploadState.INITIATION_COMPLETE, 0L, 60001L); + verifyZeroInteractions(mockLogger); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/KVTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/KVTest.java new file mode 100644 index 0000000000000..dae544fb033af --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/KVTest.java @@ -0,0 +1,73 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Comparator; + +/** + * Tests for KV. + */ +@RunWith(JUnit4.class) +public class KVTest { + static final Integer testValues[] = + {null, Integer.MIN_VALUE, -1, 0, 1, Integer.MAX_VALUE}; + + // Wrapper around Integer.compareTo() to support null values. + private int compareInt(Integer a, Integer b) { + if (a == null) { + return b == null ? 0 : -1; + } else { + return b == null ? 1 : a.compareTo(b); + } + } + + @Test + public void testOrderByKey() { + Comparator> orderByKey = new KV.OrderByKey<>(); + for (Integer key1 : testValues) { + for (Integer val1 : testValues) { + for (Integer key2 : testValues) { + for (Integer val2 : testValues) { + assertEquals(compareInt(key1, key2), + orderByKey.compare(KV.of(key1, val1), KV.of(key2, val2))); + } + } + } + } + } + + @Test + public void testOrderByValue() { + Comparator> orderByValue = new KV.OrderByValue<>(); + for (Integer key1 : testValues) { + for (Integer val1 : testValues) { + for (Integer key2 : testValues) { + for (Integer val2 : testValues) { + assertEquals(compareInt(val1, val2), + orderByValue.compare(KV.of(key1, val1), KV.of(key2, val2))); + } + } + } + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PCollectionListTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PCollectionListTest.java new file mode 100644 index 0000000000000..a6c180fc9abe3 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PCollectionListTest.java @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Collections; + +/** + * Tests for PCollectionLists. + */ +@RunWith(JUnit4.class) +public class PCollectionListTest { + @Test + public void testEmptyListFailure() { + try { + PCollectionList.of(Collections.>emptyList()); + fail("should have failed"); + } catch (IllegalArgumentException exn) { + assertThat( + exn.toString(), + containsString( + "must either have a non-empty list of PCollections, " + + "or must first call empty(Pipeline)")); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PDoneTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PDoneTest.java new file mode 100644 index 0000000000000..e886f350c12d7 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PDoneTest.java @@ -0,0 +1,98 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import static com.google.cloud.dataflow.sdk.TestUtils.LINES; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.PTransform; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.File; + +/** + * Tests for PDone. + */ +@RunWith(JUnit4.class) +public class PDoneTest { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + /** + * A PTransform that just returns a fresh PDone. + */ + static class EmptyTransform extends PTransform { + @Override + public PDone apply(PBegin begin) { + return new PDone(); + } + } + + /** + * A PTransform that's composed of something that returns a PDone. + */ + static class SimpleTransform extends PTransform { + private final String filename; + + public SimpleTransform(String filename) { + this.filename = filename; + } + + @Override + public PDone apply(PBegin begin) { + return + begin + .apply(Create.of(LINES)) + .apply(TextIO.Write.to(filename)); + } + } + + // TODO: This test doesn't work, because we can't handle composite + // transforms that contain no nested transforms. + // @Test + // @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void DISABLED_testEmptyTransform() { + Pipeline p = TestPipeline.create(); + + p.begin().apply(new EmptyTransform()); + + p.run(); + } + + // Cannot run on the service, unless we allocate a GCS temp file + // instead of a local temp file. Or switch to applying a different + // transform that returns PDone. + @Test + public void testSimpleTransform() throws Exception { + File tmpFile = tmpFolder.newFile("file.txt"); + String filename = tmpFile.getPath(); + + Pipeline p = TestPipeline.create(); + + p.begin().apply(new SimpleTransform(filename)); + + p.run(); + } +} From 13a46e000dbdd814f60e1558804173e586405066 Mon Sep 17 00:00:00 2001 From: vanya Date: Sat, 13 Dec 2014 14:36:27 -0800 Subject: [PATCH 0003/1541] Add TextIO.Read.{,.Bound}.withoutValidation() method and plumb the validate_source value. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82063199 --- .../google/cloud/dataflow/sdk/io/TextIO.java | 43 ++++++++++++++++--- .../runners/DataflowPipelineTranslator.java | 12 ++++++ .../runners/dataflow/TextIOTranslator.java | 1 + .../dataflow/sdk/util/PropertyNames.java | 1 + .../cloud/dataflow/sdk/io/TextIOTest.java | 9 ++++ 5 files changed, 61 insertions(+), 5 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java index 5d1cb205b4224..f6ff0e29fd8fe 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java @@ -137,6 +137,18 @@ public static Bound withCoder(Coder coder) { return new Bound<>(coder); } + /** + * Returns a TextIO.Read PTransform that has GCS path validation on + * pipeline creation disabled. + * + *

This can be useful in the case where the GCS input does not + * exist at the pipeline creation time, but is expected to be + * available at execution time. + */ + public static Bound withoutValidation() { + return new Bound<>(DEFAULT_TEXT_CODER).withoutValidation(); + } + // TODO: strippingNewlines, gzipped, etc. /** @@ -156,14 +168,18 @@ public static class Bound /** The Coder to use to decode each line. */ @Nullable final Coder coder; + /** An option to indicate if input validation is desired. Default is true. */ + final boolean validate; + Bound(Coder coder) { - this(null, null, coder); + this(null, null, coder, true); } - Bound(String name, String filepattern, Coder coder) { + Bound(String name, String filepattern, Coder coder, boolean validate) { super(name); this.coder = coder; this.filepattern = filepattern; + this.validate = validate; } /** @@ -171,7 +187,7 @@ public static class Bound * with the given step name. Does not modify this object. */ public Bound named(String name) { - return new Bound<>(name, filepattern, coder); + return new Bound<>(name, filepattern, coder, validate); } /** @@ -181,7 +197,7 @@ public Bound named(String name) { * filepatterns.) Does not modify this object. */ public Bound from(String filepattern) { - return new Bound<>(name, filepattern, coder); + return new Bound<>(name, filepattern, coder, validate); } /** @@ -194,7 +210,20 @@ public Bound from(String filepattern) { * elements of the resulting PCollection */ public Bound withCoder(Coder coder) { - return new Bound<>(name, filepattern, coder); + return new Bound<>(name, filepattern, coder, validate); + } + + /** + * Returns a new TextIO.Read PTransform that's like this one but + * that has GCS path validation on pipeline creation disabled. + * Does not modify this object. + * + *

This can be useful in the case where the GCS input does not + * exist at the pipeline creation time, but is expected to be + * available at execution time. + */ + public Bound withoutValidation() { + return new Bound<>(name, filepattern, coder, false); } @Override @@ -222,6 +251,10 @@ public String getFilepattern() { return filepattern; } + public boolean needsValidation() { + return validate; + } + static { DirectPipelineRunner.registerDefaultTransformEvaluator( Bound.class, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index 6f39a2bae5b8a..5f0b5a1985f04 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -20,6 +20,7 @@ import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray; import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString; import static com.google.cloud.dataflow.sdk.util.StringUtils.jsonStringToByteArray; +import static com.google.cloud.dataflow.sdk.util.Structs.addBoolean; import static com.google.cloud.dataflow.sdk.util.Structs.addDictionary; import static com.google.cloud.dataflow.sdk.util.Structs.addList; import static com.google.cloud.dataflow.sdk.util.Structs.addLong; @@ -212,6 +213,12 @@ public interface TranslationContext { */ public void addEncodingInput(Coder value); + /** + * Adds an input with the given name and value to the current + * Dataflow step. + */ + public void addInput(String name, Boolean value); + /** * Adds an input with the given name and value to the current * Dataflow step. @@ -535,6 +542,11 @@ public void addEncodingInput(Coder coder) { addObject(getProperties(), PropertyNames.ENCODING, encoding); } + @Override + public void addInput(String name, Boolean value) { + addBoolean(getProperties(), name, value); + } + @Override public void addInput(String name, String value) { addString(getProperties(), name, value); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java index 05a44648eba95..0a2f1f8a32571 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java @@ -66,6 +66,7 @@ private void translateReadHelper( context.addInput(PropertyNames.FORMAT, "text"); context.addInput(PropertyNames.FILEPATTERN, gcsPath); context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); + context.addInput(PropertyNames.VALIDATE_SOURCE, transform.needsValidation()); // TODO: Orderedness? } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java index 85a81cdeff9c9..c10f07fd98ed2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java @@ -83,5 +83,6 @@ public class PropertyNames { public static final String USER_FN = "user_fn"; public static final String USER_NAME = "user_name"; public static final String USES_KEYED_STATE = "uses_keyed_state"; + public static final String VALIDATE_SOURCE = "validate_source"; public static final String VALUE = "value"; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java index b6aaf59b51ad8..31bb18cd955f5 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java @@ -23,7 +23,9 @@ import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES_ARRAY; import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; @@ -410,4 +412,11 @@ public void testBadWildcardBrackets() throws Exception { thrown.expectMessage("wildcard"); pipeline.run(); } + + @Test + public void testWithoutValidationFlag() throws Exception { + TextIO.Read.Bound read = TextIO.Read.from("gs://bucket/foo*/baz"); + assertTrue(read.needsValidation()); + assertFalse(read.withoutValidation().needsValidation()); + } } From c266ce35a39f93d19ee4ffe837aebb326b7bde67 Mon Sep 17 00:00:00 2001 From: earhart Date: Sat, 13 Dec 2014 19:42:46 -0800 Subject: [PATCH 0004/1541] Fix some misc. warnings. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82071212 --- .../cloud/dataflow/sdk/coders/AvroCoder.java | 1 + .../dataflow/sdk/coders/CoderException.java | 2 ++ .../cloud/dataflow/sdk/coders/MapCoder.java | 1 + .../google/cloud/dataflow/sdk/io/TextIO.java | 4 ++++ .../sdk/runners/worker/AvroByteSink.java | 2 +- .../sdk/runners/worker/AvroByteSource.java | 2 +- .../dataflow/sdk/util/WindowedValue.java | 20 +++++++++++-------- .../dataflow/sdk/util/common/Counter.java | 4 +++- .../sdk/util/common/ForwardingReiterator.java | 4 +++- .../cloud/dataflow/sdk/values/TupleTag.java | 8 ++++++-- .../dataflow/sdk/values/TupleTagList.java | 2 ++ 11 files changed, 36 insertions(+), 14 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java index 5ea631a970a79..9532da725ac97 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java @@ -81,6 +81,7 @@ * @param the type of elements handled by this coder */ public class AvroCoder extends StandardCoder { + private static final long serialVersionUID = 0; /** * Returns an {@code AvroCoder} instance for the provided element type. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderException.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderException.java index 1bbc3fa176b7e..b19eb202dfed2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderException.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderException.java @@ -23,6 +23,8 @@ * decoding a value. */ public class CoderException extends IOException { + private static final long serialVersionUID = 0; + public CoderException(String message) { super(message); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java index fa3fc58950150..244419da6ac0e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java @@ -41,6 +41,7 @@ * @param the type of the values of the KVs being transcoded */ public class MapCoder extends MapCoderBase> { + private static final long serialVersionUID = 0; /** * Produces a MapCoder with the given keyCoder and valueCoder. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java index f6ff0e29fd8fe..c0e97ff7527b8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java @@ -162,6 +162,8 @@ public static Bound withoutValidation() { */ public static class Bound extends PTransform> { + private static final long serialVersionUID = 0; + /** The filepattern to read from. */ @Nullable final String filepattern; @@ -367,6 +369,8 @@ public static Bound withCoder(Coder coder) { */ public static class Bound extends PTransform, PDone> { + private static final long serialVersionUID = 0; + /** The filename to write to. */ @Nullable final String filenamePrefix; /** Suffix to use for each filename. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSink.java index 404b2d261fc98..504aa69eecfcd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSink.java @@ -48,7 +48,7 @@ public AvroByteSink(String filenamePrefix, Coder coder) { public AvroByteSink(String filenamePrefix, String shardFormat, String filenameSuffix, int shardCount, Coder coder) { this.coder = coder; - avroSink = new AvroSink( + avroSink = new AvroSink<>( filenamePrefix, shardFormat, filenameSuffix, shardCount, WindowedValue.getValueOnlyCoder(AvroCoder.of(ByteBuffer.class, schema))); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSource.java index b71700a08fcab..5a8524eb88853 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSource.java @@ -49,7 +49,7 @@ public AvroByteSource(String filename, @Nullable Long endPosition, Coder coder) { this.coder = coder; - avroSource = new AvroSource( + avroSource = new AvroSource<>( filename, startPosition, endPosition, WindowedValue.getValueOnlyCoder(AvroCoder.of(ByteBuffer.class, schema))); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java index de310b8271149..636b77d8b1109 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java @@ -79,9 +79,9 @@ public static WindowedValue valueInGlobalWindow(V value) { * Returns a {@code WindowedValue} with the given value and default timestamp and empty windows. */ public static WindowedValue valueInEmptyWindows(V value) { - return new WindowedValue<>(value, - new Instant(Long.MIN_VALUE), - new ArrayList()); + return new WindowedValue(value, + new Instant(Long.MIN_VALUE), + new ArrayList<>()); } private WindowedValue(V value, @@ -140,11 +140,11 @@ public static WindowedValueCoder getValueOnlyCoder(Coder valueCoder) { @Override public boolean equals(Object o) { if (o instanceof WindowedValue) { - WindowedValue that = (WindowedValue) o; + WindowedValue that = (WindowedValue) o; if (that.timestamp.isEqual(timestamp) && that.windows.size() == windows.size()) { - for (Iterator thatIterator = that.windows.iterator(), thisIterator = windows.iterator(); + for (Iterator thatIterator = that.windows.iterator(), thisIterator = windows.iterator(); thatIterator.hasNext() && thisIterator.hasNext(); - /* do nothng */) { + /* do nothing */) { if (!thatIterator.next().equals(thisIterator.next())) { return false; } @@ -197,6 +197,8 @@ public Coder getValueCoder() { * Coder for {@code WindowedValue}. */ public static class FullWindowedValueCoder extends WindowedValueCoder { + private static final long serialVersionUID = 0; + private final Coder windowCoder; // Precompute and cache the coder for a list of windows. private final Coder> windowsCoder; @@ -213,8 +215,9 @@ public static FullWindowedValueCoder of( List> components) { checkArgument(components.size() == 2, "Expecting 2 components, got " + components.size()); - return of(components.get(0), - (Coder) components.get(1)); + @SuppressWarnings("unchecked") + Coder window = (Coder) components.get(1); + return of(components.get(0), window); } @SuppressWarnings("unchecked") @@ -305,6 +308,7 @@ public List> getComponents() { * timestamp and windows for encoding, and uses defaults timestamp, and windows for decoding. */ public static class ValueOnlyWindowedValueCoder extends WindowedValueCoder { + private static final long serialVersionUID = 0; public static ValueOnlyWindowedValueCoder of( Coder valueCoder) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Counter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Counter.java index 8b5f636ac5da3..4516edd5dbdfe 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Counter.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Counter.java @@ -245,7 +245,9 @@ public AggregationKind getKind() { * Returns the counter's type. */ public Class getType() { - return new TypeToken(getClass()) {}.getRawType(); + return new TypeToken(getClass()) { + private static final long serialVersionUID = 0; + }.getRawType(); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ForwardingReiterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ForwardingReiterator.java index f3008232a1074..179b5abf4d42e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ForwardingReiterator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ForwardingReiterator.java @@ -41,7 +41,9 @@ public ForwardingReiterator(Reiterator base) { protected ForwardingReiterator clone() { ForwardingReiterator result; try { - result = (ForwardingReiterator) super.clone(); + @SuppressWarnings("unchecked") + ForwardingReiterator declResult = (ForwardingReiterator) super.clone(); + result = declResult; } catch (CloneNotSupportedException e) { throw new AssertionError( "Object.clone() for a ForwardingReiterator threw " diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java index 58562163f4a85..4b47e80fdbabc 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java @@ -51,6 +51,8 @@ * e.g., a {@code PCollection}. */ public class TupleTag implements Serializable { + private static final long serialVersionUID = 0; + /** * Constructs a new {@code TupleTag}, with a fresh unique id. * @@ -113,7 +115,9 @@ public String getOutName(int outIndex) { * e.g., {@code new TupleTag(){}}. */ public TypeToken getTypeToken() { - return new TypeToken(getClass()) {}; + return new TypeToken(getClass()) { + private static final long serialVersionUID = 0; + }; } @@ -138,7 +142,7 @@ static String genId() { private static TupleTag fromJson( @JsonProperty(PropertyNames.VALUE) String id, @JsonProperty(PropertyNames.IS_GENERATED) boolean generated) { - return new TupleTag(id, generated); + return new TupleTag<>(id, generated); } private TupleTag(String id, boolean generated) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTagList.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTagList.java index 27a0683bab5aa..515a388b22e2d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTagList.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTagList.java @@ -52,6 +52,8 @@ * } */ public class TupleTagList implements Serializable { + private static final long serialVersionUID = 0; + /** * Returns an empty TupleTagList. * From 146b8c8c5c714f1a1ad3957c4f2305d5ae726b44 Mon Sep 17 00:00:00 2001 From: fjp Date: Sat, 13 Dec 2014 20:13:42 -0800 Subject: [PATCH 0005/1541] Logging improvements - decrease level for scary looking but inert credential and upload messages - increase user counters to INFO - change aggregator example to really be empty lines - add brief logging to DirectPipelineRunner [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82071972 --- .../google/cloud/dataflow/examples/WordCount.java | 12 ++++++------ .../sdk/runners/DataflowPipelineRunner.java | 4 ++-- .../sdk/runners/DirectPipelineRunner.java | 6 +++++- .../cloud/dataflow/sdk/util/Credentials.java | 4 ++-- .../cloud/dataflow/sdk/util/PackageUtil.java | 15 +++++++++++++-- .../sdk/util/RetryHttpRequestInitializer.java | 8 ++++---- .../LoggingMediaHttpUploaderProgressListener.java | 6 +++--- ...gingMediaHttpUploaderProgressListenerTest.java | 8 ++++---- 8 files changed, 39 insertions(+), 24 deletions(-) diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java b/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java index 96893b909bc7a..4295e7868d26e 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java @@ -66,14 +66,14 @@ public void startBundle(Context c) { @Override public void processElement(ProcessContext c) { - // Split the line into words. - String[] words = c.element().split("[^a-zA-Z']+"); - - // Keep track of the number of lines without any words encountered while tokenizing. - // This aggregator is visible in the monitoring UI when run using DataflowPipelineRunner. - if (words.length == 0) { + // Keep track of the number of empty lines. (When using the [Blocking]DataflowPipelineRunner, + // Aggregators are shown in the monitoring UI.) + if (c.element().trim().isEmpty()) { emptyLines.addValue(1L); } + + // Split the line into words. + String[] words = c.element().split("[^a-zA-Z']+"); // Output each word encountered into the output PCollection. for (String word : words) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index ed01b8345c186..9f770fcbb33cd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -124,8 +124,8 @@ public static DataflowPipelineRunner fromOptions(PipelineOptions options) { if (dataflowOptions.getFilesToStage() == null) { dataflowOptions.setFilesToStage(detectClassPathResourcesToStage( DataflowPipelineRunner.class.getClassLoader())); - LOG.info("No specified files to stage. Defaulting to files: {}", - dataflowOptions.getFilesToStage()); + LOG.info("PipelineOptions.filesToStage was not specified. " + + "Defaulting to files from the classpath: {}", dataflowOptions.getFilesToStage()); } // Verify jobName according to service requirements. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java index a19b2055a0b99..cc1fd8f38cdae 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java @@ -320,13 +320,17 @@ public T ensureSerializableByCoder( @Override public EvaluationResults run(Pipeline pipeline) { + LOG.info("Executing pipeline using the DirectPipelineRunner."); + Evaluator evaluator = new Evaluator(); evaluator.run(pipeline); // Log all counter values for debugging purposes. for (Counter counter : evaluator.getCounters()) { - LOG.debug("Final aggregator value: {}", counter); + LOG.info("Final aggregator value: {}", counter); } + + LOG.info("Pipeline execution complete."); return evaluator; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java index 2a24a76fde9f3..419339a83415e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java @@ -152,7 +152,7 @@ public static Credential getUserCredential(GcpOptions options) try { return GoogleCredential.getApplicationDefault().createScoped(USER_SCOPES); } catch (IOException e) { - LOG.info("Failed to get application default credentials, falling back to gcloud."); + LOG.debug("Failed to get application default credentials, falling back to gcloud."); } String gcloudPath = options.getGCloudPath(); @@ -196,7 +196,7 @@ private static Credential getCredentialFromGCloud(String gcloudPath) throw new RuntimeException("Could not obtain credential using gcloud", e); } - LOG.info("Got credential from GCloud"); + LOG.info("Got user credential from GCloud"); return credential; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java index c108ceb4f1571..c9e3185edd8c3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java @@ -108,6 +108,8 @@ static List stageClasspathElementsToGcs( Collection classpathElements, GcsPath gcsStaging, Sleeper retrySleeper) { + LOG.info("Uploading {} files from PipelineOptions.filesToStage to GCS to prepare for execution " + + "in the cloud.", classpathElements.size()); ArrayList packages = new ArrayList<>(); if (gcsStaging == null) { @@ -115,6 +117,8 @@ static List stageClasspathElementsToGcs( "Can't stage classpath elements on GCS because no GCS location has been provided"); } + int numUploaded = 0; + int numCached = 0; for (String classpathElement : classpathElements) { String packageName = null; if (classpathElement.contains("=")) { @@ -134,7 +138,9 @@ static List stageClasspathElementsToGcs( try { long remoteLength = gcsUtil.fileSize(target); if (remoteLength >= 0 && remoteLength == getClasspathElementLength(classpathElement)) { - LOG.info("Skipping classpath element already on gcs: {} at {}", classpathElement, target); + LOG.debug("Skipping classpath element already on gcs: {} at {}", + classpathElement, target); + numCached++; continue; } @@ -144,10 +150,11 @@ static List stageClasspathElementsToGcs( INITIAL_BACKOFF_INTERVAL_MS); while (true) { try { - LOG.info("Uploading classpath element {} to {}", classpathElement, target); + LOG.debug("Uploading classpath element {} to {}", classpathElement, target); try (WritableByteChannel writer = gcsUtil.create(target, MimeTypes.BINARY)) { copyContent(classpathElement, writer); } + numUploaded++; break; } catch (IOException e) { if (BackOffUtils.next(retrySleeper, backoff)) { @@ -165,6 +172,10 @@ static List stageClasspathElementsToGcs( throw new RuntimeException("Could not stage classpath element: " + classpathElement, e); } } + + LOG.info("Uploading PipelineOptions.filesToStage complete: {} files newly uploaded, " + + "{} files cached", + numUploaded, numCached); return packages; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java index 34d40f1470793..c673b10204bc1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java @@ -72,9 +72,9 @@ public boolean handleIOException(HttpRequest request, boolean supportsRetry) throws IOException { boolean willRetry = super.handleIOException(request, supportsRetry); if (willRetry) { - LOG.info("Request failed with IOException, will retry: {}", request.getUrl()); + LOG.debug("Request failed with IOException, will retry: {}", request.getUrl()); } else { - LOG.info("Request failed with IOException, will NOT retry: {}", request.getUrl()); + LOG.debug("Request failed with IOException, will NOT retry: {}", request.getUrl()); } return willRetry; } @@ -104,11 +104,11 @@ public boolean handleResponse(HttpRequest request, HttpResponse response, boolean supportsRetry) throws IOException { boolean retry = handler.handleResponse(request, response, supportsRetry); if (retry) { - LOG.info("Request failed with code {} will retry: {}", + LOG.debug("Request failed with code {} will retry: {}", response.getStatusCode(), request.getUrl()); } else if (!IGNORED_RESPONSE_CODES.contains(response.getStatusCode())) { - LOG.info("Request failed with code {}, will NOT retry: {}", + LOG.debug("Request failed with code {}, will NOT retry: {}", response.getStatusCode(), request.getUrl()); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListener.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListener.java index c215f4aeafafc..26c88838b0045 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListener.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListener.java @@ -66,7 +66,7 @@ void progressChanged(Logger log, UploadState uploadState, long bytesUploaded, lo case INITIATION_STARTED: startTime = currentTime; prevTime = currentTime; - log.info("Uploading: {}", name); + log.debug("Uploading: {}", name); break; case MEDIA_IN_PROGRESS: // Limit messages to be emitted for in progress uploads. @@ -75,7 +75,7 @@ void progressChanged(Logger log, UploadState uploadState, long bytesUploaded, lo / ((currentTime - startTime) / 1000.0); double currentRate = ((bytesUploaded - prevUploadedBytes) / BYTES_IN_MB) / ((currentTime - prevTime) / 1000.0); - log.info(String.format( + log.debug(String.format( "Uploading: %s Average Rate: %.3f MiB/s, Current Rate: %.3f MiB/s, Total: %.3f MiB", name, averageRate, currentRate, bytesUploaded / BYTES_IN_MB)); prevTime = currentTime; @@ -83,7 +83,7 @@ void progressChanged(Logger log, UploadState uploadState, long bytesUploaded, lo } break; case MEDIA_COMPLETE: - log.info("Finished Uploading: {}", name); + log.debug("Finished Uploading: {}", name); break; default: } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListenerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListenerTest.java index 96e5bf6b49883..f577ff5c80a9c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListenerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/gcsio/LoggingMediaHttpUploaderProgressListenerTest.java @@ -46,7 +46,7 @@ public void setUp() { @Test public void testLoggingInitiation() { listener.progressChanged(mockLogger, UploadState.INITIATION_STARTED, 0L, 0L); - verify(mockLogger).info("Uploading: {}", "NAME"); + verify(mockLogger).debug("Uploading: {}", "NAME"); verifyNoMoreInteractions(mockLogger); } @@ -54,9 +54,9 @@ public void testLoggingInitiation() { public void testLoggingProgressAfterSixtySeconds() { listener.progressChanged(mockLogger, UploadState.MEDIA_IN_PROGRESS, 10485760L, 60001L); listener.progressChanged(mockLogger, UploadState.MEDIA_IN_PROGRESS, 104857600L, 120002L); - verify(mockLogger).info( + verify(mockLogger).debug( "Uploading: NAME Average Rate: 0.167 MiB/s, Current Rate: 0.167 MiB/s, Total: 10.000 MiB"); - verify(mockLogger).info( + verify(mockLogger).debug( "Uploading: NAME Average Rate: 0.833 MiB/s, Current Rate: 1.500 MiB/s, Total: 100.000 MiB"); verifyNoMoreInteractions(mockLogger); } @@ -70,7 +70,7 @@ public void testSkippingLoggingAnInProgressUpdate() { @Test public void testLoggingCompletion() { listener.progressChanged(mockLogger, UploadState.MEDIA_COMPLETE, 104857600L, 60000L); - verify(mockLogger).info("Finished Uploading: {}", "NAME"); + verify(mockLogger).debug("Finished Uploading: {}", "NAME"); verifyNoMoreInteractions(mockLogger); } From a4d86f9a4cb1bc18d7a97ffe3cabb2e2922a2ede Mon Sep 17 00:00:00 2001 From: mariand Date: Sat, 13 Dec 2014 20:15:51 -0800 Subject: [PATCH 0006/1541] Removed shuffle pool options. Shuffle is no longer started as a separate pool. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82072028 --- .../sdk/options/DataflowPipelineOptions.java | 2 +- .../DataflowPipelineShuffleOptions.java | 58 ----------------- .../runners/DataflowPipelineTranslator.java | 25 -------- .../DataflowPipelineTranslatorTest.java | 62 ++----------------- .../sdk/testing/TestPipelineTest.java | 4 +- 5 files changed, 7 insertions(+), 144 deletions(-) delete mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineShuffleOptions.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java index 7d05088732328..686d72e20c2e6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java @@ -33,7 +33,7 @@ */ public interface DataflowPipelineOptions extends PipelineOptions, GcpOptions, ApplicationNameOptions, DataflowPipelineDebugOptions, - DataflowPipelineShuffleOptions, DataflowPipelineWorkerPoolOptions, BigQueryOptions, + DataflowPipelineWorkerPoolOptions, BigQueryOptions, GcsOptions, StreamingOptions { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineShuffleOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineShuffleOptions.java deleted file mode 100644 index f59f5eb5d78cd..0000000000000 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineShuffleOptions.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Copyright (C) 2014 Google Inc. - * - * Licensed 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 com.google.cloud.dataflow.sdk.options; - -/** - * Options for Shuffle workers. Most users should not need to adjust the settings in this section. - */ -public interface DataflowPipelineShuffleOptions { - /** - * Disk source image to use by shuffle VMs for jobs. - * @see Compute Engine Images - */ - @Description("Dataflow shuffle VM disk image.") - String getShuffleDiskSourceImage(); - void setShuffleDiskSourceImage(String value); - - /** - * Number of workers to use with the shuffle appliance, or 0 to use - * the default number of workers. - */ - @Description("Number of shuffle workers, when using remote execution") - int getShuffleNumWorkers(); - void setShuffleNumWorkers(int value); - - /** - * Remote shuffle worker disk size, in gigabytes, or 0 to use the - * default size. - */ - @Description("Remote shuffle worker disk size, in gigabytes, or 0 to use the default size.") - int getShuffleDiskSizeGb(); - void setShuffleDiskSizeGb(int value); - - /** - * GCE availability zone for launching shuffle workers. - * - *

Default is up to the service. - */ - @Description("GCE availability zone for launching shuffle workers. " - + "Default is up to the service") - String getShuffleZone(); - void setShuffleZone(String value); -} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index 5f0b5a1985f04..b3b24269c365a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -101,7 +101,6 @@ public class DataflowPipelineTranslator { // Must be kept in sync with their internal counterparts. public static final String HARNESS_WORKER_POOL = "harness"; - public static final String SHUFFLE_WORKER_POOL = "shuffle"; private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineTranslator.class); /** @@ -358,9 +357,6 @@ public Job translate(List packages) { workerPool.setTaskrunnerSettings(taskRunnerSettings); - WorkerPool shufflePool = new WorkerPool(); - shufflePool.setKind(SHUFFLE_WORKER_POOL); - if (options.isStreaming()) { job.setType("JOB_TYPE_STREAMING"); } else { @@ -373,10 +369,8 @@ public Job translate(List packages) { workerPool.setPackages(packages); workerPool.setNumWorkers(options.getNumWorkers()); - shufflePool.setNumWorkers(options.getNumWorkers()); if (options.getDiskSourceImage() != null) { workerPool.setDiskSourceImage(options.getDiskSourceImage()); - shufflePool.setDiskSourceImage(options.getDiskSourceImage()); } if (options.getMachineType() != null) { @@ -396,33 +390,14 @@ public Job translate(List packages) { } if (!Strings.isNullOrEmpty(options.getZone())) { workerPool.setZone(options.getZone()); - shufflePool.setZone(options.getZone()); } if (options.getDiskSizeGb() > 0) { workerPool.setDiskSizeGb(options.getDiskSizeGb()); - shufflePool.setDiskSizeGb(options.getDiskSizeGb()); - } - - // Set up any specific shuffle pool parameters - if (options.getShuffleNumWorkers() > 0) { - shufflePool.setNumWorkers(options.getShuffleNumWorkers()); - } - if (options.getShuffleDiskSourceImage() != null) { - shufflePool.setDiskSourceImage(options.getShuffleDiskSourceImage()); - } - if (!Strings.isNullOrEmpty(options.getShuffleZone())) { - shufflePool.setZone(options.getShuffleZone()); - } - if (options.getShuffleDiskSizeGb() > 0) { - shufflePool.setDiskSizeGb(options.getShuffleDiskSizeGb()); } List workerPools = new LinkedList<>(); workerPools.add(workerPool); - if (!options.isStreaming()) { - workerPools.add(shufflePool); - } environment.setWorkerPools(workerPools); pipeline.traverseTopologically(this); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java index e2edb9fdc2232..228dbfd73acf7 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java @@ -20,7 +20,6 @@ import static com.google.cloud.dataflow.sdk.util.Structs.getDictionary; import static com.google.cloud.dataflow.sdk.util.Structs.getString; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; import static org.mockito.Matchers.argThat; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; @@ -152,11 +151,9 @@ public void testZoneConfig() throws IOException { Job job = DataflowPipelineTranslator.fromOptions(options).translate( p, Collections.emptyList()); - assertEquals(2, job.getEnvironment().getWorkerPools().size()); + assertEquals(1, job.getEnvironment().getWorkerPools().size()); assertEquals(testZone, job.getEnvironment().getWorkerPools().get(0).getZone()); - assertEquals(testZone, - job.getEnvironment().getWorkerPools().get(1).getZone()); } @Test @@ -171,27 +168,9 @@ public void testWorkerMachineTypeConfig() throws IOException { Job job = DataflowPipelineTranslator.fromOptions(options).translate( p, Collections.emptyList()); - assertEquals(2, job.getEnvironment().getWorkerPools().size()); - - WorkerPool workerPool = null; - - if (job - .getEnvironment() - .getWorkerPools() - .get(0) - .getKind() - .equals(DataflowPipelineTranslator.HARNESS_WORKER_POOL)) { - workerPool = job.getEnvironment().getWorkerPools().get(0); - } else if (job - .getEnvironment() - .getWorkerPools() - .get(1) - .getKind() - .equals(DataflowPipelineTranslator.HARNESS_WORKER_POOL)) { - workerPool = job.getEnvironment().getWorkerPools().get(1); - } else { - fail("Missing worker pool."); - } + assertEquals(1, job.getEnvironment().getWorkerPools().size()); + + WorkerPool workerPool = job.getEnvironment().getWorkerPools().get(0); assertEquals(testMachineType, workerPool.getMachineType()); } @@ -207,40 +186,9 @@ public void testDiskSizeGbConfig() throws IOException { Job job = DataflowPipelineTranslator.fromOptions(options).translate( p, Collections.emptyList()); - assertEquals(2, job.getEnvironment().getWorkerPools().size()); + assertEquals(1, job.getEnvironment().getWorkerPools().size()); assertEquals(diskSizeGb, job.getEnvironment().getWorkerPools().get(0).getDiskSizeGb()); - assertEquals(diskSizeGb, - job.getEnvironment().getWorkerPools().get(1).getDiskSizeGb()); - } - - @Test - public void testShufflePoolConfig() throws IOException { - final Integer numWorkers = 10; - final String diskSource = "test-disk-source"; - final Integer diskSizeGb = 12345; - final String zone = "test-zone-1"; - - DataflowPipelineOptions options = buildPipelineOptions(); - options.setShuffleNumWorkers(numWorkers); - options.setShuffleDiskSourceImage(diskSource); - options.setShuffleDiskSizeGb(diskSizeGb); - options.setShuffleZone(zone); - - Pipeline p = buildPipeline(options); - p.traverseTopologically(new RecordingPipelineVisitor()); - Job job = DataflowPipelineTranslator.fromOptions(options).translate( - p, Collections.emptyList()); - - assertEquals(2, job.getEnvironment().getWorkerPools().size()); - WorkerPool shufflePool = - job.getEnvironment().getWorkerPools().get(1); - assertEquals(shufflePool.getKind(), - DataflowPipelineTranslator.SHUFFLE_WORKER_POOL); - assertEquals(numWorkers, shufflePool.getNumWorkers()); - assertEquals(diskSource, shufflePool.getDiskSourceImage()); - assertEquals(diskSizeGb, shufflePool.getDiskSizeGb()); - assertEquals(zone, shufflePool.getZone()); } @Test diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java index da4f66ec07758..b61c9fdd95599 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java @@ -44,7 +44,7 @@ public void testCreationUsingDefaults() { public void testCreationOfPipelineOptions() throws Exception { ObjectMapper mapper = new ObjectMapper(); String stringOptions = mapper.writeValueAsString( - ImmutableMap.of("options", + ImmutableMap.of("options", ImmutableMap.builder() .put("runner", DataflowPipelineRunner.class.getName()) .put("project", "testProject") @@ -56,7 +56,6 @@ public void testCreationOfPipelineOptions() throws Exception { .put("zone", "testZone") .put("numWorkers", "1") .put("diskSizeGb", "2") - .put("shuffleDiskSizeGb", "3") .build())); System.getProperties().put("dataflowOptions", stringOptions); TestDataflowPipelineOptions options = TestPipeline.getPipelineOptions(); @@ -71,6 +70,5 @@ public void testCreationOfPipelineOptions() throws Exception { assertEquals("testServiceAccountKeyfile", options.getServiceAccountKeyfile()); assertEquals("testZone", options.getZone()); assertEquals(2, options.getDiskSizeGb()); - assertEquals(3, options.getShuffleDiskSizeGb()); } } From 8767bb0eac5627e31e4fc843fcf7ae5cb53e6383 Mon Sep 17 00:00:00 2001 From: robertwb Date: Sat, 13 Dec 2014 22:16:41 -0800 Subject: [PATCH 0007/1541] Fixes a bunch of warnings in the transforms package. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82074882 --- .../sdk/transforms/ApproximateQuantiles.java | 2 +- .../dataflow/sdk/transforms/Combine.java | 1 + .../cloud/dataflow/sdk/transforms/Create.java | 9 +++++-- .../cloud/dataflow/sdk/transforms/DoFn.java | 7 +++--- .../dataflow/sdk/transforms/DoFnTester.java | 1 + .../dataflow/sdk/transforms/Flatten.java | 6 ++--- .../dataflow/sdk/transforms/GroupByKey.java | 24 +++++++++---------- .../cloud/dataflow/sdk/transforms/Mean.java | 2 +- .../dataflow/sdk/transforms/PTransform.java | 10 +++----- .../dataflow/sdk/transforms/Partition.java | 4 +++- .../cloud/dataflow/sdk/transforms/Top.java | 2 +- .../cloud/dataflow/sdk/transforms/View.java | 6 +++-- .../transforms/windowing/FixedWindows.java | 4 ++-- .../windowing/InvalidWindowingFn.java | 6 ++--- .../sdk/transforms/windowing/Sessions.java | 2 +- .../transforms/windowing/SlidingWindows.java | 2 +- .../sdk/transforms/windowing/Window.java | 6 ++--- .../sdk/transforms/windowing/WindowingFn.java | 2 +- .../cloud/dataflow/sdk/util/DoFnRunner.java | 6 ++--- .../dataflow/sdk/values/PCollection.java | 2 +- 20 files changed, 55 insertions(+), 49 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java index ff5687fe30fb3..b2c93c4d701cd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java @@ -54,7 +54,7 @@ public class ApproximateQuantiles { /** * Returns a {@code PTransform} that takes a {@code PCollection} - * and returns a {@code PCollection>} whose sinlge value is a + * and returns a {@code PCollection>} whose single value is a * {@code List} of the approximate {@code N}-tiles of the elements * of the input {@code PCollection}. This gives an idea of the * distribution of the input elements. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java index 9b374665451ed..6746f9e75672b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -1020,6 +1020,7 @@ private KvCoder getKvCoder() { "Combine.GroupedValues requires its input values to use " + "IterableCoder"); } + @SuppressWarnings("unchecked") IterableCoder inputValuesCoder = (IterableCoder) kvValueCoder; Coder inputValueCoder = inputValuesCoder.getElemCoder(); return KvCoder.of(keyCoder, inputValueCoder); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java index 93747ea6462f7..0ff9deb039bc9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java @@ -26,6 +26,7 @@ import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PInput; import com.google.cloud.dataflow.sdk.values.TimestampedValue; +import com.google.cloud.dataflow.sdk.values.TimestampedValue.TimestampedValueCoder; import com.google.common.reflect.TypeToken; import org.joda.time.Instant; @@ -151,8 +152,9 @@ public static CreateTimestamped timestamped(Iterable> * *

The argument should not be modified after this is called. */ + @SuppressWarnings("unchecked") public static CreateTimestamped timestamped(TimestampedValue... elems) { - return new CreateTimestamped(Arrays.asList(elems)); + return new CreateTimestamped<>(Arrays.asList(elems)); } /** @@ -264,7 +266,10 @@ public PCollection apply(PBegin input) { // There aren't any elements, so we can provide a fake coder instance. // If we don't set a Coder here, users of CreateTimestamped have // no way to set the coder of the intermediate PCollection. - intermediate.setCoder((Coder) TimestampedValue.TimestampedValueCoder.of(VoidCoder.of())); + @SuppressWarnings("unchecked") + TimestampedValueCoder fakeCoder = + (TimestampedValueCoder) TimestampedValue.TimestampedValueCoder.of(VoidCoder.of()); + intermediate.setCoder(fakeCoder); } return intermediate.apply(ParDo.of(new ConvertTimestamps())); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java index 3c61ab38557d7..c9935fc25810b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java @@ -16,6 +16,7 @@ package com.google.cloud.dataflow.sdk.transforms; +import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.values.CodedTupleTag; @@ -68,7 +69,7 @@ public abstract class Context { * Returns the value of the side input. * * @throws IllegalArgumentException if this is not a side input - * @see ParDo#withSideInput + * @see ParDo#withSideInputs */ public abstract T sideInput(PCollectionView view); @@ -84,7 +85,7 @@ public abstract class Context { * windowed by the * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. * If this is the case, the output element will have a timestamp - * of negative infinity and be in the + * of negative infinity and be in the * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. */ public abstract void output(O output); @@ -103,7 +104,7 @@ public abstract class Context { * windowed by the * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. * If this is the case, the output element's timestamp will be - * the given timestamp and its window will be the + * the given timestamp and its window will be the * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. */ public abstract void outputWithTimestamp(O output, Instant timestamp); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java index 3e23b5ed04506..70ae664a6fc20 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java @@ -221,6 +221,7 @@ public List peekOutputElements() { return Lists.transform(fnRunner.getReceiver(mainOutputTag), new Function() { @Override + @SuppressWarnings("unchecked") public O apply(Object input) { return ((WindowedValue) input).getValue(); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java index 14b2169b97bfa..c52ae265fd5f2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java @@ -52,7 +52,7 @@ public class Flatten { /** - * Returns a {@link PTransform} that flattens a {@link PCollectionList} + * Returns a {@link PTransform} that flattens a {@link CollectionList} * into a {@link PCollection} containing all the elements of all * the {@link PCollection}s in its input. * @@ -112,7 +112,7 @@ private FlattenPCollectionList() { } @Override public PCollection apply(PCollectionList inputs) { - WindowingFn windowingFn; + WindowingFn windowingFn; if (!getInput().getAll().isEmpty()) { windowingFn = getInput().get(0).getWindowingFn(); for (PCollection input : getInput().getAll()) { @@ -145,7 +145,7 @@ protected Coder getDefaultOutputCoder() { /** * {@code FlattenIterables} takes a {@code PCollection>} and returns a * {@code PCollection} that contains all the elements from each iterable. - * Implements {@link #fromIterable}. + * Implements {@link #iterables}. * * @param the type of the elements of the input {@code Iterable}s and * the output {@code PCollection} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index d7a4de64e50d3..54e60cfcb6209 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -36,8 +36,6 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; -import org.joda.time.Instant; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -155,8 +153,8 @@ public static class ReifyTimestampsAndWindows @Override public PCollection>> apply( PCollection> input) { - Coder> inputCoder = getInput().getCoder(); - KvCoder inputKvCoder = (KvCoder) inputCoder; + @SuppressWarnings("unchecked") + KvCoder inputKvCoder = (KvCoder) getInput().getCoder(); Coder keyCoder = inputKvCoder.getKeyCoder(); Coder inputValueCoder = inputKvCoder.getValueCoder(); Coder> outputValueCoder = FullWindowedValueCoder.of( @@ -229,19 +227,18 @@ public int compare(WindowedValue e1, WindowedValue e2) { public static class GroupAlsoByWindow extends PTransform>>>, PCollection>>> { - private final WindowingFn windowingFn; + private final WindowingFn windowingFn; - public GroupAlsoByWindow(WindowingFn windowingFn) { + public GroupAlsoByWindow(WindowingFn windowingFn) { this.windowingFn = windowingFn; } @Override public PCollection>> apply( PCollection>>> input) { - Coder>>> inputCoder = - getInput().getCoder(); + @SuppressWarnings("unchecked") KvCoder>> inputKvCoder = - (KvCoder>>) inputCoder; + (KvCoder>>) getInput().getCoder(); Coder keyCoder = inputKvCoder.getKeyCoder(); Coder>> inputValueCoder = inputKvCoder.getValueCoder(); @@ -260,7 +257,7 @@ public PCollection>> apply( return input.apply(ParDo.of( new GroupAlsoByWindowsDoFn( - windowingFn, inputIterableElementValueCoder))) + (WindowingFn) windowingFn, inputIterableElementValueCoder))) .setCoder(outputKvCoder); } } @@ -280,6 +277,7 @@ public static class GroupByKeyOnly public GroupByKeyOnly() { } + @SuppressWarnings({"rawtypes", "unchecked"}) @Override public PCollection>> apply(PCollection> input) { WindowingFn windowingFn = getInput().getWindowingFn(); @@ -314,6 +312,7 @@ public void finishSpecifying() { * Returns the {@code Coder} of the input to this transform, which * should be a {@code KvCoder}. */ + @SuppressWarnings("unchecked") KvCoder getInputKvCoder() { Coder> inputCoder = getInput().getCoder(); if (!(inputCoder instanceof KvCoder)) { @@ -398,7 +397,6 @@ private static void evaluateHelper( for (ValueWithMetadata> elem : inputElems) { K key = elem.getValue().getKey(); V value = elem.getValue().getValue(); - Instant timestamp = elem.getTimestamp(); byte[] encodedKey; try { encodedKey = encodeToByteArray(keyCoder, key); @@ -447,9 +445,9 @@ public PCollection>> applyHelper( // merging windows as needed, using the windows assigned to the // key/value input elements and the window merge operation of the // windowing function associated with the input PCollection. - WindowingFn windowingFn = getInput().getWindowingFn(); + WindowingFn windowingFn = getInput().getWindowingFn(); if (windowingFn instanceof InvalidWindowingFn) { - String cause = ((InvalidWindowingFn) windowingFn).getCause(); + String cause = ((InvalidWindowingFn) windowingFn).getCause(); throw new IllegalStateException( "GroupByKey must have a valid Window merge function. " + "Invalid because: " + cause); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Mean.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Mean.java index 34fbb1fc29088..b198a0da25041 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Mean.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Mean.java @@ -137,7 +137,7 @@ public Coder getAccumulatorCoder( // The casts are needed because CountSum.class is a // Class, but we need a // Class.CountSum>. - return SerializableCoder.of((Class) (Class) CountSum.class); + return SerializableCoder.of((Class) (Class) CountSum.class); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java index 5906d7212dba0..6fc2618dcc021 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java @@ -24,7 +24,6 @@ import com.google.cloud.dataflow.sdk.values.POutput; import com.google.cloud.dataflow.sdk.values.TypedPValue; -import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.io.Serializable; @@ -95,9 +94,7 @@ * *

Each PCollection output produced by a PTransform, * either directly or within a "bundling" class, automatically gets - * its own name derived from the name of its producing PTransform. An - * output's name can be changed by invoking - * {@link com.google.cloud.dataflow.sdk.values.PValue#setName}. + * its own name derived from the name of its producing PTransform. * *

Each PCollection output produced by a PTransform * also records a {@link com.google.cloud.dataflow.sdk.coders.Coder} @@ -348,14 +345,13 @@ protected String getKindString() { return StringUtils.approximateSimpleName(getClass()); } - private void writeObject(ObjectOutputStream oos) throws IOException { + private void writeObject(ObjectOutputStream oos) { // We don't really want to be serializing this object, but we // often have serializable anonymous DoFns nested within a // PTransform. } - private void readObject(ObjectInputStream oos) - throws IOException, ClassNotFoundException { + private void readObject(ObjectInputStream oos) { // We don't really want to be serializing this object, but we // often have serializable anonymous DoFns nested within a // PTransform. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java index 74a1359aa5ed0..a6444b2774f4c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java @@ -162,7 +162,9 @@ public void processElement(ProcessContext c) { T1 input = c.element(); int partition = partitionFn.partitionFor(input, numPartitions); if (0 <= partition && partition < numPartitions) { - c.sideOutput((TupleTag) outputTags.get(partition), input); + @SuppressWarnings("unchecked") + TupleTag typedTag = (TupleTag) outputTags.get(partition); + c.sideOutput(typedTag, input); } else { throw new IndexOutOfBoundsException( "Partition function returned out of bounds index: " + diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java index 1f63808fc2237..2e5e334b8a73d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java @@ -460,7 +460,7 @@ public void registerByteSizeObserver( throws Exception { listCoder.registerByteSizeObserver(value.asList(), observer, context); } - }; + } } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java index d3bb863888707..637aeb73f6f3b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java @@ -70,7 +70,7 @@ public static AsIterable asIterable() { * A {@PTransform} that produces a {@link PCollectionView} of a singleton {@link PCollection} * yielding the single element it contains. * - *

Instantiate via {@link View.asIterable}. + *

Instantiate via {@link View#asIterable}. */ public static class AsIterable extends PTransform< PCollection, @@ -91,7 +91,7 @@ public PCollectionView, Iterable>> apply( * A {@PTransform} that produces a {@link PCollectionView} of a singleton {@link PCollection} * yielding the single element it contains. * - *

Instantiate via {@link View.asIterable}. + *

Instantiate via {@link View#asIterable}. */ public static class AsSingleton extends PTransform, PCollectionView>> { @@ -164,6 +164,7 @@ public SingletonPCollectionView(Pipeline pipeline) { setPipelineInternal(pipeline); } + @SuppressWarnings("unchecked") @Override public T fromIterableInternal(Iterable> contents) { try { @@ -189,6 +190,7 @@ public IterablePCollectionView(Pipeline pipeline) { @Override public Iterable fromIterableInternal(Iterable> contents) { return Iterables.transform(contents, new Function, T>() { + @SuppressWarnings("unchecked") @Override public T apply(WindowedValue input) { return (T) input.getValue(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java index ea7a22c8fc41a..91c591c901900 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java @@ -57,7 +57,7 @@ public static FixedWindows of(Duration size) { * [N * size + offset, (N + 1) * size + offset), * where 0 is the epoch. * - * @throws IllegalAgumentException if offset is not in [0, size) + * @throws IllegalArgumentException if offset is not in [0, size) */ public FixedWindows withOffset(Duration offset) { return new FixedWindows(size, offset); @@ -85,7 +85,7 @@ public Coder windowCoder() { } @Override - public boolean isCompatible(WindowingFn other) { + public boolean isCompatible(WindowingFn other) { return (other instanceof FixedWindows) && (size.equals(((FixedWindows) other).size)) && (offset.equals(((FixedWindows) other).offset)); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowingFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowingFn.java index 7ad7f29f6655a..53dc93a213945 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowingFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowingFn.java @@ -47,7 +47,7 @@ public String getCause() { public WindowingFn getOriginalWindowingFn() { return originalWindowingFn; } - + @Override public Collection assignWindows(AssignContext c) { throw new UnsupportedOperationException(); @@ -67,9 +67,9 @@ public Coder windowCoder() { * {@code InvalidWindowingFn} objects with the same {@code originalWindowingFn} are compatible. */ @Override - public boolean isCompatible(WindowingFn other) { + public boolean isCompatible(WindowingFn other) { return getClass() == other.getClass() && getOriginalWindowingFn().isCompatible( - ((InvalidWindowingFn) other).getOriginalWindowingFn()); + ((InvalidWindowingFn) other).getOriginalWindowingFn()); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java index 47f8a08005830..26744a549ba2a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java @@ -75,7 +75,7 @@ public Coder windowCoder() { } @Override - public boolean isCompatible(WindowingFn other) { + public boolean isCompatible(WindowingFn other) { return other instanceof Sessions; } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java index 6643289071ef5..d98dab908240c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java @@ -118,7 +118,7 @@ public Collection assignWindows(AssignContext c) { } @Override - public boolean isCompatible(WindowingFn other) { + public boolean isCompatible(WindowingFn other) { if (other instanceof SlidingWindows) { SlidingWindows that = (SlidingWindows) other; return period.equals(that.period) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java index 68796c908aba5..c450e0fa7980f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java @@ -224,10 +224,10 @@ public static Remerge remerge() { public static class Remerge extends PTransform, PCollection> { @Override public PCollection apply(PCollection input) { - WindowingFn windowingFn = getInput().getWindowingFn(); - WindowingFn outputWindowingFn = + WindowingFn windowingFn = getInput().getWindowingFn(); + WindowingFn outputWindowingFn = (windowingFn instanceof InvalidWindowingFn) - ? ((InvalidWindowingFn) windowingFn).getOriginalWindowingFn() + ? ((InvalidWindowingFn) windowingFn).getOriginalWindowingFn() : windowingFn; return input.apply(ParDo.named("Identity").of(new DoFn() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingFn.java index 0f049372555bb..935deb09f2b7c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingFn.java @@ -107,7 +107,7 @@ public abstract void merge(Collection toBeMerged, W mergeResult) * Returns whether this performs the same merging as the given * {@code WindowingFn}. */ - public abstract boolean isCompatible(WindowingFn other); + public abstract boolean isCompatible(WindowingFn other); /** * Returns the {@link Coder} used for serializing the windows used diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java index 975af472a4b6d..3a6d4af1c1a92 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java @@ -38,7 +38,7 @@ public class DoFnRunner { public interface OutputManager { /** Returns the receiver to use for a given tag. */ - public R initialize(TupleTag tag); + public R initialize(TupleTag tag); /** Outputs a single element to the provided receiver. */ public void output(R receiver, WindowedValue output); @@ -91,7 +91,7 @@ public static DoFnRunner createWithListOutputs( options, fn, sideInputs, new OutputManager() { @Override - public List initialize(TupleTag tag) { + public List initialize(TupleTag tag) { return new ArrayList<>(); } @Override @@ -141,7 +141,7 @@ public void finishBundle() { } /** Returns the receiver who gets outputs with the provided tag. */ - public R getReceiver(TupleTag tag) { + public R getReceiver(TupleTag tag) { return context.getReceiver(tag); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java index fc4b0886b7d5b..34ad2163a8a1e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java @@ -170,7 +170,7 @@ public Output apply( /** * Returns the {@link WindowingFn} of this {@code PCollection}. */ - public WindowingFn getWindowingFn() { + public WindowingFn getWindowingFn() { return windowingFn; } From cda8313c2d8b487d377c31dc502857d2bbde1825 Mon Sep 17 00:00:00 2001 From: malo Date: Sun, 14 Dec 2014 20:15:06 -0800 Subject: [PATCH 0008/1541] Set the sdk major_version to 0. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82113067 --- .../cloud/dataflow/sdk/runners/DataflowPipelineRunner.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index 9f770fcbb33cd..c13ef51c7c18a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -196,9 +196,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { // Requirements about the service. Map environmentVersion = new HashMap<>(); - // TODO: Specify the environment major version. - // environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, - // ENVIRONMENT_MAJOR_VERSION); + environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, ENVIRONMENT_MAJOR_VERSION); newJob.getEnvironment().setVersion(environmentVersion); // Default jobType is DATA_PARALLEL which is for java batch. String jobType = "DATA_PARALLEL"; From 22af2f3351a66296e930033790d9130b57de17ab Mon Sep 17 00:00:00 2001 From: fjp Date: Sun, 14 Dec 2014 22:08:30 -0800 Subject: [PATCH 0009/1541] Example Usability improvements: - print description when missing required options - document BigQuery dataset creation requirements - standardize example instructions [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82117381 --- .../dataflow/examples/BigQueryTornadoes.java | 24 ++++++++++- .../google/cloud/dataflow/examples/TfIdf.java | 43 ++++++------------- .../examples/TopWikipediaSessions.java | 27 ++++++------ .../cloud/dataflow/examples/WordCount.java | 19 ++++---- .../cloud/dataflow/sdk/io/BigQueryIO.java | 4 +- .../sdk/options/PipelineOptionsValidator.java | 15 ++++--- .../dataflow/BigQueryIOTranslator.java | 3 +- .../options/PipelineOptionsValidatorTest.java | 13 +++--- 8 files changed, 80 insertions(+), 68 deletions(-) diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/BigQueryTornadoes.java b/examples/src/main/java/com/google/cloud/dataflow/examples/BigQueryTornadoes.java index 43e94c08633b0..a5f9a19705218 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/BigQueryTornadoes.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/BigQueryTornadoes.java @@ -39,6 +39,26 @@ /** * An example that reads the public samples of weather data from BigQuery, counts the number of * tornadoes that occur in each month, and writes the results to BigQuery. + * + *

Concepts: Reading/writing BigQuery; counting a PCollection; user-defined PTransforms + * + *

Note: Before running this example, you must create a BigQuery dataset to contain your output + * table. + * + *

To execute this pipeline locally, specify general pipeline configuration: + * --project= + * and the BigQuery table for the output: + * --output=:. + * + *

To execute this pipeline using the Dataflow service, specify pipeline configuration: + * --project= + * --stagingLocation=gs:// + * --runner=BlockingDataflowPipelineRunner + * and the BigQuery table for the output: + * --output=:. + * + *

The BigQuery input table defaults to clouddataflow-readonly:samples.weather_stations and can + * be overridden with --input. */ public class BigQueryTornadoes { // Default to using a 1000 row subset of the public weather station table publicdata:samples.gsod. @@ -118,8 +138,8 @@ private static interface Options extends PipelineOptions { String getInput(); void setInput(String value); - @Description("Table to write to, specified as " - + ":.") + @Description("BigQuery table to write to, specified as " + + ":.. The dataset must already exist.") @Validation.Required String getOutput(); void setOutput(String value); diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java b/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java index a6bd4f27fd61d..15948782e2b60 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java @@ -60,39 +60,22 @@ /** * An example that computes a basic TF-IDF search table for a directory or GCS prefix. + * + *

Concepts: joining data; side inputs * - *

Command-line usage for this example: + *

To execute this pipeline locally, specify general pipeline configuration: + * --project= + * and a local output file or output prefix on GCS: + * --output=[ | gs://] * - *

- *     java com.google.cloud.dataflow.examples.TfIdf \
- *       --runner= \
- *       --input= \
- *       --output=
- * 
+ *

To execute this pipeline using the Dataflow service, specify pipeline configuration: + * --project= + * --stagingLocation=gs:// + * --runner=BlockingDataflowPipelineRunner + * and an output prefix on GCS: + * --output=gs:// * - *

For example, to execute this pipeline locally to index a local directory: - * - *

- *     java com.google.cloud.dataflow.examples.TfIdf \
- *       --runner=DirectPipelineRunner \
- *       --input= \
- *       --output=
- * 
- * - *

To execute this pipeline using the Dataflow service - * to index the works of Shakespeare and write the results to a GCS bucket: - * (For execution via the Dataflow service, only GCS locations are supported) - * - *

- *     java com.google.cloud.dataflow.examples.TfIdf \
- *       --project= \
- *       --stagingLocation=gs:// \
- *       --runner=BlockingDataflowPipelineRunner \
- *       [--input=gs://] \
- *       --output=gs://
- * 
- * - *

The default input is gs://dataflow-samples/shakespeare/ + *

The default input is gs://dataflow-samples/shakespeare/ and can be overridden with --input. */ public class TfIdf { /** diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java b/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java index baa520ea0447f..e4b2473c2f80e 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java @@ -45,24 +45,23 @@ import java.util.List; /** - * Pipeline that reads Wikipedia edit data from BigQuery and computes the user with + * An example that reads Wikipedia edit data from BigQuery and computes the user with * the longest string of edits separated by no more than an hour within each month. * - *

This pipeline demonstrates how the Windowing API can be used to perform - * various time-based aggregations of data. + *

Concepts: Using Windowing to perform time-based aggregations of data. + * + *

It is not recommended to execute this pipeline locally, given the size of the default input + * data. * - *

To run this pipeline, the following options must be provided: - *

{@code
+ * 

To execute this pipeline using the Dataflow service, specify pipeline configuration: * --project= - * --output=gs:// - * --stagingLocation=gs:// - * --runner=(Blocking)DataflowPipelineRunner - * }

- * - *

To run this example using Dataflow service, you must additionally - * provide either {@literal --stagingLocation} or {@literal --tempLocation}, and - * select one of the Dataflow pipeline runners, eg - * {@literal --runner=BlockingDataflowPipelineRunner}. + * --stagingLocation=gs:// + * --runner=BlockingDataflowPipelineRunner + * and an output prefix on GCS: + * --output=gs:// + * + *

The default input is gs://dataflow-samples/wikipedia_edits/*.json and can be overridden with + * --input. */ public class TopWikipediaSessions { private static final String EXPORTED_WIKI_TABLE = "gs://dataflow-samples/wikipedia_edits/*.json"; diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java b/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java index 4295e7868d26e..4fe304925c0b6 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java @@ -38,19 +38,22 @@ * An example that counts words in Shakespeare. For a detailed walkthrough of this * example see: * https://developers.google.com/cloud-dataflow/java-sdk/wordcount-example + * + *

Concepts: Reading/writing text files; counting a PCollection; user-defined PTransforms * - * To execute this pipeline locally, specify general pipeline configuration: + *

To execute this pipeline locally, specify general pipeline configuration: * --project= - * and example configuration: - * --output=[ | gs://] + * and a local output file or output prefix on GCS: + * --output=[ | gs://] * - * To execute this pipeline using the Dataflow service, specify pipeline configuration: - * --project= --stagingLocation=gs:// + *

To execute this pipeline using the Dataflow service, specify pipeline configuration: + * --project= + * --stagingLocation=gs:// * --runner=BlockingDataflowPipelineRunner - * and example configuration: - * --output=gs:// + * and an output prefix on GCS: + * --output=gs:// * - * The input file defaults to gs://dataflow-samples/shakespeare/kinglear.txt and can be + *

The input file defaults to gs://dataflow-samples/shakespeare/kinglear.txt and can be * overridden with --input. */ public class WordCount { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java index 2fffe4de2c45b..37e2963035e64 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java @@ -106,7 +106,7 @@ *

* See {@link BigQueryIO.Write} for details on how to specify if a write should * append to an existing table, replace the table, or verify that the table is - * empty. + * empty. Note that the dataset being written to must already exist. * * @see TableRow */ @@ -349,6 +349,8 @@ public boolean getValidate() { * provided (via {@link Write#withSchema}), or else the transform may fail * at runtime with an {@link java.lang.IllegalArgumentException}. *

+ * The dataset being written must already exist. + *

* By default, writes require an empty table, which corresponds to * a {@code WriteDisposition.WRITE_EMPTY} disposition which matches the * default of BigQuery's Jobs API. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java index bb7bcf3de831f..046cfd7fda05e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java @@ -18,7 +18,6 @@ import com.google.common.base.Preconditions; -import java.lang.annotation.Annotation; import java.lang.reflect.Method; import java.lang.reflect.Proxy; @@ -47,13 +46,17 @@ public static T validate(Class klass, PipelineOpt ProxyInvocationHandler handler = (ProxyInvocationHandler) Proxy.getInvocationHandler(options); for (Method method : PipelineOptionsFactory.getClosureOfMethodsOnInterface(klass)) { - for (Annotation annotation : method.getAnnotations()) { - if (annotation instanceof Validation.Required) { - Preconditions.checkArgument(handler.invoke(options, method, null) != null, - "Expected non-null property to be set for [" + method + "]."); - } + if (method.getAnnotation(Validation.Required.class) != null) { + Preconditions.checkArgument(handler.invoke(options, method, null) != null, + "Missing required value for [" + method + ", \"" + getDescription(method) + "\"]. "); } } return options.as(klass); } + + private static String getDescription(Method method) { + Description description = method.getAnnotation(Description.class); + return description == null ? "" : description.value(); + } } + diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BigQueryIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BigQueryIOTranslator.java index fd2731949c414..b3d0c19977db2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BigQueryIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BigQueryIOTranslator.java @@ -150,7 +150,8 @@ private static void verifyDatasetPresence( ApiErrorExtractor errorExtractor = new ApiErrorExtractor(); if (errorExtractor.itemNotFound(e)) { throw new IllegalArgumentException( - "BigQuery dataset not found for table: " + BigQueryIO.toTableSpec(table), e); + "BigQuery dataset not found for table \"" + BigQueryIO.toTableSpec(table) + + "\". Please create the dataset before pipeline execution."); } else { throw new RuntimeException( "unable to confirm BigQuery dataset presence", e); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidatorTest.java index e0decb9f92255..ad584ea32aeb4 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidatorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidatorTest.java @@ -30,6 +30,7 @@ public class PipelineOptionsValidatorTest { /** A test interface with an {@link Validation.Required} annotation. */ public static interface Required extends PipelineOptions { @Validation.Required + @Description("Fake Description") public String getObject(); public void setObject(String value); } @@ -44,9 +45,9 @@ public void testWhenRequiredOptionIsSet() { @Test public void testWhenRequiredOptionIsSetAndCleared() { expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Expected non-null property to be set for " + expectedException.expectMessage("Missing required value for " + "[public abstract java.lang.String com.google.cloud.dataflow." - + "sdk.options.PipelineOptionsValidatorTest$Required.getObject()]."); + + "sdk.options.PipelineOptionsValidatorTest$Required.getObject(), \"Fake Description\"]."); Required required = PipelineOptionsFactory.as(Required.class); required.setObject("blah"); @@ -57,9 +58,9 @@ public void testWhenRequiredOptionIsSetAndCleared() { @Test public void testWhenRequiredOptionIsNeverSet() { expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Expected non-null property to be set for " + expectedException.expectMessage("Missing required value for " + "[public abstract java.lang.String com.google.cloud.dataflow." - + "sdk.options.PipelineOptionsValidatorTest$Required.getObject()]."); + + "sdk.options.PipelineOptionsValidatorTest$Required.getObject(), \"Fake Description\"]."); Required required = PipelineOptionsFactory.as(Required.class); PipelineOptionsValidator.validate(Required.class, required); @@ -76,9 +77,9 @@ public static interface SubClassValidation extends Required { @Test public void testValidationOnOverriddenMethods() throws Exception { expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Expected non-null property to be set for " + expectedException.expectMessage("Missing required value for " + "[public abstract java.lang.String com.google.cloud.dataflow." - + "sdk.options.PipelineOptionsValidatorTest$Required.getObject()]."); + + "sdk.options.PipelineOptionsValidatorTest$Required.getObject(), \"Fake Description\"]."); SubClassValidation required = PipelineOptionsFactory.as(SubClassValidation.class); PipelineOptionsValidator.validate(Required.class, required); From 5581b9fe032c9dedc3a11c595a46312bbfbdc8d3 Mon Sep 17 00:00:00 2001 From: fjp Date: Sun, 14 Dec 2014 23:05:30 -0800 Subject: [PATCH 0010/1541] Add an easy option for configuring the autoscaling algorithm. Leave the default as NONE, but now basic, experimental autoscaling can be enabled with: --maxNumWorkers=20 --autoscalingAlgorithm=BASIC [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82119376 --- .../examples/TopWikipediaSessions.java | 10 ++++- .../DataflowPipelineWorkerPoolOptions.java | 45 +++++++++++++++++++ .../runners/DataflowPipelineTranslator.java | 8 ++++ 3 files changed, 61 insertions(+), 2 deletions(-) diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java b/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java index e4b2473c2f80e..3d945b6394acb 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java @@ -45,7 +45,7 @@ import java.util.List; /** - * An example that reads Wikipedia edit data from BigQuery and computes the user with + * An example that reads Wikipedia edit data from Cloud Storage and computes the user with * the longest string of edits separated by no more than an hour within each month. * *

Concepts: Using Windowing to perform time-based aggregations of data. @@ -59,9 +59,15 @@ * --runner=BlockingDataflowPipelineRunner * and an output prefix on GCS: * --output=gs:// - * + * *

The default input is gs://dataflow-samples/wikipedia_edits/*.json and can be overridden with * --input. + * + *

The input for this example is large enough that it's a good place to enable (experimental) + * autoscaling: + * --autoscalingAlgorithm=BASIC + * --maxNumWorkers=20 + * This will automatically scale the number of workers up over time until the job completes. */ public class TopWikipediaSessions { private static final String EXPORTED_WIKI_TABLE = "gs://dataflow-samples/wikipedia_edits/*.json"; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java index 6cd9839318630..b388da4313ee7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java @@ -37,6 +37,51 @@ public interface DataflowPipelineWorkerPoolOptions { @Default.Integer(3) int getNumWorkers(); void setNumWorkers(int value); + + /** + * Type of autoscaling algorithm to use. These types are experimental and subject to change. + */ + public enum AutoscalingAlgorithmType { + /** Use numWorkers machines. Do not autoscale the worker pool. */ + NONE("AUTOSCALING_ALGORITHM_NONE"), + + /** Autoscale the workerpool size up to maxNumWorkers until the job completes. */ + BASIC("AUTOSCALING_ALGORITHM_BASIC"); + + private final String algorithm; + + private AutoscalingAlgorithmType(String algorithm) { + this.algorithm = algorithm; + } + + /** Returns the string representation of this type. */ + public String getAlgorithm() { + return this.algorithm; + } + } + + @Description("(experimental) The autoscaling algorithm to use for the workerpool.") + @Default.InstanceFactory(AutoscalingAlgorithmTypeFactory.class) + AutoscalingAlgorithmType getAutoscalingAlgorithm(); + void setAutoscalingAlgorithm(AutoscalingAlgorithmType value); + + /** Returns the default NONE AutoscalingAlgorithmType. */ + public static class AutoscalingAlgorithmTypeFactory implements + DefaultValueFactory { + @Override + public AutoscalingAlgorithmType create(PipelineOptions options) { + return AutoscalingAlgorithmType.NONE; + } + } + + /** + * Max number of workers to use when using workerpool autoscaling. + * This option is experimental and subject to change. + */ + @Description("Max number of workers to use, when using autoscaling") + @Default.Integer(20) + int getMaxNumWorkers(); + void setMaxNumWorkers(int value); /** * Remote worker disk size, in gigabytes, or 0 to use the default size. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index b3b24269c365a..823b9d44a1f6e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -29,6 +29,7 @@ import static com.google.cloud.dataflow.sdk.util.Structs.getString; import com.google.api.client.util.Preconditions; +import com.google.api.services.dataflow.model.AutoscalingSettings; import com.google.api.services.dataflow.model.DataflowPackage; import com.google.api.services.dataflow.model.Disk; import com.google.api.services.dataflow.model.Environment; @@ -48,6 +49,7 @@ import com.google.cloud.dataflow.sdk.io.PubsubIO; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType; import com.google.cloud.dataflow.sdk.runners.dataflow.AvroIOTranslator; import com.google.cloud.dataflow.sdk.runners.dataflow.BigQueryIOTranslator; import com.google.cloud.dataflow.sdk.runners.dataflow.DatastoreIOTranslator; @@ -394,6 +396,12 @@ public Job translate(List packages) { if (options.getDiskSizeGb() > 0) { workerPool.setDiskSizeGb(options.getDiskSizeGb()); } + if (!options.getAutoscalingAlgorithm().equals(AutoscalingAlgorithmType.NONE)) { + AutoscalingSettings settings = new AutoscalingSettings(); + settings.setAlgorithm(options.getAutoscalingAlgorithm().getAlgorithm()); + settings.setMaxNumWorkers(options.getMaxNumWorkers()); + workerPool.setAutoscalingSettings(settings); + } List workerPools = new LinkedList<>(); From 3aa23436ed57daab40b2a3ce28c2c6accf8b9aff Mon Sep 17 00:00:00 2001 From: earhart Date: Sun, 14 Dec 2014 23:37:39 -0800 Subject: [PATCH 0011/1541] Fixing a few more warnings. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82120439 --- .../java/com/google/cloud/dataflow/sdk/Pipeline.java | 12 ++++++------ .../dataflow/sdk/coders/BigEndianIntegerCoder.java | 1 + .../dataflow/sdk/coders/BigEndianLongCoder.java | 1 + .../cloud/dataflow/sdk/coders/DoubleCoder.java | 1 + .../cloud/dataflow/sdk/coders/EntityCoder.java | 1 + .../cloud/dataflow/sdk/coders/InstantCoder.java | 1 + .../google/cloud/dataflow/sdk/coders/KvCoder.java | 1 + .../cloud/dataflow/sdk/coders/StringUtf8Coder.java | 1 + .../cloud/dataflow/sdk/coders/VarIntCoder.java | 1 + .../cloud/dataflow/sdk/coders/VarLongCoder.java | 1 + .../com/google/cloud/dataflow/sdk/io/PubsubIO.java | 2 ++ .../dataflow/sdk/options/PipelineOptionsFactory.java | 2 ++ .../dataflow/sdk/runners/DirectPipelineRunner.java | 5 +++-- .../dataflow/sdk/runners/worker/CombineValuesFn.java | 1 + .../cloud/dataflow/sdk/transforms/Combine.java | 1 + .../google/cloud/dataflow/sdk/transforms/Count.java | 2 ++ .../google/cloud/dataflow/sdk/transforms/DoFn.java | 1 + .../cloud/dataflow/sdk/transforms/Flatten.java | 2 ++ .../cloud/dataflow/sdk/transforms/GroupByKey.java | 1 + .../google/cloud/dataflow/sdk/transforms/Keys.java | 1 + .../google/cloud/dataflow/sdk/transforms/KvSwap.java | 1 + .../google/cloud/dataflow/sdk/transforms/Max.java | 4 ++++ .../google/cloud/dataflow/sdk/transforms/ParDo.java | 2 ++ .../cloud/dataflow/sdk/transforms/RateLimiting.java | 1 + .../google/cloud/dataflow/sdk/transforms/Sum.java | 1 + .../google/cloud/dataflow/sdk/transforms/Top.java | 4 ++++ .../google/cloud/dataflow/sdk/transforms/Values.java | 1 + .../cloud/dataflow/sdk/transforms/WithKeys.java | 1 + .../dataflow/sdk/transforms/join/CoGbkResult.java | 1 + .../sdk/transforms/join/CoGbkResultSchema.java | 1 + .../dataflow/sdk/transforms/join/CoGroupByKey.java | 1 + .../dataflow/sdk/transforms/join/UnionCoder.java | 1 + .../sdk/transforms/windowing/GlobalWindow.java | 1 + .../sdk/transforms/windowing/IntervalWindow.java | 2 ++ .../sdk/transforms/windowing/SlidingWindows.java | 1 + .../dataflow/sdk/transforms/windowing/Window.java | 2 ++ .../google/cloud/dataflow/sdk/util/CoderUtils.java | 1 + .../google/cloud/dataflow/sdk/util/DoFnContext.java | 2 +- .../sdk/util/StreamingGroupAlsoByWindowsDoFn.java | 1 + .../cloud/dataflow/sdk/util/TimerOrElement.java | 1 + .../cloud/dataflow/sdk/util/WindowedValue.java | 8 ++++---- .../cloud/dataflow/sdk/values/CodedTupleTag.java | 1 + .../com/google/cloud/dataflow/sdk/values/KV.java | 4 ++++ .../cloud/dataflow/sdk/coders/AvroCoderTest.java | 1 + .../cloud/dataflow/sdk/coders/CoderRegistryTest.java | 1 + .../cloud/dataflow/sdk/coders/CustomCoderTest.java | 1 + .../cloud/dataflow/sdk/coders/DefaultCoderTest.java | 1 + .../dataflow/sdk/coders/SerializableCoderTest.java | 1 + .../sdk/runners/DataflowPipelineRunnerTest.java | 1 + .../sdk/runners/DataflowPipelineTranslatorTest.java | 1 + .../dataflow/sdk/runners/TransformTreeTest.java | 1 + .../sdk/runners/worker/CombineValuesFnTest.java | 1 + .../runners/worker/MapTaskExecutorFactoryTest.java | 1 + .../sdk/runners/worker/NormalParDoFnTest.java | 1 + .../sdk/runners/worker/ParDoFnFactoryTest.java | 1 + .../sdk/transforms/ApproximateQuantilesTest.java | 1 + .../sdk/transforms/ApproximateUniqueTest.java | 1 + .../cloud/dataflow/sdk/transforms/CombineTest.java | 1 + .../cloud/dataflow/sdk/transforms/CreateTest.java | 1 + .../cloud/dataflow/sdk/transforms/FirstTest.java | 1 + .../dataflow/sdk/transforms/GroupByKeyTest.java | 1 + .../cloud/dataflow/sdk/transforms/ParDoTest.java | 1 + .../cloud/dataflow/sdk/transforms/PartitionTest.java | 1 + .../dataflow/sdk/transforms/RateLimitingTest.java | 1 + .../cloud/dataflow/sdk/transforms/SampleTest.java | 1 + .../cloud/dataflow/sdk/transforms/TopTest.java | 1 + .../cloud/dataflow/sdk/transforms/ViewTest.java | 1 + .../cloud/dataflow/sdk/transforms/WithKeysTest.java | 1 + .../sdk/transforms/join/CoGroupByKeyTest.java | 1 + .../sdk/transforms/windowing/WindowingTest.java | 1 + .../cloud/dataflow/sdk/util/AggregatorImplTest.java | 1 + .../cloud/dataflow/sdk/util/CoderUtilsTest.java | 1 + .../google/cloud/dataflow/sdk/util/PTupleTest.java | 1 + .../dataflow/sdk/util/SerializableUtilsTest.java | 1 + .../google/cloud/dataflow/sdk/values/PDoneTest.java | 1 + 75 files changed, 101 insertions(+), 13 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java index ec67fd7aabc3f..df7411f6521b6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java @@ -282,10 +282,10 @@ Output applyInternal(Input input, * Returns all producing transforms for the {@link PValue}s contained * in {@code output}. */ - private List getProducingTransforms(POutput output) { - List producingTransforms = new ArrayList<>(); + private List> getProducingTransforms(POutput output) { + List> producingTransforms = new ArrayList<>(); for (PValue value : output.expand()) { - PTransform transform = value.getProducingTransformInternal(); + PTransform transform = value.getProducingTransformInternal(); if (transform != null) { producingTransforms.add(transform); } @@ -301,9 +301,9 @@ private List getProducingTransforms(POutput output) { */ private void verifyOutputState(POutput output, TransformTreeNode node) { if (!node.isCompositeNode()) { - PTransform thisTransform = node.getTransform(); - List producingTransforms = getProducingTransforms(output); - for (PTransform producingTransform : producingTransforms) { + PTransform thisTransform = node.getTransform(); + List> producingTransforms = getProducingTransforms(output); + for (PTransform producingTransform : producingTransforms) { if (thisTransform != producingTransform) { throw new IllegalArgumentException("Output of non-composite transform " + thisTransform + " is registered as being produced by" diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoder.java index 6af2d6f5ac4ea..0f872f637673d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoder.java @@ -29,6 +29,7 @@ /** * A BigEndianIntegerCoder encodes Integers in 4 bytes, big-endian. */ +@SuppressWarnings("serial") public class BigEndianIntegerCoder extends AtomicCoder { @JsonCreator public static BigEndianIntegerCoder of() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoder.java index 43ee9cab34be5..f9e8d3ea34ee3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoder.java @@ -29,6 +29,7 @@ /** * A BigEndianLongCoder encodes Longs in 8 bytes, big-endian. */ +@SuppressWarnings("serial") public class BigEndianLongCoder extends AtomicCoder { @JsonCreator public static BigEndianLongCoder of() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java index 6b531ad0dc454..1726ea9ed127d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java @@ -29,6 +29,7 @@ /** * A DoubleCoder encodes Doubles in 8 bytes. */ +@SuppressWarnings("serial") public class DoubleCoder extends AtomicCoder { @JsonCreator public static DoubleCoder of() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java index 988a04c03160c..da9769303b702 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java @@ -27,6 +27,7 @@ /** * An EntityCoder encodes/decodes Datastore Entity objects. */ +@SuppressWarnings("serial") public class EntityCoder extends AtomicCoder { @JsonCreator diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java index 3190124391703..5918eb220aef6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java @@ -26,6 +26,7 @@ /** * A InstantCoder encodes joda Instant. */ +@SuppressWarnings("serial") public class InstantCoder extends AtomicCoder { @JsonCreator public static InstantCoder of() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java index 000d6ca75807a..bf6e7c9d78892 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java @@ -39,6 +39,7 @@ * @param the type of the keys of the KVs being transcoded * @param the type of the values of the KVs being transcoded */ +@SuppressWarnings("serial") public class KvCoder extends KvCoderBase> { public static KvCoder of(Coder keyCoder, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java index 17995c31b65be..94db9e4ed3899 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java @@ -35,6 +35,7 @@ * A StringUtf8Coder encodes Java Strings in UTF-8 encoding. * If in a nested context, prefixes the string with a VarInt length field. */ +@SuppressWarnings("serial") public class StringUtf8Coder extends AtomicCoder { @JsonCreator public static StringUtf8Coder of() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java index eff03fb737324..2ce9ffd632e0f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java @@ -31,6 +31,7 @@ * numbers always take 5 bytes, so BigEndianIntegerCoder may be preferable for * ints that are known to often be large or negative. */ +@SuppressWarnings("serial") public class VarIntCoder extends AtomicCoder { @JsonCreator public static VarIntCoder of() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java index 74f9b6092288f..50866383bca2a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java @@ -31,6 +31,7 @@ * numbers always take 10 bytes, so BigEndianLongCoder may be preferable for * longs that are known to often be large or negative. */ +@SuppressWarnings("serial") public class VarLongCoder extends AtomicCoder { @JsonCreator public static VarLongCoder of() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index b9f0514841590..75e17bed55264 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -181,6 +181,7 @@ public static Bound subscription(String subscription) { * A PTransform that reads from a PubSub source and returns * a unbounded PCollection containing the items from the stream. */ + @SuppressWarnings("serial") public static class Bound extends PTransform> { /** The Pubsub topic to read from. */ @@ -278,6 +279,7 @@ public static Bound topic(String topic) { * A PTransfrom that writes a unbounded {@code PCollection} * to a PubSub stream. */ + @SuppressWarnings("serial") public static class Bound extends PTransform, PDone> { /** The Pubsub topic to publish to. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java index 89a31b07e888a..9ff9280803fa6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java @@ -280,6 +280,7 @@ Class getProxyClass() { private static final Logger LOG = LoggerFactory.getLogger(PipelineOptionsFactory.class); + @SuppressWarnings("rawtypes") private static final Class[] EMPTY_CLASS_ARRAY = new Class[0]; private static final ObjectMapper MAPPER = new ObjectMapper(); @@ -404,6 +405,7 @@ static synchronized Registration validateWellForm // Validate that the local view of the class is well formed. if (!INTERFACE_CACHE.containsKey(iface)) { + @SuppressWarnings("rawtypes") Class proxyClass = Proxy.getProxyClass( PipelineOptionsFactory.class.getClassLoader(), new Class[] {iface}); try { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java index cc1fd8f38cdae..2b0ca03beaab0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java @@ -232,6 +232,7 @@ private PCollection> applyTestCombine( * question. */ // @VisibleForTesting + @SuppressWarnings("serial") public static class TestCombineDoFn extends DoFn>, KV> { private final KeyedCombineFn fn; @@ -321,7 +322,7 @@ public T ensureSerializableByCoder( @Override public EvaluationResults run(Pipeline pipeline) { LOG.info("Executing pipeline using the DirectPipelineRunner."); - + Evaluator evaluator = new Evaluator(); evaluator.run(pipeline); @@ -329,7 +330,7 @@ public EvaluationResults run(Pipeline pipeline) { for (Counter counter : evaluator.getCounters()) { LOG.info("Final aggregator value: {}", counter); } - + LOG.info("Pipeline execution complete."); return evaluator; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java index 16230571fae12..62a371d05271f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java @@ -42,6 +42,7 @@ /** * A wrapper around a decoded user value combining function. */ +@SuppressWarnings("serial") public class CombineValuesFn extends NormalParDoFn { /** * The optimizer may split run the user combiner in 3 separate diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java index 6746f9e75672b..f455d5763ac1a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -38,6 +38,7 @@ * {@code PTransform}s for combining {@code PCollection} elements * globally and per-key. */ +@SuppressWarnings("serial") public class Combine { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java index 1303b0a98634a..29336d859ced8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java @@ -88,6 +88,7 @@ public static PerElement perElement() { * * @param the type of the elements of the input {@code PCollection} */ + @SuppressWarnings("serial") public static class Globally extends PTransform, PCollection> { @@ -136,6 +137,7 @@ public void processElement(ProcessContext c) { * @param the type of the elements of the input {@code PCollection}, and * the type of the keys of the output {@code PCollection} */ + @SuppressWarnings("serial") public static class PerElement extends PTransform, PCollection>> { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java index c9935fc25810b..c18a646e4278a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java @@ -52,6 +52,7 @@ * @param the type of the (main) input elements * @param the type of the (main) output elements */ +@SuppressWarnings("serial") public abstract class DoFn implements Serializable { /** Information accessible to all methods in this {@code DoFn}. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java index c52ae265fd5f2..3ef5e3a369a30 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java @@ -105,6 +105,7 @@ public static FlattenIterables iterables() { * @param the type of the elements in the input and output * {@code PCollection}s. */ + @SuppressWarnings("serial") public static class FlattenPCollectionList extends PTransform, PCollection> { @@ -150,6 +151,7 @@ protected Coder getDefaultOutputCoder() { * @param the type of the elements of the input {@code Iterable}s and * the output {@code PCollection} */ + @SuppressWarnings("serial") public static class FlattenIterables extends PTransform>, PCollection> { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index 54e60cfcb6209..72e475f31002a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -116,6 +116,7 @@ * and the elements of the {@code Iterable}s in the output * {@code PCollection} */ +@SuppressWarnings("serial") public class GroupByKey extends PTransform>, PCollection>>> { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Keys.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Keys.java index 08a801b15ec2d..0e78f4ad1e840 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Keys.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Keys.java @@ -40,6 +40,7 @@ * @param the type of the keys in the input {@code PCollection}, * and the type of the elements in the output {@code PCollection} */ +@SuppressWarnings("serial") public class Keys extends PTransform>, PCollection> { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/KvSwap.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/KvSwap.java index ee73ae4087f5d..4b12d5db08c28 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/KvSwap.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/KvSwap.java @@ -42,6 +42,7 @@ * @param the type of the values in the input {@code PCollection} * and the keys in the output {@code PCollection} */ +@SuppressWarnings("serial") public class KvSwap extends PTransform>, PCollection>> { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java index fce9a328f1c84..fc901664be5a1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java @@ -139,6 +139,7 @@ public static Combine.PerKey doublesPerKey() { * * @param the type of the {@code Number}s being compared */ + @SuppressWarnings("serial") public static class MaxFn> implements SerializableFunction, N> { @@ -172,6 +173,7 @@ public N apply(Iterable input) { * {@code Iterable} of {@code Integer}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ + @SuppressWarnings("serial") public static class MaxIntegerFn extends MaxFn { public MaxIntegerFn() { super(Integer.MIN_VALUE); } } @@ -181,6 +183,7 @@ public static class MaxIntegerFn extends MaxFn { * {@code Iterable} of {@code Long}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ + @SuppressWarnings("serial") public static class MaxLongFn extends MaxFn { public MaxLongFn() { super(Long.MIN_VALUE); } } @@ -190,6 +193,7 @@ public static class MaxLongFn extends MaxFn { * {@code Iterable} of {@code Double}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ + @SuppressWarnings("serial") public static class MaxDoubleFn extends MaxFn { public MaxDoubleFn() { super(Double.MIN_VALUE); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java index c7d925b2b418b..fad1235de0b90 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java @@ -638,6 +638,7 @@ public Bound of(DoFn fn) { * @param the type of the (main) input {@code PCollection} elements * @param the type of the (main) output {@code PCollection} elements */ + @SuppressWarnings("serial") public static class Bound extends PTransform, PCollection> { // Inherits name. @@ -831,6 +832,7 @@ public BoundMulti of(DoFn fn) { * @param the type of the (main) input {@code PCollection} elements * @param the type of the main output {@code PCollection} elements */ + @SuppressWarnings("serial") public static class BoundMulti extends PTransform, PCollectionTuple> { // Inherits name. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java index 2124acfbb84a5..fd32303613bcb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java @@ -87,6 +87,7 @@ public static RateLimitingTransform perWorker(DoFn doFn) { * @param the type of the (main) input elements * @param the type of the (main) output elements */ + @SuppressWarnings("serial") public static class RateLimitingTransform extends PTransform, PCollection> { private final DoFn doFn; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java index e925e4a5cc90d..ce62e40b4580a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java @@ -35,6 +35,7 @@ * .apply(Sum.integersPerKey()); * } */ +@SuppressWarnings("serial") public class Sum { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java index 2e5e334b8a73d..0f8a3f0565850 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java @@ -321,6 +321,7 @@ PTransform, PCollection>> largest(int count) { * * @param type of element being compared */ + @SuppressWarnings("serial") public static class TopCombineFn extends AccumulatingCombineFn.Heap, List> { @@ -423,6 +424,7 @@ public Coder getAccumulatorCoder( return new HeapCoder(inputCoder); } + @SuppressWarnings("serial") private class HeapCoder extends CustomCoder { private final Coder> listCoder; @@ -467,6 +469,7 @@ public void registerByteSizeObserver( * {@code Serializable} {@code Comparator} that that uses the * compared elements' natural ordering. */ + @SuppressWarnings("serial") public static class Largest> implements Comparator, Serializable { @Override @@ -479,6 +482,7 @@ public int compare(T a, T b) { * {@code Serializable} {@code Comparator} that that uses the * reverse of the compared elements' natural ordering. */ + @SuppressWarnings("serial") public static class Smallest> implements Comparator, Serializable { @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Values.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Values.java index ae008b196ad3b..4ca84e45c6501 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Values.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Values.java @@ -40,6 +40,7 @@ * @param the type of the values in the input {@code PCollection}, * and the type of the elements in the output {@code PCollection} */ +@SuppressWarnings("serial") public class Values extends PTransform>, PCollection> { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java index 1754c20a7916b..82ce93cdf3274 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java @@ -48,6 +48,7 @@ * {@code PCollection} and the values in the output * {@code PCollection} */ +@SuppressWarnings("serial") public class WithKeys extends PTransform, PCollection>> { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java index f91d7d2ca669c..cbacb5fec1385 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java @@ -145,6 +145,7 @@ public V getOnly(TupleTag tag, V defaultValue) { /** * A coder for CoGbkResults. */ + @SuppressWarnings("serial") public static class CoGbkResultCoder extends StandardCoder { private final CoGbkResultSchema schema; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultSchema.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultSchema.java index 93883b80750c8..6ab1042e3b186 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultSchema.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultSchema.java @@ -37,6 +37,7 @@ * between TupleTags and Union Tags (which are used as secondary keys in the * CoGroupByKey). */ +@SuppressWarnings("serial") class CoGbkResultSchema implements Serializable { private final TupleTagList tupleTagList; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKey.java index d81c9ef707ca7..bc0eed2c78de7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKey.java @@ -68,6 +68,7 @@ * @param the type of the keys in the input and output * {@code PCollection}s */ +@SuppressWarnings("serial") public class CoGroupByKey extends PTransform, PCollection>> { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java index a6bb4bcb45860..f0987dfaffee5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java @@ -34,6 +34,7 @@ /** * A UnionCoder encodes RawUnionValues. */ +@SuppressWarnings("serial") class UnionCoder extends StandardCoder { // TODO: Think about how to integrate this with a schema object (i.e. // a tuple of tuple tags). diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java index bfcb9c7fa1595..5d0773d598a52 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java @@ -29,6 +29,7 @@ /** * Default {@link WindowingFn} where all data is in the same bucket. */ +@SuppressWarnings("serial") public class GlobalWindow extends NonMergingWindowingFn { @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/IntervalWindow.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/IntervalWindow.java index 8ac23501c97e5..ab34ad5049a45 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/IntervalWindow.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/IntervalWindow.java @@ -178,6 +178,7 @@ public static Coder getFixedSizeCoder(final Duration size) { return FixedSizeIntervalWindowCoder.of(size); } + @SuppressWarnings("serial") private static class IntervalWindowCoder extends AtomicCoder { private static final IntervalWindowCoder INSTANCE = new IntervalWindowCoder(); @@ -209,6 +210,7 @@ public IntervalWindow decode(InputStream inStream, Context context) public boolean isDeterministic() { return true; } } + @SuppressWarnings("serial") private static class FixedSizeIntervalWindowCoder extends AtomicCoder { private static final Coder instantCoder = InstantCoder.of(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java index d98dab908240c..45b90dd33d1e4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java @@ -37,6 +37,7 @@ * Window.by(SlidingWindows.of(Duration.standardMinutes(10)))); * } */ +@SuppressWarnings("serial") public class SlidingWindows extends NonMergingWindowingFn { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java index c450e0fa7980f..424fdd7c02120 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java @@ -167,6 +167,7 @@ public Bound into(WindowingFn fn) { * * @param The type of elements this {@code Window} is applied to */ + @SuppressWarnings("serial") public static class Bound extends PTransform, PCollection> { WindowingFn fn; @@ -221,6 +222,7 @@ public static Remerge remerge() { * windows to be merged again as part of the next * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}. */ + @SuppressWarnings("serial") public static class Remerge extends PTransform, PCollection> { @Override public PCollection apply(PCollection input) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CoderUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CoderUtils.java index c77f35a45da2a..5dbfc2d4ef6ed 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CoderUtils.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CoderUtils.java @@ -112,6 +112,7 @@ public static CloudObject makeCloudEncoding( * A {@link com.fasterxml.jackson.databind.module.Module} which adds the type * resolver needed for Coder definitions created by the Dataflow service. */ + @SuppressWarnings("serial") static final class Jackson2Module extends SimpleModule { /** * The Coder custom type resolver. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java index 80d8f34edd04d..08d53a7d464b6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java @@ -52,7 +52,7 @@ class DoFnContext extends DoFn.Context { final DoFn fn; final PTuple sideInputs; final OutputManager outputManager; - final Map outputMap; + final Map, R> outputMap; final TupleTag mainOutputTag; final StepContext stepContext; final CounterSet.AddCounterMutator addCounterMutator; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java index dcfd58aee92d2..61911a9decd4c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java @@ -33,6 +33,7 @@ * @param output value element type * @param window type */ +@SuppressWarnings("serial") public class StreamingGroupAlsoByWindowsDoFn extends DoFn>, KV> implements DoFn.RequiresKeyedState { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerOrElement.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerOrElement.java index 4859f8ae5f39b..d0216d3075782 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerOrElement.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerOrElement.java @@ -109,6 +109,7 @@ public E element() { * {@code TimerOrElement} objects never need to be encoded, so this class does not * support the {@code encode} and {@code decode} methods. */ + @SuppressWarnings("serial") public static class TimerOrElementCoder extends StandardCoder> { final Coder elemCoder; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java index 636b77d8b1109..d47e810fa9909 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java @@ -37,9 +37,9 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Objects; @@ -79,9 +79,9 @@ public static WindowedValue valueInGlobalWindow(V value) { * Returns a {@code WindowedValue} with the given value and default timestamp and empty windows. */ public static WindowedValue valueInEmptyWindows(V value) { - return new WindowedValue(value, - new Instant(Long.MIN_VALUE), - new ArrayList<>()); + return new WindowedValue(value, + new Instant(Long.MIN_VALUE), + Collections.emptyList()); } private WindowedValue(V value, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTag.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTag.java index 3caed1a8bcce9..770d460249339 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTag.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTag.java @@ -27,6 +27,7 @@ * * @param the type of the values associated with this tag */ +@SuppressWarnings("serial") public class CodedTupleTag extends TupleTag { /** * Returns a {@code CodedTupleTag} with the given id which uses the diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java index d354707ebb0c0..febb1302bd8c1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java @@ -32,6 +32,8 @@ * @param the type of the value */ public class KV implements Serializable { + private static final long serialVersionUID = 0; + /** Returns a KV with the given key and value. */ public static KV of(K key, V value) { return new KV<>(key, value); @@ -74,6 +76,7 @@ public boolean equals(Object o) { } /** Orders the KV by the key. A null key is less than any non-null key. */ + @SuppressWarnings("serial") public static class OrderByKey, V> implements Comparator>, Serializable { @Override @@ -89,6 +92,7 @@ public int compare(KV a, KV b) { } /** Orders the KV by the value. A null value is less than any non-null value. */ + @SuppressWarnings("serial") public static class OrderByValue> implements Comparator>, Serializable { @Override diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java index 725c0e852022d..fffbb5839e7ca 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java @@ -43,6 +43,7 @@ * Tests for AvroCoder. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class AvroCoderTest { @DefaultCoder(AvroCoder.class) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java index ace3094827335..a05816c776886 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java @@ -43,6 +43,7 @@ * Tests for CoderRegistry. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class CoderRegistryTest { public static CoderRegistry getStandardRegistry() { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CustomCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CustomCoderTest.java index e532d44dc66b5..c71a50843fd02 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CustomCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CustomCoderTest.java @@ -33,6 +33,7 @@ /** Unit tests for {@link CustomCoder}. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class CustomCoderTest { private static class MyCustomCoder extends CustomCoder> { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/DefaultCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/DefaultCoderTest.java index 769d1e6fb144f..dc0d20c4f8a53 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/DefaultCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/DefaultCoderTest.java @@ -31,6 +31,7 @@ * Tests of Coder defaults. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class DefaultCoderTest { @DefaultCoder(AvroCoder.class) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/SerializableCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/SerializableCoderTest.java index 3e56832a3faad..d8a7e03092518 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/SerializableCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/SerializableCoderTest.java @@ -46,6 +46,7 @@ * Tests SerializableCoder. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class SerializableCoderTest implements Serializable { @DefaultCoder(SerializableCoder.class) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java index 7995445c9869b..3d6f804ed3a21 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java @@ -74,6 +74,7 @@ * Tests for DataflowPipelineRunner. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class DataflowPipelineRunnerTest { @Rule public ExpectedException thrown = ExpectedException.none(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java index 228dbfd73acf7..12649de8acfca 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java @@ -76,6 +76,7 @@ * Tests for DataflowPipelineTranslator. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class DataflowPipelineTranslatorTest { @Rule public ExpectedException thrown = ExpectedException.none(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/TransformTreeTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/TransformTreeTest.java index d0308e87a33f5..cceefe90f622e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/TransformTreeTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/TransformTreeTest.java @@ -47,6 +47,7 @@ * Tests for {@link TransformTreeNode} and {@link TransformHierarchy}. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class TransformTreeTest { enum TransformsSeen { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java index b616f6d75f372..160b5456d6193 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java @@ -57,6 +57,7 @@ * Tests for CombineValuesFn. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class CombineValuesFnTest { /** Example AccumulatingCombineFn. */ public static class MeanInts extends diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java index fae22797ef89c..2060865759331 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java @@ -76,6 +76,7 @@ * Tests for MapTaskExecutorFactory. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class MapTaskExecutorFactoryTest { @Test public void testCreateMapTaskExecutor() throws Exception { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java index f94ab8339f9d1..2c6811c27b5ea 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java @@ -51,6 +51,7 @@ * Tests for NormalParDoFn. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class NormalParDoFnTest { static class TestDoFn extends DoFn { enum State { UNSTARTED, STARTED, PROCESSING, FINISHED } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java index 05a3864d9bd4d..63d38ebdd4109 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java @@ -43,6 +43,7 @@ * Tests for ParDoFnFactory. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class ParDoFnFactoryTest { static class TestDoFn extends DoFn { final String stringState; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java index b0493491634c2..406fb0730d023 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java @@ -48,6 +48,7 @@ * Tests for ApproximateQuantiles */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class ApproximateQuantilesTest { static final List> TABLE = Arrays.asList( diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java index 2b2ff0ac9c967..ce357ca7e2b76 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java @@ -52,6 +52,7 @@ * Tests for the ApproximateUnique aggregator transform. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class ApproximateUniqueTest { @Test diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java index 52b0b230a19d1..646977441e03b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java @@ -68,6 +68,7 @@ * Tests for Combine transforms. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class CombineTest { @SuppressWarnings("unchecked") diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CreateTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CreateTest.java index 8202da086240c..2d2615c8a5eaf 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CreateTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CreateTest.java @@ -47,6 +47,7 @@ * Tests for Create. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class CreateTest { @Rule public final ExpectedException thrown = ExpectedException.none(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FirstTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FirstTest.java index bcd14d0c6e894..1a7301379cb03 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FirstTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FirstTest.java @@ -42,6 +42,7 @@ * Tests for First. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class FirstTest implements Serializable /* to allow anon inner classes */ { // PRE: lines contains no duplicates. diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java index ebb141f38b7b6..f20e740c84c3f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java @@ -53,6 +53,7 @@ * Tests for GroupByKey. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class GroupByKeyTest { @Rule diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java index 7e46bb31a7858..fdd557d08a7d4 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java @@ -69,6 +69,7 @@ * Tests for ParDo. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class ParDoTest implements Serializable { // This test is Serializable, just so that it's easy to have // anonymous inner classes inside the non-static test methods. diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java index 0d19f082ee07e..c129b8e6b912e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java @@ -42,6 +42,7 @@ * Tests for Partition */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class PartitionTest implements Serializable { static class ModFn implements PartitionFn { public int partitionFor(Integer elem, int numPartitions) { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java index d6de05af6d3f4..5a1c2a8f1340c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java @@ -40,6 +40,7 @@ * Tests for RateLimiter. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class RateLimitingTest { /** diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java index 7c51d096fe4e1..95f88489dbd54 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java @@ -39,6 +39,7 @@ * Tests for Sample transform. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class SampleTest { static final Integer[] EMPTY = new Integer[] { }; static final Integer[] DATA = new Integer[] {1, 2, 3, 4, 5}; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java index 63625a7f5f2b1..561e17b871e7e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java @@ -44,6 +44,7 @@ /** Tests for Top */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class TopTest { @Rule diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ViewTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ViewTest.java index 3a7c8187d9237..7b0313962fede 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ViewTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ViewTest.java @@ -42,6 +42,7 @@ * observed via {@link ParDo}. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class ViewTest implements Serializable { // This test is Serializable, just so that it's easy to have // anonymous inner classes inside the non-static test methods. diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/WithKeysTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/WithKeysTest.java index 3e4e359022c96..f0e608b2257d7 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/WithKeysTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/WithKeysTest.java @@ -34,6 +34,7 @@ * Tests for ExtractKeys transform. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class WithKeysTest { static final String[] COLLECTION = new String[] { "a", diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java index 016ba15d5ae44..6369d06aee522 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java @@ -51,6 +51,7 @@ * Tests for CoGroupByKeyTest. Implements Serializable for anonymous DoFns. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class CoGroupByKeyTest implements Serializable { /** diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java index 85c0bf6b8b6d2..e9041c860cee7 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java @@ -50,6 +50,7 @@ /** Unit tests for bucketing. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class WindowingTest implements Serializable { @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java index 45cc267d2d740..8f8dfbb9bafca 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java @@ -46,6 +46,7 @@ * Unit tests for the {@link Aggregator} API. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class AggregatorImplTest { @Rule public final ExpectedException expectedEx = ExpectedException.none(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CoderUtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CoderUtilsTest.java index 92f9e7481558f..1aec837d82ea4 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CoderUtilsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CoderUtilsTest.java @@ -40,6 +40,7 @@ * Tests for CoderUtils. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class CoderUtilsTest { static class TestCoder extends AtomicCoder { public static TestCoder of() { return new TestCoder(); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PTupleTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PTupleTest.java index 3692411a4a751..f09d59ce671b0 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PTupleTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PTupleTest.java @@ -32,6 +32,7 @@ public final class PTupleTest { @Test public void accessingNullVoidValuesShouldNotCauseExceptions() { + @SuppressWarnings("serial") TupleTag tag = new TupleTag() {}; PTuple tuple = PTuple.of(tag, null); assertTrue(tuple.has(tag)); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializableUtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializableUtilsTest.java index 90f10cdc97134..daddde564aa09 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializableUtilsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializableUtilsTest.java @@ -29,6 +29,7 @@ * Tests for SerializableUtils. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class SerializableUtilsTest { static class TestClass implements Serializable { final String stringValue; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PDoneTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PDoneTest.java index e886f350c12d7..f41fef6d840bf 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PDoneTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PDoneTest.java @@ -36,6 +36,7 @@ * Tests for PDone. */ @RunWith(JUnit4.class) +@SuppressWarnings("serial") public class PDoneTest { @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); From a77a8ab51a4738fdf7a8345dc526a72ee2fdc13e Mon Sep 17 00:00:00 2001 From: millsd Date: Mon, 15 Dec 2014 09:12:01 -0800 Subject: [PATCH 0012/1541] Remove type parameter from CalendarWindows [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82148657 --- .../transforms/windowing/CalendarWindows.java | 46 +++++++++---------- 1 file changed, 23 insertions(+), 23 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java index bb0de796f86ab..70590665be5ac 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java @@ -40,7 +40,7 @@ public class CalendarWindows { *

For example, {@code CalendarWindows.days(1)} will window elements into * separate windows for each day. */ - public static DaysWindows days(int number) { + public static DaysWindows days(int number) { return new DaysWindows(number, new DateTime(0, DateTimeZone.UTC), DateTimeZone.UTC); } @@ -50,7 +50,7 @@ public static DaysWindows days(int number) { *

For example, {@code CalendarWindows.weeks(1, DateTimeConstants.TUESDAY)} will * window elements into week-long windows starting on Tuesdays. */ - public static DaysWindows weeks(int number, int startDayOfWeek) { + public static DaysWindows weeks(int number, int startDayOfWeek) { return new DaysWindows( 7 * number, new DateTime(0, DateTimeZone.UTC).withDayOfWeek(startDayOfWeek), @@ -65,7 +65,7 @@ public static DaysWindows weeks(int number, int startDayOfWeek) { * will window elements into 8 month windows where that start on the 10th day of month, * and the first window begins in January 2014. */ - public static MonthsWindows months(int number) { + public static MonthsWindows months(int number) { return new MonthsWindows(number, 1, new DateTime(0, DateTimeZone.UTC), DateTimeZone.UTC); } @@ -77,7 +77,7 @@ public static MonthsWindows months(int number) { * will window elements into year-long windows that start at midnight on Jan 1, in the * America/Los_Angeles time zone. */ - public static YearsWindows years(int number) { + public static YearsWindows years(int number) { return new YearsWindows(number, 1, 1, new DateTime(0, DateTimeZone.UTC), DateTimeZone.UTC); } @@ -90,15 +90,15 @@ public static YearsWindows years(int number) { *

The time zone used to determine calendar boundaries is UTC, unless this * is overridden with the {@link #withTimeZone} method. */ - public static class DaysWindows extends PartitioningWindowingFn { + public static class DaysWindows extends PartitioningWindowingFn { - public DaysWindows withStartingDay(int year, int month, int day) { - return new DaysWindows( + public DaysWindows withStartingDay(int year, int month, int day) { + return new DaysWindows( number, new DateTime(year, month, day, 0, 0, timeZone), timeZone); } - public DaysWindows withTimeZone(DateTimeZone timeZone) { - return new DaysWindows( + public DaysWindows withTimeZone(DateTimeZone timeZone) { + return new DaysWindows( number, startDate.withZoneRetainFields(timeZone), timeZone); } @@ -155,20 +155,20 @@ public boolean isCompatible(WindowingFn other) { *

The time zone used to determine calendar boundaries is UTC, unless this * is overridden with the {@link #withTimeZone} method. */ - public static class MonthsWindows extends PartitioningWindowingFn { + public static class MonthsWindows extends PartitioningWindowingFn { - public MonthsWindows beginningOnDay(int dayOfMonth) { - return new MonthsWindows( + public MonthsWindows beginningOnDay(int dayOfMonth) { + return new MonthsWindows( number, dayOfMonth, startDate, timeZone); } - public MonthsWindows withStartingMonth(int year, int month) { - return new MonthsWindows( + public MonthsWindows withStartingMonth(int year, int month) { + return new MonthsWindows( number, dayOfMonth, new DateTime(year, month, 1, 0, 0, timeZone), timeZone); } - public MonthsWindows withTimeZone(DateTimeZone timeZone) { - return new MonthsWindows( + public MonthsWindows withTimeZone(DateTimeZone timeZone) { + return new MonthsWindows( number, dayOfMonth, startDate.withZoneRetainFields(timeZone), timeZone); } @@ -230,20 +230,20 @@ public boolean isCompatible(WindowingFn other) { *

The time zone used to determine calendar boundaries is UTC, unless this * is overridden with the {@link #withTimeZone} method. */ - public static class YearsWindows extends PartitioningWindowingFn { + public static class YearsWindows extends PartitioningWindowingFn { - public YearsWindows beginningOnDay(int monthOfYear, int dayOfMonth) { - return new YearsWindows( + public YearsWindows beginningOnDay(int monthOfYear, int dayOfMonth) { + return new YearsWindows( number, monthOfYear, dayOfMonth, startDate, timeZone); } - public YearsWindows withStartingYear(int year) { - return new YearsWindows( + public YearsWindows withStartingYear(int year) { + return new YearsWindows( number, monthOfYear, dayOfMonth, new DateTime(year, 1, 1, 0, 0, timeZone), timeZone); } - public YearsWindows withTimeZone(DateTimeZone timeZone) { - return new YearsWindows( + public YearsWindows withTimeZone(DateTimeZone timeZone) { + return new YearsWindows( number, monthOfYear, dayOfMonth, startDate.withZoneRetainFields(timeZone), timeZone); } From 1aca9d28f08fa4a65709a22048361df68133d446 Mon Sep 17 00:00:00 2001 From: davor Date: Mon, 15 Dec 2014 11:14:56 -0800 Subject: [PATCH 0013/1541] Dataflow launch: cleanup a subset of checkstyle warnings in the SDK. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82158870 --- checkstyle.xml | 5 ++-- .../dataflow/examples/BigQueryTornadoes.java | 10 +++---- .../google/cloud/dataflow/examples/TfIdf.java | 4 +-- .../examples/TopWikipediaSessions.java | 4 +-- .../cloud/dataflow/examples/WordCount.java | 6 ++--- .../dataflow/sdk/coders/package-info.java | 26 +++++++++---------- .../cloud/dataflow/sdk/io/PubsubIO.java | 4 +-- .../cloud/dataflow/sdk/io/package-info.java | 2 +- .../DataflowPipelineWorkerPoolOptions.java | 8 +++--- .../dataflow/sdk/options/GcpOptions.java | 2 +- .../dataflow/sdk/options/GcsOptions.java | 2 +- .../sdk/options/PipelineOptionsFactory.java | 2 +- .../sdk/options/PipelineOptionsValidator.java | 2 +- .../sdk/options/ProxyInvocationHandler.java | 2 +- .../sdk/options/StreamingOptions.java | 6 ++--- .../dataflow/sdk/options/package-info.java | 6 ++--- .../cloud/dataflow/sdk/package-info.java | 18 ++++++------- .../sdk/runners/DataflowPipelineRunner.java | 4 +-- .../sdk/runners/dataflow/package-info.java | 2 +- .../dataflow/sdk/runners/package-info.java | 10 +++---- .../runners/worker/DataflowWorkerHarness.java | 2 +- .../sdk/runners/worker/FileBasedSource.java | 2 +- .../DataflowWorkerLoggingFormatter.java | 2 +- .../DataflowWorkerLoggingInitializer.java | 4 +-- .../sdk/runners/worker/package-info.java | 2 +- .../dataflow/sdk/testing/TestPipeline.java | 2 +- .../dataflow/sdk/testing/package-info.java | 4 +-- .../cloud/dataflow/sdk/transforms/Count.java | 2 +- .../sdk/transforms/join/package-info.java | 2 +- .../dataflow/sdk/transforms/package-info.java | 20 +++++++------- .../sdk/util/AppEngineEnvironment.java | 2 +- .../cloud/dataflow/sdk/util/Credentials.java | 2 +- .../cloud/dataflow/sdk/util/GcsUtil.java | 2 +- .../cloud/dataflow/sdk/util/PackageUtil.java | 2 +- .../dataflow/sdk/values/package-info.java | 20 +++++++------- .../options/DataflowPipelineOptionsTest.java | 2 +- .../options/PipelineOptionsValidatorTest.java | 2 +- .../options/ProxyInvocationHandlerTest.java | 20 +++++++------- .../DataflowWorkerLoggingInitializerTest.java | 2 +- .../AttemptBoundedExponentialBackOffTest.java | 2 +- .../cloud/dataflow/sdk/util/GcsUtilTest.java | 2 +- .../util/common/worker/ReadOperationTest.java | 2 +- 42 files changed, 113 insertions(+), 114 deletions(-) diff --git a/checkstyle.xml b/checkstyle.xml index 08df965ae6bb1..702558ac34acb 100644 --- a/checkstyle.xml +++ b/checkstyle.xml @@ -44,8 +44,7 @@ page at http://checkstyle.sourceforge.net/config.html --> - + @@ -60,7 +59,7 @@ page at http://checkstyle.sourceforge.net/config.html --> - + diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/BigQueryTornadoes.java b/examples/src/main/java/com/google/cloud/dataflow/examples/BigQueryTornadoes.java index a5f9a19705218..b6974e74eb829 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/BigQueryTornadoes.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/BigQueryTornadoes.java @@ -39,19 +39,19 @@ /** * An example that reads the public samples of weather data from BigQuery, counts the number of * tornadoes that occur in each month, and writes the results to BigQuery. - * + * *

Concepts: Reading/writing BigQuery; counting a PCollection; user-defined PTransforms - * - *

Note: Before running this example, you must create a BigQuery dataset to contain your output + * + *

Note: Before running this example, you must create a BigQuery dataset to contain your output * table. - * + * *

To execute this pipeline locally, specify general pipeline configuration: * --project= * and the BigQuery table for the output: * --output=:. * *

To execute this pipeline using the Dataflow service, specify pipeline configuration: - * --project= + * --project= * --stagingLocation=gs:// * --runner=BlockingDataflowPipelineRunner * and the BigQuery table for the output: diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java b/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java index 15948782e2b60..3f22e27eab264 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java @@ -60,7 +60,7 @@ /** * An example that computes a basic TF-IDF search table for a directory or GCS prefix. - * + * *

Concepts: joining data; side inputs * *

To execute this pipeline locally, specify general pipeline configuration: @@ -69,7 +69,7 @@ * --output=[ | gs://] * *

To execute this pipeline using the Dataflow service, specify pipeline configuration: - * --project= + * --project= * --stagingLocation=gs:// * --runner=BlockingDataflowPipelineRunner * and an output prefix on GCS: diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java b/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java index 3d945b6394acb..a64e2fc98f49a 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java @@ -49,7 +49,7 @@ * the longest string of edits separated by no more than an hour within each month. * *

Concepts: Using Windowing to perform time-based aggregations of data. - * + * *

It is not recommended to execute this pipeline locally, given the size of the default input * data. * @@ -62,7 +62,7 @@ * *

The default input is gs://dataflow-samples/wikipedia_edits/*.json and can be overridden with * --input. - * + * *

The input for this example is large enough that it's a good place to enable (experimental) * autoscaling: * --autoscalingAlgorithm=BASIC diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java b/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java index 4fe304925c0b6..948b83e032fab 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java @@ -38,7 +38,7 @@ * An example that counts words in Shakespeare. For a detailed walkthrough of this * example see: * https://developers.google.com/cloud-dataflow/java-sdk/wordcount-example - * + * *

Concepts: Reading/writing text files; counting a PCollection; user-defined PTransforms * *

To execute this pipeline locally, specify general pipeline configuration: @@ -69,12 +69,12 @@ public void startBundle(Context c) { @Override public void processElement(ProcessContext c) { - // Keep track of the number of empty lines. (When using the [Blocking]DataflowPipelineRunner, + // Keep track of the number of empty lines. (When using the [Blocking]DataflowPipelineRunner, // Aggregators are shown in the monitoring UI.) if (c.element().trim().isEmpty()) { emptyLines.addValue(1L); } - + // Split the line into words. String[] words = c.element().split("[^a-zA-Z']+"); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/package-info.java index ea305e776bc9a..3366ff2267010 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/package-info.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/package-info.java @@ -14,31 +14,31 @@ * the License. */ -/** - * Defines {@link com.google.cloud.dataflow.sdk.coders.Coder}s +/** + * Defines {@link com.google.cloud.dataflow.sdk.coders.Coder}s * to specify how data is encoded to and decoded from byte strings. - * + * *

During execution of a Pipeline, elements in a * {@link com.google.cloud.dataflow.sdk.values.PCollection} * may need to be encoded into byte strings. * This happens both at the beginning and end of a pipeline when data is read from and written to * persistent storage and also during execution of a pipeline when elements are communicated between * machines. - * - *

Exactly when PCollection elements are encoded during execution depends on which + * + *

Exactly when PCollection elements are encoded during execution depends on which * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} is being used and how that runner * chooses to execute the pipeline. As such, Dataflow requires that all PCollections have an - * appropriate Coder in case it becomes necessary. In many cases, the Coder can be inferred from + * appropriate Coder in case it becomes necessary. In many cases, the Coder can be inferred from * the available Java type * information and the Pipeline's {@link com.google.cloud.dataflow.sdk.coders.CoderRegistry}. It - * can be specified per PCollection via + * can be specified per PCollection via * {@link com.google.cloud.dataflow.sdk.values.PCollection#setCoder(Coder)} or per type using the * {@link com.google.cloud.dataflow.sdk.coders.DefaultCoder} annotation. - * - *

This package provides a number of coders for common types like {@code Integer}, - * {@code String}, and {@code List}, as well as coders like - * {@link com.google.cloud.dataflow.sdk.coders.AvroCoder} that can be used to encode many custom - * types. - * + * + *

This package provides a number of coders for common types like {@code Integer}, + * {@code String}, and {@code List}, as well as coders like + * {@link com.google.cloud.dataflow.sdk.coders.AvroCoder} that can be used to encode many custom + * types. + * */ package com.google.cloud.dataflow.sdk.coders; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index 75e17bed55264..5d87f2563fa52 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -35,9 +35,9 @@ *

Important: PubsubIO is experimental. It is not supported by the * {@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner} and is only supported in the * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner} for users whitelisted in a - * streaming early access program and who enable + * streaming early access program and who enable * {@link com.google.cloud.dataflow.sdk.options.StreamingOptions#setStreaming(boolean)}. - * + * *

You should expect this class to change significantly in future versions of the SDK * or be removed entirely. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/package-info.java index 886255e271d23..b472b3f7c6200 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/package-info.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/package-info.java @@ -19,7 +19,7 @@ * {@link com.google.cloud.dataflow.sdk.io.AvroIO}, * {@link com.google.cloud.dataflow.sdk.io.BigQueryIO}, and * {@link com.google.cloud.dataflow.sdk.io.TextIO}. - * + * *

The classes in this package provide {@code Read} transforms which create PCollections * from existing storage: *

{@code
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java
index b388da4313ee7..0193ddaac1077 100644
--- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java
@@ -37,14 +37,14 @@ public interface DataflowPipelineWorkerPoolOptions {
   @Default.Integer(3)
   int getNumWorkers();
   void setNumWorkers(int value);
-  
+
   /**
    * Type of autoscaling algorithm to use. These types are experimental and subject to change.
    */
   public enum AutoscalingAlgorithmType {
     /** Use numWorkers machines. Do not autoscale the worker pool. */
     NONE("AUTOSCALING_ALGORITHM_NONE"),
-    
+
     /** Autoscale the workerpool size up to maxNumWorkers until the job completes. */
     BASIC("AUTOSCALING_ALGORITHM_BASIC");
 
@@ -73,9 +73,9 @@ public AutoscalingAlgorithmType create(PipelineOptions options) {
       return AutoscalingAlgorithmType.NONE;
     }
   }
-  
+
   /**
-   * Max number of workers to use when using workerpool autoscaling. 
+   * Max number of workers to use when using workerpool autoscaling.
    * This option is experimental and subject to change.
    */
   @Description("Max number of workers to use, when using autoscaling")
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java
index 7dbaa5fb32d9f..34faf03acfb66 100644
--- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java
@@ -51,7 +51,7 @@
  *     with the service account name.
  *   
  * 
- * The default mechanism is to use the 
+ * The default mechanism is to use the
  * 
  * application default credentials falling back to gcloud. The other options can be
  * used by setting the corresponding properties.
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java
index 543c9cac6c406..20685b7cf383f 100644
--- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java
@@ -62,7 +62,7 @@ public ExecutorService create(PipelineOptions options) {
       }
       /* The SDK requires an unbounded thread pool because a step may create X writers
        * each requiring their own thread to perform the writes otherwise a writer may
-       * block causing deadlock for the step because the writers buffer is full. 
+       * block causing deadlock for the step because the writers buffer is full.
        * Also, the MapTaskExecutor launches the steps in reverse order and completes
        * them in forward order thus requiring enough threads so that each step's writers
        * can be active.
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java
index 9ff9280803fa6..a46ce313e3383 100644
--- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java
@@ -385,7 +385,7 @@ static synchronized  Registration validateWellForm
       Class iface, Set> validatedPipelineOptionsInterfaces) {
     Preconditions.checkArgument(iface.isInterface(), "Only interface types are supported.");
 
-    Set> combinedPipelineOptionsInterfaces = 
+    Set> combinedPipelineOptionsInterfaces =
         FluentIterable.from(validatedPipelineOptionsInterfaces).append(iface).toSet();
     // Validate that the view of all currently passed in options classes is well formed.
     if (!COMBINED_CACHE.containsKey(combinedPipelineOptionsInterfaces)) {
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java
index 046cfd7fda05e..9878056af64db 100644
--- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidator.java
@@ -53,7 +53,7 @@ public static  T validate(Class klass, PipelineOpt
     }
     return options.as(klass);
   }
-  
+
   private static String getDescription(Method method) {
     Description description = method.getAnnotation(Description.class);
     return description == null ? "" : description.value();
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java
index aefbe1dec294e..88e84f6142313 100644
--- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java
@@ -138,7 +138,7 @@ synchronized  T as(Class iface) {
     Preconditions.checkNotNull(iface);
     Preconditions.checkArgument(iface.isInterface());
     if (!interfaceToProxyCache.containsKey(iface)) {
-      Registration registration = 
+      Registration registration =
           PipelineOptionsFactory.validateWellFormed(iface, knownInterfaces);
       List propertyDescriptors = registration.getPropertyDescriptors();
       Class proxyClass = registration.getProxyClass();
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java
index 725d845d5b9f2..6deca04b4888f 100644
--- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java
@@ -18,11 +18,11 @@
 
 /**
  * [Whitelisting Required] Options used to configure the streaming backend.
- * 
+ *
  * 

Important: Streaming support is experimental. It is only supported in the * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner} for users whitelisted in a - * streaming early access program. - * + * streaming early access program. + * *

You should expect this class to change significantly in future * versions of the SDK or be removed entirely. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/package-info.java index 557e377676b71..0541b942a0c21 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/package-info.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/package-info.java @@ -14,10 +14,10 @@ * the License. */ -/** - * Defines {@link com.google.cloud.dataflow.sdk.options.PipelineOptions} for +/** + * Defines {@link com.google.cloud.dataflow.sdk.options.PipelineOptions} for * configuring pipeline execution. - * + * *

{@link com.google.cloud.dataflow.sdk.options.PipelineOptions} encapsulates the various * parameters that describe how a pipeline should be run. {@code PipelineOptions} are created * using a {@link com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory}. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/package-info.java index e27ac01476606..ad5f04ca68ca9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/package-info.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/package-info.java @@ -14,19 +14,19 @@ * the License. */ -/** - * Provides a simple, powerful model for building both batch and +/** + * Provides a simple, powerful model for building both batch and * streaming parallel data processing - * {@link com.google.cloud.dataflow.sdk.Pipeline}s. - * - *

To use the Google Cloud Dataflow SDK, you build a - * {@link com.google.cloud.dataflow.sdk.Pipeline} which manages a graph of - * {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s - * and the {@link com.google.cloud.dataflow.sdk.values.PCollection}s that + * {@link com.google.cloud.dataflow.sdk.Pipeline}s. + * + *

To use the Google Cloud Dataflow SDK, you build a + * {@link com.google.cloud.dataflow.sdk.Pipeline} which manages a graph of + * {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s + * and the {@link com.google.cloud.dataflow.sdk.values.PCollection}s that * the PTransforms consume and produce. * *

Each Pipeline has a - * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} to specify + * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} to specify * where and how it should run after pipeline construction is complete. * */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index c13ef51c7c18a..1a0d873307375 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -96,7 +96,7 @@ public class DataflowPipelineRunner extends PipelineRunner * @return The newly created runner. */ public static DataflowPipelineRunner fromOptions(PipelineOptions options) { - DataflowPipelineOptions dataflowOptions = + DataflowPipelineOptions dataflowOptions = PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options); ArrayList missing = new ArrayList<>(); @@ -230,7 +230,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { .execute(); } catch (GoogleJsonResponseException e) { throw new RuntimeException( - "Failed to create a workflow job: " + "Failed to create a workflow job: " + (e.getDetails() != null ? e.getDetails().getMessage() : e), e); } catch (IOException e) { throw new RuntimeException("Failed to create a workflow job", e); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/package-info.java index c2fcc288cf3c4..751ca69e5572c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/package-info.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/package-info.java @@ -14,7 +14,7 @@ * the License. */ -/** +/** * Implementation of the {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}. */ package com.google.cloud.dataflow.sdk.runners.dataflow; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/package-info.java index c75fe2f8348e0..8d543a4041b87 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/package-info.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/package-info.java @@ -15,19 +15,19 @@ */ /** - * Defines runners for executing Pipelines in different modes, including + * Defines runners for executing Pipelines in different modes, including * {@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner} and * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}. - * + * *

{@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner} executes a {@code Pipeline} - * locally, without contacting the Dataflow service. - * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner} submits a + * locally, without contacting the Dataflow service. + * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner} submits a * {@code Pipeline} to the Dataflow service, which executes it on Dataflow-managed Compute Engine * instances. {@code DataflowPipelineRunner} returns * as soon as the {@code Pipeline} has been submitted. Use * {@link com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner} to have execution * updates printed to the console. - * + * *

The runner is specified as part {@link com.google.cloud.dataflow.sdk.options.PipelineOptions}. */ package com.google.cloud.dataflow.sdk.runners; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java index fa17cf67390d0..f46a8d4a00039 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java @@ -104,7 +104,7 @@ static void processWork(DataflowWorker worker) throws IOException { static DataflowWorker createFromSystemProperties() { return create(PipelineOptionsFactory.createFromSystemProperties()); } - + static DataflowWorker create(DataflowWorkerHarnessOptions options) { MDC.put(DataflowWorkerLoggingFormatter.MDC_DATAFLOW_JOB_ID, options.getJobId()); MDC.put(DataflowWorkerLoggingFormatter.MDC_DATAFLOW_WORKER_ID, options.getWorkerId()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedSource.java index beea88747c1cf..2e152e794f80c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedSource.java @@ -199,7 +199,7 @@ public Position updateStopPosition(Progress proposedStopPosition) { // CloudPosition in a file-based Source. If stop position in // other types is proposed, the end position in iterator will // not be updated, and return null. - com.google.api.services.dataflow.model.ApproximateProgress stopPosition = + com.google.api.services.dataflow.model.ApproximateProgress stopPosition = sourceProgressToCloudProgress(proposedStopPosition); if (stopPosition == null) { LOG.warn( diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java index 85805773c7060..a8f54b48668fd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java @@ -47,7 +47,7 @@ public class DataflowWorkerLoggingFormatter extends Formatter { public String format(LogRecord record) { String exception = formatException(record.getThrown()); return DATE_FORMATTER.print(record.getMillis()) - + " " + MoreObjects.firstNonNull(LEVELS.get(record.getLevel()), + + " " + MoreObjects.firstNonNull(LEVELS.get(record.getLevel()), record.getLevel().getName()) + " " + MoreObjects.firstNonNull(MDC.get(MDC_DATAFLOW_JOB_ID), "unknown") + " " + MoreObjects.firstNonNull(MDC.get(MDC_DATAFLOW_WORKER_ID), "unknown") diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java index 80ccf7084bcbf..6be47019e3c86 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java @@ -34,14 +34,14 @@ * the logging level and location by specifying the Java system properties * "dataflow.worker.logging.level" and "dataflow.worker.logging.location" respectively. * The default log level is INFO and the default location is a file named dataflow-worker.log - * within the systems temporary directory. + * within the systems temporary directory. */ public class DataflowWorkerLoggingInitializer { private static final String DEFAULT_LOGGING_LOCATION = "/tmp/dataflow-worker.log"; private static final String ROOT_LOGGER_NAME = ""; public static final String DATAFLOW_WORKER_LOGGING_LEVEL = "dataflow.worker.logging.level"; public static final String DATAFLOW_WORKER_LOGGING_LOCATION = "dataflow.worker.logging.location"; - public static final ImmutableBiMap LEVELS = + public static final ImmutableBiMap LEVELS = ImmutableBiMap.builder() .put(Level.SEVERE, "ERROR") .put(Level.WARNING, "WARNING") diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/package-info.java index 615ed64743922..ffc9df2e43f26 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/package-info.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/package-info.java @@ -14,7 +14,7 @@ * the License. */ -/** +/** * Implementation of the harness that runs on each Google Compute Engine instance to coordinate * execution of Pipeline code. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java index 6044365a664d9..ff003d5dbf9ef 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java @@ -125,7 +125,7 @@ private static String getAppName() { Optional stackTraceElement = findCallersStackTrace(); if (stackTraceElement.isPresent()) { String className = stackTraceElement.get().getClassName(); - return className.contains(".") + return className.contains(".") ? className.substring(className.lastIndexOf(".") + 1) : className; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/package-info.java index 799c1ac98bc8c..9a410fb229f1f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/package-info.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/package-info.java @@ -14,8 +14,8 @@ * the License. */ -/** +/** * Defines utilities for unit testing Dataflow pipelines. The tests for the {@code PTransform}s and - * examples included the Dataflow SDK provide examples of using these utilities. + * examples included the Dataflow SDK provide examples of using these utilities. */ package com.google.cloud.dataflow.sdk.testing; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java index 29336d859ced8..49251dfeca210 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java @@ -120,7 +120,7 @@ public void processElement(ProcessContext c) { * encoding each element using the input {@code PCollection}'s * {@code Coder}, then comparing the encoded bytes. Because of this, * the input coder must be deterministic. (See - * {@link com.google.cloud.dataflow.sdk.coders.Coder#isDeterministic()} for more detail). + * {@link com.google.cloud.dataflow.sdk.coders.Coder#isDeterministic()} for more detail). * Performing the comparison in this manner admits efficient parallel evaluation. * *

By default, the {@code Coder} of the keys of the output diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/package-info.java index ba907ac2cd734..25eafd34aac18 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/package-info.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/package-info.java @@ -14,7 +14,7 @@ * the License. */ -/** +/** * Defines the {@link com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey} transform * for joining multiple PCollections. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/package-info.java index b72e90e780ac0..5b50fd37634b0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/package-info.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/package-info.java @@ -14,12 +14,12 @@ * the License. */ -/** - * Defines {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s for transforming +/** + * Defines {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s for transforming * data in a pipeline. - * + * *

A {@link com.google.cloud.dataflow.sdk.transforms.PTransform} is an operation that takes an - * {@code Input} (some subtype of {@link com.google.cloud.dataflow.sdk.values.PInput}) + * {@code Input} (some subtype of {@link com.google.cloud.dataflow.sdk.values.PInput}) * and produces an * {@code Output} (some subtype of {@link com.google.cloud.dataflow.sdk.values.POutput}). * @@ -29,15 +29,15 @@ * conversion operations like {@link com.google.cloud.dataflow.sdk.transforms.ParDo}, * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}, * {@link com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey}, - * {@link com.google.cloud.dataflow.sdk.transforms.Combine}, and + * {@link com.google.cloud.dataflow.sdk.transforms.Combine}, and * {@link com.google.cloud.dataflow.sdk.transforms.Count}, and outputting * PTransforms like - * {@link com.google.cloud.dataflow.sdk.io.TextIO.Write}. - * - *

New PTransforms can be created by composing existing PTransforms. + * {@link com.google.cloud.dataflow.sdk.io.TextIO.Write}. + * + *

New PTransforms can be created by composing existing PTransforms. * Most PTransforms in this package are composites, and users can also create composite PTransforms - * for their own application-specific logic. - * + * for their own application-specific logic. + * */ package com.google.cloud.dataflow.sdk.transforms; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AppEngineEnvironment.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AppEngineEnvironment.java index f3b57a4508b0e..c83fd1da79d89 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AppEngineEnvironment.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AppEngineEnvironment.java @@ -32,7 +32,7 @@ public class AppEngineEnvironment { * Purposely copied and left private from private code.google.common.util.concurrent.MoreExecutors#isAppEngine. - * + * * @return true if we are inside of AppEngine, false otherwise. */ static boolean isAppEngine() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java index 419339a83415e..6d82eed7a7236 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java @@ -127,7 +127,7 @@ public static Credential getWorkerCredential(GcpOptions options) * "service_account_name" to use this mechanism. * * - * The default mechanism is to use the + * The default mechanism is to use the * * application default credentials falling back to gcloud. The other options can be * used by providing the corresponding properties. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java index c3edd2ac2c338..cc429955f5662 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java @@ -47,7 +47,7 @@ */ public class GcsUtil { /** - * This is a {@link DefaultValueFactory} able to create a {@link GcsUtil} using + * This is a {@link DefaultValueFactory} able to create a {@link GcsUtil} using * any transport flags specified on the {@link PipelineOptions}. */ public static class GcsUtilFactory implements DefaultValueFactory { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java index c9e3185edd8c3..46f610836f57d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java @@ -172,7 +172,7 @@ static List stageClasspathElementsToGcs( throw new RuntimeException("Could not stage classpath element: " + classpathElement, e); } } - + LOG.info("Uploading PipelineOptions.filesToStage complete: {} files newly uploaded, " + "{} files cached", numUploaded, numCached); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/package-info.java index ba6e927e0996b..232e0b4942a40 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/package-info.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/package-info.java @@ -14,18 +14,18 @@ * the License. */ -/** - * Defines {@link com.google.cloud.dataflow.sdk.values.PCollection} and other classes for +/** + * Defines {@link com.google.cloud.dataflow.sdk.values.PCollection} and other classes for * representing data in a {@link com.google.cloud.dataflow.sdk.Pipeline}. - * - *

A {@link com.google.cloud.dataflow.sdk.values.PCollection} is an immutable collection of - * values of type {@code T} and is the main representation for data. + * + *

A {@link com.google.cloud.dataflow.sdk.values.PCollection} is an immutable collection of + * values of type {@code T} and is the main representation for data. * A {@link com.google.cloud.dataflow.sdk.values.PCollectionTuple} is a tuple of PCollections - * used in cases where PTransforms take or return multiple PCollections. - * + * used in cases where PTransforms take or return multiple PCollections. + * *

A {@link com.google.cloud.dataflow.sdk.values.PCollectionTuple} is an immutable tuple of * heterogeneously-typed {@link com.google.cloud.dataflow.sdk.values.PCollection}s, "keyed" by - * {@link com.google.cloud.dataflow.sdk.values.TupleTag}s. + * {@link com.google.cloud.dataflow.sdk.values.TupleTag}s. * A PCollectionTuple can be used as the input or * output of a * {@link com.google.cloud.dataflow.sdk.transforms.PTransform} taking @@ -33,10 +33,10 @@ * different types, for instance a * {@link com.google.cloud.dataflow.sdk.transforms.ParDo} with side * outputs. - * + * *

A {@link com.google.cloud.dataflow.sdk.values.PCollectionView} is an immutable view of a * PCollection that can be accessed from a DoFn and other user Fns * as a side input. - * + * */ package com.google.cloud.dataflow.sdk.values; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java index f4d6f0499d44f..50ea20bf539c6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java @@ -81,7 +81,7 @@ public void testUserNameIsTooLong() { assertTrue(options.getJobName().length() <= 40); } - + @Test public void testUtf8UserNameAndApplicationNameIsNormalized() { resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z"); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidatorTest.java index ad584ea32aeb4..714647e35de88 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidatorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsValidatorTest.java @@ -34,7 +34,7 @@ public static interface Required extends PipelineOptions { public String getObject(); public void setObject(String value); } - + @Test public void testWhenRequiredOptionIsSet() { Required required = PipelineOptionsFactory.as(Required.class); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java index b9b07e8626b1d..d33d42e4bf165 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java @@ -111,7 +111,7 @@ public void testGettingJLSDefaults() throws Exception { public static class TestOptionFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { - return "testOptionFactory"; + return "testOptionFactory"; } } @@ -224,7 +224,7 @@ public void testInvokeWithUnknownMethod() throws Exception { expectedException.expectMessage("Unknown method [public abstract void com.google.cloud." + "dataflow.sdk.options.ProxyInvocationHandlerTest$UnknownMethod.unknownMethod()] " + "invoked with args [null]."); - + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); handler.invoke(handler, UnknownMethod.class.getMethod("unknownMethod"), null); } @@ -292,7 +292,7 @@ public static interface MethodConflict extends Simple { public void testMethodConflictProvidesSameValue() throws Exception { ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); MethodConflict methodConflict = handler.as(MethodConflict.class); - + methodConflict.setString("conflictValue"); assertEquals("conflictValue", methodConflict.getString()); assertEquals("conflictValue", methodConflict.as(Simple.class).getString()); @@ -364,7 +364,7 @@ public static interface PartialMethodConflict extends Simple { @Override void setPrimitive(int value); } - + @Test public void testPartialMethodConflictProvidesSameValue() throws Exception { ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); @@ -404,13 +404,13 @@ public void testJsonConversionForSimpleTypes() throws Exception { assertEquals(5, options2.getInteger()); assertEquals("TestValue", options2.getString()); } - + @Test public void testJsonConversionOfAJsonConvertedType() throws Exception { SimpleTypes options = PipelineOptionsFactory.as(SimpleTypes.class); options.setString("TestValue"); options.setInteger(5); - SimpleTypes options2 = serializeDeserialize(SimpleTypes.class, + SimpleTypes options2 = serializeDeserialize(SimpleTypes.class, serializeDeserialize(SimpleTypes.class, options)); assertEquals(5, options2.getInteger()); assertEquals("TestValue", options2.getString()); @@ -478,7 +478,7 @@ static InnerType of(double value) { @Override public boolean equals(Object obj) { return obj != null - && getClass().equals(obj.getClass()) + && getClass().equals(obj.getClass()) && Objects.equals(doubleField, ((InnerType) obj).doubleField); } } @@ -493,7 +493,7 @@ private static class ComplexType { @Override public boolean equals(Object obj) { return obj != null - && getClass().equals(obj.getClass()) + && getClass().equals(obj.getClass()) && Objects.equals(stringField, ((ComplexType) obj).stringField) && Objects.equals(intField, ((ComplexType) obj).intField) && Objects.equals(genericType, ((ComplexType) obj).genericType) @@ -531,7 +531,7 @@ private static interface IgnoredProperty extends PipelineOptions { public void testJsonConversionOfIgnoredProperty() throws Exception { IgnoredProperty options = PipelineOptionsFactory.as(IgnoredProperty.class); options.setValue("TestValue"); - + IgnoredProperty options2 = serializeDeserialize(IgnoredProperty.class, options); assertNull(options2.getValue()); } @@ -577,7 +577,7 @@ public void testJsonConversionOfIgnoredNotSerializableProperty() throws Exceptio IgnoredNotSerializableProperty options = PipelineOptionsFactory.as(IgnoredNotSerializableProperty.class); options.setValue(new NotSerializable("TestString")); - + IgnoredNotSerializableProperty options2 = serializeDeserialize(IgnoredNotSerializableProperty.class, options); assertNull(options2.getValue()); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java index 71e51f430d88d..68a6b58a2d9bc 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java @@ -94,7 +94,7 @@ public void testWithOverrides() { assertTrue(isConsoleHandler(handlers.get(0), Level.WARNING)); assertTrue(isFileHandler(handlers.get(1), Level.WARNING)); } - + private boolean isConsoleHandler(Handler handler, Level level) { return handler instanceof ConsoleHandler && level.equals(handler.getLevel()) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOffTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOffTest.java index 0c262e2f1cb04..e4f332db12e88 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOffTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOffTest.java @@ -56,7 +56,7 @@ public void testThatFixedNumberOfAttemptsExits() throws Exception { assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); } - + @Test public void testThatResettingAllowsReuse() throws Exception { BackOff backOff = new AttemptBoundedExponentialBackOff(3, 500); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GcsUtilTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GcsUtilTest.java index cae705cea5a46..c7a7c55f5b585 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GcsUtilTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GcsUtilTest.java @@ -99,7 +99,7 @@ public void run() { // Release the last latch starting the chain reaction. countDownLatches[countDownLatches.length - 1].countDown(); executorService.shutdown(); - assertTrue("Expected tasks to complete", + assertTrue("Expected tasks to complete", executorService.awaitTermination(10, TimeUnit.SECONDS)); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java index b3e29f8e5cf55..d1a69e7a10cf0 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java @@ -295,7 +295,7 @@ private static Position makePosition(long offset) { private static ApproximateProgress makeApproximateProgress(long offset) { return makeApproximateProgress(makePosition(offset)); } - + private static ApproximateProgress makeApproximateProgress( com.google.api.services.dataflow.model.Position position) { return new ApproximateProgress().setPosition(position); From 7fb008fac423dfb570f697f52becfd7cce50b584 Mon Sep 17 00:00:00 2001 From: fjp Date: Mon, 15 Dec 2014 11:55:00 -0800 Subject: [PATCH 0014/1541] Markdown-ify README [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82162508 --- README | 19 ------------------- README.md | 42 ++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 42 insertions(+), 19 deletions(-) delete mode 100644 README create mode 100644 README.md diff --git a/README b/README deleted file mode 100644 index 2da8ea5ba93c4..0000000000000 --- a/README +++ /dev/null @@ -1,19 +0,0 @@ -Greetings! Welcome to the (Alpha) Google Cloud Dataflow Java SDK. - -Dataflow provides a simple, powerful model for building both batch and -streaming parallel data processing Pipelines. - -To use the Google Cloud Dataflow SDK, you build a Pipeline which manages a -graph of PTransforms and PCollections that the PTransforms consume and produce. - -You then use a PipelineRunner to specify where and how the pipeline should -execute. Currently there are two runners: - 1. The DirectPipelineRunner runs the pipeline on your local machine. - 2. The [Blocking]DataflowPipelineRunner runs the pipeline on the Dataflow - Service using the Google Cloud Platform. The Dataflow Service is - currently in the Alpha phase of development and access is limited to - whitelisted users. - -For more about both the Dataflow SDK and the Dataflow Service, visit: - http://cloud.google.com/dataflow - diff --git a/README.md b/README.md new file mode 100644 index 0000000000000..12eba01f5e831 --- /dev/null +++ b/README.md @@ -0,0 +1,42 @@ +# Cloud Dataflow Java SDK (Alpha) + +[Google Cloud Dataflow](http://cloud.google.com/dataflow) +provides a simple, powerful programming model for building both batch +and streaming parallel data processing pipelines. + +## Overview + +The key concepts in this programming model are: + +* [PCollection](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java): +represents a collection of data, which could be bounded or unbounded in size. +* [PTransform](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java): +represents a computation that transform input PCollections into output PCollections. +* [Pipeline](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java): +manages a directed acyclic graph of PTransforms and PCollections, which is ready for excution. +* [PipelineRunner](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java): +specifies where and how the pipeline should execute. + +Currently there are three runners: + + 1. The [DirectPipelineRunner](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java) +runs the pipeline on your local machine. + 2. The +[DataflowPipelineRunner](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java) +submits the pipeline to the Dataflow Service**\***, where it runs using managed +resources in the [Google Cloud Platform](http://cloud.google.com). + 3. The +[BlockingDataflowPipelineRunner](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunner.java) +submits the pipeline to the Dataflow Service**\*** via the DataflowPipelineRunner and then prints messages +about the job status until execution is complete. + +**\***_The Dataflow Service is currently in the Alpha phase of development and access +is limited to whitelisted users._ + +## Getting Started + +## More Information + +* [Google Cloud Dataflow](http://cloud.google.com/dataflow) +* [Dataflow Concepts and Programming Model](https://cloud.google.com/dataflow/java-sdk/building-a-pipeline) +* [Javadoc](https://cloud.google.com/dataflow/java-sdk/JavaDoc/index) From 5586b61fe7c4bd2de8549e7c6c3ac060c2ac1951 Mon Sep 17 00:00:00 2001 From: robertwb Date: Mon, 15 Dec 2014 12:18:39 -0800 Subject: [PATCH 0015/1541] Serializables is a very inefficient encoding for AccumulatingCombineFn.Accumulators, should be avoided. Also allow AccumulatingCombineFn.Accumulators to be a static class. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82164661 --- .../sdk/transforms/ApproximateQuantiles.java | 5 +- .../dataflow/sdk/transforms/Combine.java | 12 ++++- .../cloud/dataflow/sdk/transforms/Mean.java | 50 ++++++++++++++++--- .../cloud/dataflow/sdk/transforms/Top.java | 6 +-- .../runners/worker/CombineValuesFnTest.java | 4 +- .../dataflow/sdk/transforms/CombineTest.java | 13 +++-- 6 files changed, 67 insertions(+), 23 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java index b2c93c4d701cd..a2f0094cbb463 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java @@ -367,9 +367,8 @@ public Coder getAccumulatorCoder( * estimated. */ class QuantileState - extends AccumulatingCombineFn - .QuantileState, List> - .Accumulator { + implements AccumulatingCombineFn.Accumulator + .QuantileState, List> { private T min; private T max; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java index f455d5763ac1a..f27308b739589 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -355,6 +355,10 @@ public VO apply(Iterable inputs) { * for {@code VI} values and the enclosing {@code Pipeline}'s * {@code CoderRegistry} to try to infer the Coder for {@code VA} * values. + * + *

This is the Coder used to send data through a communication-intensive + * shuffle step, so a compact and efficient representation may have + * significant performance benefits. */ public Coder getAccumulatorCoder( CoderRegistry registry, Coder inputCoder) { @@ -476,14 +480,14 @@ public Coder getDefaultOutputCoder( * @param type of output values */ public abstract static class AccumulatingCombineFn - .Accumulator, VO> + , VO> extends CombineFn { /** * The type of mutable accumulator values used by this * {@code AccumulatingCombineFn}. */ - public abstract class Accumulator implements Serializable { + public abstract static interface Accumulator { /** * Adds the given input value to this accumulator, modifying * this accumulator. @@ -675,6 +679,10 @@ public VO apply(K key, Iterable inputs) { * used for {@code K} keys and input {@code VI} values and the * enclosing {@code Pipeline}'s {@code CoderRegistry} to try to * infer the Coder for {@code VA} values. + * + *

This is the Coder used to send data through a communication-intensive + * shuffle step, so a compact and efficient representation may have + * significant performance benefits. */ public Coder getAccumulatorCoder( CoderRegistry registry, Coder keyCoder, Coder inputCoder) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Mean.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Mean.java index b198a0da25041..c82f4ab45d5ef 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Mean.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Mean.java @@ -16,9 +16,16 @@ package com.google.cloud.dataflow.sdk.transforms; +import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.coders.CoderRegistry; -import com.google.cloud.dataflow.sdk.coders.SerializableCoder; +import com.google.cloud.dataflow.sdk.coders.CustomCoder; +import com.google.cloud.dataflow.sdk.coders.DoubleCoder; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; /** * {@code PTransform}s for computing the arithmetic mean @@ -102,11 +109,20 @@ public MeanFn() {} * Accumulator helper class for MeanFn. */ class CountSum - extends Combine.AccumulatingCombineFn.Accumulator { + implements Combine.AccumulatingCombineFn.Accumulator { long count = 0; double sum = 0.0; + public CountSum() { + this(0, 0); + } + + public CountSum(long count, double sum) { + this.count = count; + this.sum = sum; + } + @Override public void addInput(N element) { count++; @@ -130,14 +146,36 @@ public CountSum createAccumulator() { return new CountSum(); } + private static final Coder LONG_CODER = BigEndianLongCoder.of(); + private static final Coder DOUBLE_CODER = DoubleCoder.of(); + @SuppressWarnings("unchecked") @Override public Coder getAccumulatorCoder( CoderRegistry registry, Coder inputCoder) { - // The casts are needed because CountSum.class is a - // Class, but we need a - // Class.CountSum>. - return SerializableCoder.of((Class) (Class) CountSum.class); + return new CustomCoder () { + @Override + public void encode(CountSum value, OutputStream outStream, Coder.Context context) + throws CoderException, IOException { + Coder.Context nestedContext = context.nested(); + LONG_CODER.encode(value.count, outStream, nestedContext); + DOUBLE_CODER.encode(value.sum, outStream, nestedContext); + } + + @Override + public CountSum decode(InputStream inStream, Coder.Context context) + throws CoderException, IOException { + Coder.Context nestedContext = context.nested(); + return new CountSum( + LONG_CODER.decode(inStream, nestedContext), + DOUBLE_CODER.decode(inStream, nestedContext)); + } + + @Override + public boolean isDeterministic() { + return true; + } + }; } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java index 0f8a3f0565850..c4fb15bd23232 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java @@ -337,11 +337,7 @@ public & Serializable> TopCombineFn( this.compareFn = compareFn; } - class Heap - // TODO: Why do I have to fully qualify the - // Accumulator class here? - extends AccumulatingCombineFn.Heap, List> - .Accumulator { + class Heap implements AccumulatingCombineFn.Accumulator.Heap, List> { // Exactly one of these should be set. private List asList; // ordered largest first diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java index 160b5456d6193..4833ff19a3109 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java @@ -63,8 +63,8 @@ public class CombineValuesFnTest { public static class MeanInts extends Combine.AccumulatingCombineFn { - class CountSum extends - Combine.AccumulatingCombineFn.Accumulator { + class CountSum implements + Combine.AccumulatingCombineFn.Accumulator { long count; double sum; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java index 646977441e03b..c0d8e21b15e67 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java @@ -58,6 +58,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.io.Serializable; import java.util.Arrays; import java.util.HashSet; import java.util.List; @@ -365,8 +366,8 @@ public static class MeanInts extends private static final Coder LONG_CODER = BigEndianLongCoder.of(); private static final Coder DOUBLE_CODER = DoubleCoder.of(); - class CountSum extends - Combine.AccumulatingCombineFn.Accumulator { + class CountSum implements + Combine.AccumulatingCombineFn.Accumulator { long count = 0; double sum = 0.0; @@ -450,9 +451,9 @@ public static class TestCounter extends Integer, TestCounter.Counter, Iterable> { /** An accumulator that observes its merges and outputs */ - public class Counter extends - Combine.AccumulatingCombineFn< - Integer, Counter, Iterable>.Accumulator { + public class Counter implements + Combine.AccumulatingCombineFn.Accumulator>, + Serializable { public long sum = 0; public long inputs = 0; @@ -522,6 +523,8 @@ public Counter createAccumulator() { @Override public Coder getAccumulatorCoder( CoderRegistry registry, Coder inputCoder) { + // This is a *very* inefficient encoding to send over the wire, but suffices + // for tests. return SerializableCoder.of(Counter.class); } } From 0d4061a0416fb13532540c704a4e3909d26927d3 Mon Sep 17 00:00:00 2001 From: peihe Date: Mon, 15 Dec 2014 12:45:44 -0800 Subject: [PATCH 0016/1541] KeyedState: Add ParDo test that runs on service and triggers PARTITION_KEYS reshardings, and covers more cases on DirectPipelineRunner. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82166757 --- .../dataflow/sdk/transforms/ParDoTest.java | 79 ++++++++++++++++++- 1 file changed, 76 insertions(+), 3 deletions(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java index fdd557d08a7d4..a4fed2ada20b6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java @@ -35,10 +35,13 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.TestUtils; import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder; import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; +import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionTuple; import com.google.cloud.dataflow.sdk.values.PCollectionView; @@ -266,7 +269,25 @@ public void finishBundle(Context c) { } } - static class TestUnexpectedKeyedStateDoFn extends DoFn { + /** + * Output the keys which have appeared at least three times. + */ + static class TestKeyedStateCountAtLeastThreeDoFn + extends DoFn, String> implements DoFn.RequiresKeyedState{ + @Override + public void processElement(ProcessContext c) throws IOException { + String key = c.element().getKey(); + CodedTupleTag tag = CodedTupleTag.of(key, BigEndianLongCoder.of()); + Long result = c.keyedState().lookup(tag); + long count = result == null ? 0 : result; + c.keyedState().store(tag, ++count); + if (count == 3) { + c.output(key); + } + } + } + + static class TestUnexpectedKeyedStateDoFn extends DoFn, String> { @Override public void processElement(ProcessContext c) { // Will fail since this DoFn doesn't implement RequiresKeyedState. @@ -274,6 +295,15 @@ public void processElement(ProcessContext c) { } } + static class TestKeyedStateDoFnWithNonKvInput + extends DoFn implements DoFn.RequiresKeyedState { + @Override + public void processElement(ProcessContext c) { + // Will fail since this DoFn's input isn't KV. + c.keyedState(); + } + } + private static class StrangelyNamedDoer extends DoFn { @Override public void processElement(ProcessContext c) { @@ -646,13 +676,37 @@ public void testParDoWithErrorInFinishBatch() { } } + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testParDoKeyedState() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList( + "A", "A", "B", "C", "B", "A", "D", "D", "D", "D"); + + PCollection output = + p.apply(Create.of(inputs)) + .apply(ParDo.named("ToKv") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + c.output(KV.of(c.element(), 1)); + } + })) + .apply(ParDo.of(new TestKeyedStateCountAtLeastThreeDoFn())); + + DataflowAssert.that(output).containsInAnyOrder("A", "D"); + p.run(); + } + @Test public void testParDoWithUnexpectedKeyedState() { Pipeline p = TestPipeline.create(); - List inputs = Arrays.asList(3, -42, 666); + List> inputs = Arrays.asList( + KV.of("a", 1)); - PCollection input = createInts(p, inputs); + PCollection> input = p.apply(Create.of(inputs)); input .apply(ParDo.of(new TestUnexpectedKeyedStateDoFn())); @@ -666,6 +720,25 @@ public void testParDoWithUnexpectedKeyedState() { } } + @Test + public void testParDoKeyedStateDoFnWithNonKvInput() { + Pipeline p = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollection input = createInts(p, inputs); + + input + .apply(ParDo.of(new TestKeyedStateDoFnWithNonKvInput())); + try { + p.run(); + fail("should have failed"); + } catch (RuntimeException exn) { + assertThat(exn.toString(), + containsString("Keyed state is only available")); + } + } + @Test public void testParDoName() { Pipeline p = TestPipeline.create(); From c14dbdefd0418b01481d532d6bf8531726d5a13b Mon Sep 17 00:00:00 2001 From: lcwik Date: Mon, 15 Dec 2014 13:07:18 -0800 Subject: [PATCH 0017/1541] Bundle the SDK within the examples jar. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82168727 --- examples/pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index fcb52fcdbf8d9..ad847af861d8b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -108,10 +108,10 @@ 2.4.0 true - ${project.artifactId}-bundled-${project.version} + ${project.artifactId}-bundled-${project.version} - - *;scope=compile|runtime;artifactId=!google-cloud-dataflow-java-sdk-all;inline=true + + *;scope=compile|runtime;inline=true From 344a78dd0bf274150c09f0b6b3df0dfdd08455d0 Mon Sep 17 00:00:00 2001 From: earhart Date: Mon, 15 Dec 2014 13:27:01 -0800 Subject: [PATCH 0018/1541] Fixing up a few more warnings. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82170636 --- .../cloud/dataflow/sdk/coders/AvroCoder.java | 3 +-- .../dataflow/sdk/coders/ByteArrayCoder.java | 1 + .../dataflow/sdk/coders/CollectionCoder.java | 6 ++++++ .../dataflow/sdk/coders/IterableCoder.java | 6 ++++++ .../sdk/coders/IterableLikeCoder.java | 12 +++++++++-- .../cloud/dataflow/sdk/coders/ListCoder.java | 6 ++++++ .../cloud/dataflow/sdk/coders/MapCoder.java | 3 +-- .../sdk/coders/SerializableCoder.java | 1 + .../cloud/dataflow/sdk/coders/SetCoder.java | 1 + .../sdk/coders/TableRowJsonCoder.java | 1 + .../sdk/coders/TextualIntegerCoder.java | 1 + .../cloud/dataflow/sdk/coders/URICoder.java | 1 + .../cloud/dataflow/sdk/coders/VoidCoder.java | 1 + .../cloud/dataflow/sdk/io/DatastoreIO.java | 3 +++ .../sdk/options/PipelineOptionsFactory.java | 20 ++++++++++++------- .../sdk/options/ProxyInvocationHandler.java | 4 +++- .../sdk/runners/worker/SinkFactory.java | 1 + .../dataflow/sdk/testing/DataflowAssert.java | 10 ++++++++++ .../sdk/transforms/ApproximateUnique.java | 3 +++ .../dataflow/sdk/transforms/GroupByKey.java | 6 ++++++ .../dataflow/sdk/transforms/Partition.java | 1 + .../transforms/windowing/FixedWindows.java | 1 + .../sdk/util/GroupAlsoByWindowsDoFn.java | 1 + .../dataflow/sdk/util/UserCodeException.java | 1 + 24 files changed, 80 insertions(+), 14 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java index 9532da725ac97..ca1f3c856e6fa 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java @@ -80,9 +80,8 @@ * * @param the type of elements handled by this coder */ +@SuppressWarnings("serial") public class AvroCoder extends StandardCoder { - private static final long serialVersionUID = 0; - /** * Returns an {@code AvroCoder} instance for the provided element type. * @param the element type diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java index c750d932dd066..19d97f3b7e6b0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java @@ -32,6 +32,7 @@ * If in a nested context, prefixes the encoded array with a VarInt encoding * of the length. */ +@SuppressWarnings("serial") public class ByteArrayCoder extends AtomicCoder { @JsonCreator public static ByteArrayCoder of() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java index 546695dfefe80..c75f645d344ef 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java @@ -30,6 +30,7 @@ * * @param the type of the elements of the Collections being transcoded */ +@SuppressWarnings("serial") public class CollectionCoder extends IterableLikeCoder> { public static CollectionCoder of(Coder elemCoder) { @@ -39,6 +40,11 @@ public static CollectionCoder of(Coder elemCoder) { ///////////////////////////////////////////////////////////////////////////// // Internal operations below here. + @Override + protected final Collection decodeToIterable(List decodedElements) { + return decodedElements; + } + @JsonCreator public static CollectionCoder of( @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java index 801dd2042cfdd..79d6173742a88 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java @@ -32,6 +32,7 @@ * * @param the type of the elements of the Iterables being transcoded */ +@SuppressWarnings("serial") public class IterableCoder extends IterableLikeCoder> { public static IterableCoder of(Coder elemCoder) { @@ -41,6 +42,11 @@ public static IterableCoder of(Coder elemCoder) { ///////////////////////////////////////////////////////////////////////////// // Internal operations below here. + @Override + protected final Iterable decodeToIterable(List decodedElements) { + return decodedElements; + } + @JsonCreator public static IterableCoder of( @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java index e6ecdbe26bb97..128d23bf21a2c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java @@ -42,6 +42,14 @@ public abstract class IterableLikeCoder> public Coder getElemCoder() { return elemCoder; } + /** + * Builds an instance of the coder's associated {@code Iterable} from a list + * of decoded elements. If {@code IT} is a supertype of {@code List}, the + * derived class implementation is permitted to return {@code decodedElements} + * directly. + */ + protected abstract IT decodeToIterable(List decodedElements); + ///////////////////////////////////////////////////////////////////////////// // Internal operations below here. @@ -106,7 +114,7 @@ public IT decode(InputStream inStream, Context context) for (int i = 0; i < size; i++) { elements.add(elemCoder.decode(dataInStream, nestedContext)); } - return (IT) elements; + return decodeToIterable(elements); } else { // We don't know the size a priori. Check if we're done with // each element. @@ -114,7 +122,7 @@ public IT decode(InputStream inStream, Context context) while (dataInStream.readBoolean()) { elements.add(elemCoder.decode(dataInStream, nestedContext)); } - return (IT) elements; + return decodeToIterable(elements); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java index ab9d8147aa1f1..f6f04b60d34ff 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java @@ -29,6 +29,7 @@ * * @param the type of the elements of the Lists being transcoded */ +@SuppressWarnings("serial") public class ListCoder extends IterableLikeCoder> { public static ListCoder of(Coder elemCoder) { @@ -38,6 +39,11 @@ public static ListCoder of(Coder elemCoder) { ///////////////////////////////////////////////////////////////////////////// // Internal operations below here. + @Override + protected final List decodeToIterable(List decodedElements) { + return decodedElements; + } + @JsonCreator public static ListCoder of( @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java index 244419da6ac0e..1e12ef347e70f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java @@ -40,9 +40,8 @@ * @param the type of the keys of the KVs being transcoded * @param the type of the values of the KVs being transcoded */ +@SuppressWarnings("serial") public class MapCoder extends MapCoderBase> { - private static final long serialVersionUID = 0; - /** * Produces a MapCoder with the given keyCoder and valueCoder. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java index c078e6629a2b7..8e58b4d96e291 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java @@ -45,6 +45,7 @@ * * @param the type of elements handled by this coder */ +@SuppressWarnings("serial") public class SerializableCoder extends AtomicCoder { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java index 1a234c7b40ed8..8b5fca7638a5c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java @@ -38,6 +38,7 @@ * * @param the type of the elements of the set */ +@SuppressWarnings("serial") public class SetCoder extends StandardCoder> { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java index e49dfbb9c01c8..f21aaa513eb2f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java @@ -29,6 +29,7 @@ /** * A TableRowJsonCoder encodes BigQuery TableRow objects. */ +@SuppressWarnings("serial") public class TableRowJsonCoder extends AtomicCoder { @JsonCreator diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java index 93d080b7f01cd..bd01ecbcca7b7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java @@ -25,6 +25,7 @@ /** * A TextualIntegerCoder encodes Integers as text. */ +@SuppressWarnings("serial") public class TextualIntegerCoder extends AtomicCoder { @JsonCreator public static TextualIntegerCoder of() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/URICoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/URICoder.java index ed5ae45c53e77..eedcddf787e10 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/URICoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/URICoder.java @@ -28,6 +28,7 @@ * A {@code URICoder} encodes/decodes {@link URI}s by conversion to/from {@link String}, delegating * encoding/decoding of the string to {@link StringUtf8Coder}. */ +@SuppressWarnings("serial") public class URICoder extends AtomicCoder { @JsonCreator diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java index fc9a1e0958b24..24d5e061cfc73 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java @@ -24,6 +24,7 @@ /** * A VoidCoder encodes Voids. Uses zero bytes per Void. */ +@SuppressWarnings("serial") public class VoidCoder extends AtomicCoder { @JsonCreator public static VoidCoder of() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java index 9c7fc0a1c5b33..e7b05bc8a520b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java @@ -154,6 +154,7 @@ public static Bound withHost(String host) { * A PTransform that reads from a Datastore query and returns a bounded * {@code PCollection}. */ + @SuppressWarnings("serial") public static class Bound extends PTransform> { String host; String datasetId; @@ -268,6 +269,7 @@ public static Bound withHost(String host) { * A PTransform that writes a bounded {@code PCollection} * to a Datastore. */ + @SuppressWarnings("serial") public static class Bound extends PTransform, PDone> { String host; String datasetId; @@ -352,6 +354,7 @@ public void evaluate( * A DoFn that performs query request to Datastore and converts * each QueryOptions into Entities. */ + @SuppressWarnings("serial") private static class ReadEntitiesFn extends DoFn { @Override public void processElement(ProcessContext c) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java index a46ce313e3383..dae5ca14340f0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java @@ -385,19 +385,22 @@ static synchronized Registration validateWellForm Class iface, Set> validatedPipelineOptionsInterfaces) { Preconditions.checkArgument(iface.isInterface(), "Only interface types are supported."); + @SuppressWarnings("unchecked") Set> combinedPipelineOptionsInterfaces = FluentIterable.from(validatedPipelineOptionsInterfaces).append(iface).toSet(); // Validate that the view of all currently passed in options classes is well formed. if (!COMBINED_CACHE.containsKey(combinedPipelineOptionsInterfaces)) { - Class allProxyClass = Proxy.getProxyClass(PipelineOptionsFactory.class.getClassLoader(), - combinedPipelineOptionsInterfaces.toArray(EMPTY_CLASS_ARRAY)); + @SuppressWarnings("unchecked") + Class allProxyClass = + (Class) Proxy.getProxyClass(PipelineOptionsFactory.class.getClassLoader(), + combinedPipelineOptionsInterfaces.toArray(EMPTY_CLASS_ARRAY)); try { List propertyDescriptors = getPropertyDescriptors(allProxyClass); validateClass(iface, validatedPipelineOptionsInterfaces, allProxyClass, propertyDescriptors); COMBINED_CACHE.put(combinedPipelineOptionsInterfaces, - new Registration((Class) allProxyClass, propertyDescriptors)); + new Registration(allProxyClass, propertyDescriptors)); } catch (IntrospectionException e) { throw Throwables.propagate(e); } @@ -405,20 +408,22 @@ static synchronized Registration validateWellForm // Validate that the local view of the class is well formed. if (!INTERFACE_CACHE.containsKey(iface)) { - @SuppressWarnings("rawtypes") - Class proxyClass = Proxy.getProxyClass( + @SuppressWarnings({"rawtypes", "unchecked"}) + Class proxyClass = (Class) Proxy.getProxyClass( PipelineOptionsFactory.class.getClassLoader(), new Class[] {iface}); try { List propertyDescriptors = getPropertyDescriptors(proxyClass); validateClass(iface, validatedPipelineOptionsInterfaces, proxyClass, propertyDescriptors); INTERFACE_CACHE.put(iface, - new Registration((Class) proxyClass, propertyDescriptors)); + new Registration(proxyClass, propertyDescriptors)); } catch (IntrospectionException e) { throw Throwables.propagate(e); } } - return (Registration) INTERFACE_CACHE.get(iface); + @SuppressWarnings("unchecked") + Registration result = (Registration) INTERFACE_CACHE.get(iface); + return result; } public static Set> getRegisteredOptions() { @@ -822,6 +827,7 @@ private static Map parseObjects( Class klass, ListMultimap options) { Map propertyNamesToGetters = Maps.newHashMap(); PipelineOptionsFactory.validateWellFormed(klass, getRegisteredOptions()); + @SuppressWarnings("unchecked") Iterable propertyDescriptors = PipelineOptionsFactory.getPropertyDescriptors( FluentIterable.from(getRegisteredOptions()).append(klass).toSet()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java index 88e84f6142313..cbe9c422b325b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java @@ -105,7 +105,9 @@ public Object invoke(Object proxy, Method method, Object[] args) { } else if (args == null && "hashCode".equals(method.getName())) { return hashCode(); } else if (args != null && "as".equals(method.getName()) && args[0] instanceof Class) { - return as((Class) args[0]); + @SuppressWarnings("unchecked") + Class clazz = (Class) args[0]; + return as(clazz); } String methodName = method.getName(); synchronized (this) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactory.java index df2d5ac754281..590e7f8151347 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactory.java @@ -63,6 +63,7 @@ private SinkFactory() {} * @throws Exception if the sink could not be decoded and * constructed */ + @SuppressWarnings("serial") public static Sink create( PipelineOptions options, com.google.api.services.dataflow.model.Sink cloudSink, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java index d4fe32ffd86f3..d0ce9ff62b3f1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java @@ -98,6 +98,7 @@ public static IterableAssert thatIterable( /** * An assertion about the contents of a {@link PCollectionView<, ?>} */ + @SuppressWarnings("serial") public static class IterableAssert implements Serializable { private final PCollectionView, ?> actualResults; @@ -175,6 +176,7 @@ public IterableAssert containsInOrder(Collection expectedElements) { * operation using a {@code Matcher} operation that takes an array * of elements. */ + @SuppressWarnings("serial") static class AssertThatIterable extends AssertThat, T[]> { AssertThatIterable(T[] expected, String matcherClassName, @@ -188,6 +190,7 @@ static class AssertThatIterable extends AssertThat, T[]> { * SerializableFunction that verifies that an Iterable contains * expected items in any order. */ + @SuppressWarnings("serial") static class AssertContainsInAnyOrder extends AssertThatIterable { AssertContainsInAnyOrder(T... expected) { super(expected, @@ -204,6 +207,7 @@ static class AssertContainsInAnyOrder extends AssertThatIterable { * SerializableFunction that verifies that an Iterable contains * expected items in the provided order. */ + @SuppressWarnings("serial") static class AssertContainsInOrder extends AssertThatIterable { AssertContainsInOrder(T... expected) { super(expected, @@ -230,6 +234,7 @@ public static SingletonAssert thatSingleton(PCollection futureResult) /** * An assertion about a single value. */ + @SuppressWarnings("serial") public static class SingletonAssert implements Serializable { private final PCollectionView actualResult; @@ -274,6 +279,7 @@ public SingletonAssert is(T expectedValue) { * SerializableFunction that performs an {@code Assert.assertThat()} * operation using a {@code Matcher} operation that takes a single element. */ + @SuppressWarnings("serial") static class AssertThatValue extends AssertThat { AssertThatValue(T expected, String matcherClassName, @@ -287,6 +293,7 @@ static class AssertThatValue extends AssertThat { * SerializableFunction that verifies that a value is equal to an * expected value. */ + @SuppressWarnings("serial") public static class AssertIs extends AssertThatValue { AssertIs(T expected) { super(expected, "org.hamcrest.core.IsEqual", "equalTo"); @@ -308,6 +315,7 @@ private DataflowAssert() {} * produce a Matcher to be used to check an {@code Actual} value * against. */ + @SuppressWarnings("serial") public static class AssertThat implements SerializableFunction { final Expected expected; @@ -353,6 +361,7 @@ public Void apply(Actual in) { * SerializableFunction that performs an {@code Assert.assertThat()} * operation using a {@code Matcher} operation that takes a single element. */ + @SuppressWarnings("serial") static class AssertThatValue extends AssertThat { AssertThatValue(T expected, String matcherClassName, @@ -366,6 +375,7 @@ static class AssertThatValue extends AssertThat { * SerializableFunction that verifies that a value is equal to an * expected value. */ + @SuppressWarnings("serial") public static class AssertIs extends AssertThatValue { public AssertIs(T expected) { super(expected, "org.hamcrest.core.IsEqual", "equalTo"); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java index 9308a010a2a9e..39612397de172 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java @@ -156,6 +156,7 @@ public static PerKey perKey(double maximumEstimationError) { * * @param the type of the elements in the input {@code PCollection} */ + @SuppressWarnings("serial") static class Globally extends PTransform, PCollection> { /** @@ -212,6 +213,7 @@ protected String getKindString() { * {@code PCollection}s * @param the type of the values in the input {@code PCollection} */ + @SuppressWarnings("serial") static class PerKey extends PTransform>, PCollection>> { @@ -281,6 +283,7 @@ protected String getKindString() { * * @param the type of the values being combined */ + @SuppressWarnings("serial") public static class ApproximateUniqueCombineFn extends CombineFn { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index 72e475f31002a..e8acdbad5be39 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -235,6 +235,7 @@ public GroupAlsoByWindow(WindowingFn windowingFn) { } @Override + @SuppressWarnings("unchecked") public PCollection>> apply( PCollection>>> input) { @SuppressWarnings("unchecked") @@ -371,6 +372,11 @@ boolean sortsValues() { ///////////////////////////////////////////////////////////////////////////// static { + registerWithDirectPipelineRunner(); + } + + @SuppressWarnings("rawtypes") + private static void registerWithDirectPipelineRunner() { DirectPipelineRunner.registerDefaultTransformEvaluator( GroupByKeyOnly.class, new DirectPipelineRunner.TransformEvaluator() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java index a6444b2774f4c..7e5cc00f62a9f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java @@ -61,6 +61,7 @@ * @param the type of the elements of the input and output * {@code PCollection}s */ +@SuppressWarnings("serial") public class Partition extends PTransform, PCollectionList> { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java index 91c591c901900..d381a80627b73 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java @@ -31,6 +31,7 @@ * Window.by(FixedWindows.of(Duration.standardMinutes(10)))); * }

*/ +@SuppressWarnings("serial") public class FixedWindows extends PartitioningWindowingFn { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java index 62ae4875f9651..56bba7d083feb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java @@ -47,6 +47,7 @@ * @param input value element type * @param window type */ +@SuppressWarnings("serial") public class GroupAlsoByWindowsDoFn extends DoFn>>, KV>> { // TODO: Add back RequiresKeyed state once that is supported. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UserCodeException.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UserCodeException.java index a0bfed1626f92..840261d591492 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UserCodeException.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UserCodeException.java @@ -29,6 +29,7 @@ * reached. */ public class UserCodeException extends RuntimeException { + private static final long serialVersionUID = 0; private static final Logger LOG = LoggerFactory.getLogger(UserCodeException.class); public UserCodeException(Throwable t) { From f2cd3ceb8bff483d160d9949935b5b2e51b67f11 Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 15 Dec 2014 14:16:57 -0800 Subject: [PATCH 0019/1541] Make checkstyle violations errors. Fix existing violations. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82175161 --- checkstyle.xml | 31 ++++++----- .../examples/TopWikipediaSessions.java | 2 +- .../cloud/dataflow/examples/WordCount.java | 4 +- .../cloud/dataflow/examples/TfIdfTest.java | 4 +- .../cloud/dataflow/sdk/coders/AvroCoder.java | 2 +- .../cloud/dataflow/sdk/io/DatastoreIO.java | 10 ++-- .../cloud/dataflow/sdk/io/PubsubIO.java | 21 ++++---- .../sdk/options/DataflowPipelineOptions.java | 2 +- .../dataflow/sdk/options/GcpOptions.java | 2 +- .../dataflow/sdk/options/GcsOptions.java | 2 +- .../sdk/options/PipelineOptionsFactory.java | 53 ++++++++++--------- .../runners/DataflowPipelineTranslator.java | 8 +-- .../sdk/runners/worker/CombineValuesFn.java | 19 ++++--- .../worker/DataflowWorkProgressUpdater.java | 2 +- .../runners/worker/GroupingShuffleSource.java | 2 +- .../dataflow/sdk/testing/DataflowAssert.java | 2 +- .../transforms/windowing/GlobalWindow.java | 4 +- .../cloud/dataflow/sdk/util/GcsUtil.java | 10 ++-- .../sdk/util/GroupAlsoByWindowsDoFn.java | 2 +- .../worker/BatchingShuffleEntryReader.java | 2 +- .../dataflow/sdk/io/DatastoreIOTest.java | 2 +- .../options/PipelineOptionsFactoryTest.java | 8 +-- .../sdk/options/PipelineOptionsTest.java | 2 +- .../options/ProxyInvocationHandlerTest.java | 8 +-- .../worker/GroupingShuffleSourceTest.java | 14 ++--- .../worker/PartitioningShuffleSourceTest.java | 10 ++-- .../sdk/runners/worker/ShuffleSinkTest.java | 16 +++--- .../worker/UngroupedShuffleSourceTest.java | 10 ++-- .../transforms/ApproximateQuantilesTest.java | 7 ++- .../dataflow/sdk/transforms/CombineTest.java | 4 +- .../sdk/transforms/PartitionTest.java | 8 ++- .../dataflow/sdk/transforms/TopTest.java | 9 ++-- .../cloud/dataflow/sdk/values/KVTest.java | 18 +++---- .../cloud/dataflow/sdk/values/PDoneTest.java | 9 ++-- 34 files changed, 158 insertions(+), 151 deletions(-) diff --git a/checkstyle.xml b/checkstyle.xml index 702558ac34acb..c6c07ef9f1c98 100644 --- a/checkstyle.xml +++ b/checkstyle.xml @@ -132,7 +132,7 @@ page at http://checkstyle.sourceforge.net/config.html --> - + @@ -147,7 +147,10 @@ page at http://checkstyle.sourceforge.net/config.html --> - + + + - + @@ -186,7 +189,7 @@ page at http://checkstyle.sourceforge.net/config.html --> - + @@ -198,7 +201,7 @@ page at http://checkstyle.sourceforge.net/config.html --> - + @@ -209,32 +212,32 @@ page at http://checkstyle.sourceforge.net/config.html --> - + - + - + - + - + @@ -262,7 +265,7 @@ page at http://checkstyle.sourceforge.net/config.html --> - + @@ -283,12 +286,12 @@ page at http://checkstyle.sourceforge.net/config.html --> --> - + - + @@ -376,7 +379,7 @@ page at http://checkstyle.sourceforge.net/config.html --> - + diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java b/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java index a64e2fc98f49a..f58871be308d2 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/TopWikipediaSessions.java @@ -73,7 +73,7 @@ public class TopWikipediaSessions { private static final String EXPORTED_WIKI_TABLE = "gs://dataflow-samples/wikipedia_edits/*.json"; /** - * Extracts user and timestamp from a TableRow representing a Wikipedia edit + * Extracts user and timestamp from a TableRow representing a Wikipedia edit. */ static class ExtractUserAndTimestamp extends DoFn { @Override diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java b/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java index 948b83e032fab..6b1623005785d 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java @@ -138,7 +138,9 @@ public static interface Options extends PipelineOptions { String getOutput(); void setOutput(String value); - /** Returns gs://${STAGING_LOCATION}/"counts.txt" */ + /** + * Returns gs://${STAGING_LOCATION}/"counts.txt" as the default destination. + */ public static class OutputFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/TfIdfTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/TfIdfTest.java index 341fd80c25b2b..990458f5e059c 100644 --- a/examples/src/test/java/com/google/cloud/dataflow/examples/TfIdfTest.java +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/TfIdfTest.java @@ -34,12 +34,12 @@ import java.util.Arrays; /** - * Tests of TfIdf + * Tests of {@link TfIdf}. */ @RunWith(JUnit4.class) public class TfIdfTest { - /** Test that the example runs */ + /** Test that the example runs. */ @Test @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) public void testTfIdf() throws Exception { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java index ca1f3c856e6fa..db0000cd25b60 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java @@ -58,7 +58,7 @@ * null unless annotated by * * org.apache.avro.reflect.Nullable or a - * + * * org.apache.avro.reflect.Union containing null. *

* To use, specify the {@code Coder} type on a PCollection: diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java index e7b05bc8a520b..1b66b868c97ed 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java @@ -93,7 +93,7 @@ *

 {@code
  * // Read a query from Datastore
  * PipelineOptions options =
- *     CliPipelineOptionsFactory.create(PipelineOptions.class, args);
+ *     PipelineOptionsFactory.fromArgs(args).create();
  * Pipeline p = Pipeline.create(options);
  * PCollection entities =
  *     p.apply(DatastoreIO.Read
@@ -233,12 +233,12 @@ public PCollection apply(PBegin input) {
 
   ///////////////////// Write Class /////////////////////////////////
   /**
-   * A PTransform that writes a {@code PCollection} containing
+   * A {@link PTransform} that writes a {@code PCollection} containing
    * entities to a Datastore kind.
    *
-   * Current version only supports Write operation running on
-   * DirectPipelineRunner.  If Write is used on DataflowPipelineRunner,
-   * it throws UnsupportedOperationException and won't continue on the
+   * 

Current version only supports Write operation running on + * {@link DirectPipelineRunner}. If Write is used on {@link DataflowPipelineRunner}, + * it throws {@link UnsupportedOperationException} and won't continue on the * operation. * */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index 5d87f2563fa52..f5b57f82fb5dc 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -140,9 +140,11 @@ public static Bound named(String name) { /** * Creates and returns a PubsubIO.Read PTransform for reading from * a Pubsub topic with the specified publisher topic. Format for - * Cloud Pubsub topic names should be of the form /topics//, - * where is the name of the publishing project. - * The component must comply with the below requirements. + * Cloud Pubsub topic names should be of the form + * {@code /topics//}, where {@code } is the name of + * the publishing project. The {@code } component must comply with + * the below requirements. + * *

    *
  • Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods * ('.').
  • @@ -161,9 +163,10 @@ public static Bound topic(String topic) { * a specific Pubsub subscription. Mutually exclusive with * PubsubIO.Read.topic(). * Cloud Pubsub subscription names should be of the form - * /subscriptions//<, - * where is the name of the project the subscription belongs to. - * The component must comply with the below requirements. + * {@code /subscriptions//<}, + * where {@code } is the name of the project the subscription belongs to. + * The {@code } component must comply with the below requirements. + * *
      *
    • Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods * ('.').
    • @@ -186,7 +189,7 @@ public static class Bound extends PTransform> { /** The Pubsub topic to read from. */ String topic; - /** The Pubsub subscription to read from */ + /** The Pubsub subscription to read from. */ String subscription; Bound() {} @@ -268,8 +271,8 @@ public static Bound named(String name) { } /** The topic to publish to. - * Cloud Pubsub topic names should be /topics//, - * where is the name of the publishing project. + * Cloud Pubsub topic names should be {@code /topics//}, + * where {@code } is the name of the publishing project. */ public static Bound topic(String topic) { return new Bound().topic(topic); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java index 686d72e20c2e6..d30f7dc4d96b3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java @@ -112,7 +112,7 @@ public String create(PipelineOptions options) { } } - /** Alternative Dataflow client */ + /** Alternative Dataflow client. */ @JsonIgnore @Default.InstanceFactory(DataflowClientFactory.class) Dataflow getDataflowClient(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java index 34faf03acfb66..959a4fc77f3fb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java @@ -126,7 +126,7 @@ public String create(PipelineOptions options) { String getCredentialId(); void setCredentialId(String value); - /** Alternative Google Cloud Platform Credential */ + /** Alternative Google Cloud Platform Credential. */ @JsonIgnore @Description("Google Cloud Platform user credentials.") @Default.InstanceFactory(GcpUserCredentialsFactory.class) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java index 20685b7cf383f..39614242f7f7c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java @@ -33,7 +33,7 @@ */ public interface GcsOptions extends ApplicationNameOptions, GcpOptions, PipelineOptions { - /** Alternative GcsUtil instance */ + /** Alternative GcsUtil instance. */ @JsonIgnore @Default.InstanceFactory(GcsUtil.GcsUtilFactory.class) GcsUtil getGcsUtil(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java index dae5ca14340f0..176ff8c96e509 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java @@ -100,14 +100,14 @@ public static PipelineOptions create() { } /** - * Creates and returns an object which implements @{code }. + * Creates and returns an object which implements {@code }. * This sets the {@link ApplicationNameOptions#getAppName() "appName"} to the calling * {@link Class#getSimpleName() classes simple name}. - *

      - * Note that @{code } must be composable with every registered interface with this factory. + * + *

      Note that {@code } must be composable with every registered interface with this factory. * See {@link PipelineOptionsFactory#validateWellFormed(Class, Set)} for more details. * - * @return An object which implements @{code }. + * @return An object which implements {@code }. */ public static T as(Class klass) { return new Builder(getAppName(3)).as(klass); @@ -124,13 +124,13 @@ public static T as(Class klass) { * --x=1 --x=2 --x=3 (list style property, will set the "x" property to [1, 2, 3]) * --x=1,2,3 (shorthand list style property, will set the "x" property to [1, 2, 3]) *

- * Properties are able to bound to {@link String} and Java primitives @{code boolean}, - * @{code byte}, @{code short}, @{code int}, @{code long}, @{code float}, @{code double} and + * Properties are able to bound to {@link String} and Java primitives {@code boolean}, + * {@code byte}, {@code short}, {@code int}, {@code long}, {@code float}, {@code double} and * their primitive wrapper classes. *

- * List style properties are able to be bound to @{code boolean[]}, @{code char[]}, - * @{code short[]}, @{code int[]}, @{code long[]}, @{code float[]}, @{code double[]}, - * @{code String[]} and @{code List}. + * List style properties are able to be bound to {@code boolean[]}, {@code char[]}, + * {@code short[]}, {@code int[]}, {@code long[]}, {@code float[]}, {@code double[]}, + * {@code String[]} and {@code List}. */ public static Builder fromArgs(String[] args) { return new Builder(getAppName(3)).fromArgs(args); @@ -174,13 +174,13 @@ private Builder(String defaultAppName, String[] args, boolean validation) { * --x=1 --x=2 --x=3 (list style property, will set the "x" property to [1, 2, 3]) * --x=1,2,3 (shorthand list style property, will set the "x" property to [1, 2, 3]) * - * Properties are able to bound to {@link String} and Java primitives @{code boolean}, - * @{code byte}, @{code short}, @{code int}, @{code long}, @{code float}, @{code double} and + * Properties are able to bound to {@link String} and Java primitives {@code boolean}, + * {@code byte}, {@code short}, {@code int}, {@code long}, {@code float}, {@code double} and * their primitive wrapper classes. *

- * List style properties are able to be bound to @{code boolean[]}, @{code char[]}, - * @{code short[]}, @{code int[]}, @{code long[]}, @{code float[]}, @{code double[]}, - * @{code String[]} and @{code List}. + * List style properties are able to be bound to {@code boolean[]}, {@code char[]}, + * {@code short[]}, {@code int[]}, {@code long[]}, {@code float[]}, {@code double[]}, + * {@code String[]} and {@code List}. */ public Builder fromArgs(String[] args) { Preconditions.checkNotNull(args, "Arguments should not be null."); @@ -208,13 +208,13 @@ public PipelineOptions create() { } /** - * Creates and returns an object which implements @{code } using the values configured on + * Creates and returns an object which implements {@code } using the values configured on * this builder during construction. *

* Note that {@code } must be composable with every registered interface with this factory. * See {@link PipelineOptionsFactory#validateWellFormed(Class, Set)} for more details. * - * @return An object which implements @{code }. + * @return An object which implements {@code }. */ public T as(Class klass) { Map initialOptions = Maps.newHashMap(); @@ -296,7 +296,7 @@ Class getProxyClass() { BlockingDataflowPipelineRunner.class) .build(); - /** Methods which are ignored when validating the proxy class */ + /** Methods which are ignored when validating the proxy class. */ private static final Set IGNORED_METHODS; /** The set of options which have been registered and visible to the user. */ @@ -779,8 +779,9 @@ public boolean apply(Method input) { /** * Splits string arguments based upon expected pattern of --argName=value. - *

- * Example GNU style command line arguments: + * + *

Example GNU style command line arguments: + * *

    *   --project=MyProject (simple property, will set the "project" property to "MyProject")
    *   --readOnly=true (for boolean properties, will set the "readOnly" property to "true")
@@ -788,12 +789,14 @@ public boolean apply(Method input) {
    *   --x=1 --x=2 --x=3 (list style property, will set the "x" property to [1, 2, 3])
    *   --x=1,2,3 (shorthand list style property, will set the "x" property to [1, 2, 3])
    * 
- * Properties are able to bound to {@link String} and Java primitives boolean, byte, - * short, int, long, float, double and their primitive wrapper classes. - *

- * List style properties are able to be bound to boolean[], char[], short[], - * int[], long[], float[], double[], String[] and List. - *

+ * + *

Properties are able to bound to {@link String} and Java primitives {@code boolean}, + * {@code byte}, {@code short}, {@code int}, {@code long}, {@code float}, {@code double} + * and their primitive wrapper classes. + * + *

List style properties are able to be bound to {@code boolean[]}, {@code char[]}, + * {@code short[]}, {@code int[]}, {@code long[]}, {@code float[]}, {@code double[]}, + * {@code String[]}, and {@code List}. */ private static ListMultimap parseCommandLine(String[] args) { ImmutableListMultimap.Builder builder = ImmutableListMultimap.builder(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index 823b9d44a1f6e..52305ccd54966 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -134,7 +134,7 @@ private DataflowPipelineTranslator(DataflowPipelineOptions options) { } /** - * Translates a Pipeline into a Job + * Translates a {@link Pipeline} into a {@code Job}. */ public Job translate(Pipeline pipeline, List packages) { Translator translator = new Translator(pipeline); @@ -174,7 +174,7 @@ TransformTranslator getTransformTranslator(Class transformClass) { } /** - * An translator of a PTransform. + * A translator of a {@link PTransform}. */ public interface TransformTranslator { public void translate(PT transform, @@ -183,8 +183,8 @@ public void translate(PT transform, /** * The interface provided to registered callbacks for interacting - * with the DataflowPipelineRunner, including reading and writing the - * values of PCollections and side inputs ({@link PCollectionViews}). + * with the {@link DataflowPipelineRunner}, including reading and writing the + * values of {@link PCollection}s and side inputs ({@link PCollectionView}s). */ public interface TranslationContext { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java index 62a371d05271f..761aaa950dba4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java @@ -49,11 +49,10 @@ public class CombineValuesFn extends NormalParDoFn { * 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. */ + // 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 static class CombinePhase { public static final String ALL = "all"; public static final String ADD = "add"; @@ -149,8 +148,8 @@ public void processElement(ProcessContext c) { } } - /** - * ADD phase: KV> -> KV + /* + * ADD phase: KV> -> KV. */ private static class AddInputsDoFn extends DoFn>, KV>{ @@ -174,8 +173,8 @@ public void processElement(ProcessContext c) { } } - /** - * MERGE phase: KV> -> KV + /* + * MERGE phase: KV> -> KV. */ private static class MergeAccumulatorsDoFn extends DoFn>, KV>{ @@ -196,8 +195,8 @@ public void processElement(ProcessContext c) { } } - /** - * EXTRACT phase: KV> -> KV + /* + * EXTRACT phase: KV> -> KV. */ private static class ExtractOutputDoFn extends DoFn, KV>{ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java index f2d41cfcbc45d..f2569b1488f6e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java @@ -45,7 +45,7 @@ public class DataflowWorkProgressUpdater extends WorkProgressUpdater { private static final Logger LOG = LoggerFactory.getLogger(DataflowWorkProgressUpdater.class); - /** The Dataflow Worker WorkItem client */ + /** The Dataflow Worker WorkItem client. */ private final DataflowWorker.WorkUnitClient workUnitClient; /** The WorkItem for which work progress updates are sent. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java index 2d168879a21b7..ac2bd5c3a78fe 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java @@ -158,7 +158,7 @@ private final class GroupingShuffleSourceIterator */ private ByteArrayShufflePosition stopPosition = null; - /** The next group to be consumed, if available */ + /** The next group to be consumed, if available. */ private KeyGroupedShuffleEntries nextGroup = null; public GroupingShuffleSourceIterator(ShuffleEntryReader reader) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java index d0ce9ff62b3f1..3debc9ff4f034 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java @@ -96,7 +96,7 @@ public static IterableAssert thatIterable( } /** - * An assertion about the contents of a {@link PCollectionView<, ?>} + * An assertion about the contents of a {@link PCollectionView<, ?>}. */ @SuppressWarnings("serial") public static class IterableAssert implements Serializable { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java index 5d0773d598a52..507b1cc860783 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java @@ -51,7 +51,7 @@ public Coder windowCoder() { * The default window into which all data is placed. */ public static class Window extends BoundedWindow { - public static Window INSTANCE = new Window(); + public static final Window INSTANCE = new Window(); @Override public Instant maxTimestamp() { @@ -64,7 +64,7 @@ private Window() {} * {@link Coder} for encoding and decoding {@code Window}s. */ public static class Coder extends AtomicCoder { - public static Coder INSTANCE = new Coder(); + public static final Coder INSTANCE = new Coder(); @Override public void encode(Window window, OutputStream outStream, Context context) {} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java index cc429955f5662..bcb387f29a6ed 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java @@ -42,8 +42,6 @@ /** * Provides operations on GCS. - * - * TODO: re-implement as a FileSystemProvider? */ public class GcsUtil { /** @@ -92,7 +90,7 @@ public GcsUtil create(PipelineOptions options) { ///////////////////////////////////////////////////////////////////////////// - /** Client for the GCS API */ + /** Client for the GCS API. */ private final Storage storage; // Helper delegate for turning IOExceptions from API calls into higher-level semantics. @@ -110,7 +108,7 @@ private GcsUtil(Storage storageClient, ExecutorService executorService) { * Expands a pattern into matched paths. The input path may contain * globs (in the last component only!), which are expanded in the result. * - * TODO: add support for full path matching. + *

TODO: add support for full path matching. */ public List expand(GcsPath path) throws IOException { if (!GCS_READ_PATTERN.matcher(path.getObject()).matches()) { @@ -187,7 +185,7 @@ public long fileSize(GcsPath path) throws IOException { /** * Opens an object in GCS. * - * Returns a SeekableByteChannel which provides access to data in the bucket. + *

Returns a SeekableByteChannel which provides access to data in the bucket. * * @param path the GCS filename to read from * @return a SeekableByteChannel which can read the object data @@ -202,7 +200,7 @@ public SeekableByteChannel open(GcsPath path) /** * Creates an object in GCS. * - * Returns a WritableByteChannel which can be used to write data to the + *

Returns a WritableByteChannel which can be used to write data to the * object. * * @param path the GCS file to write to diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java index 56bba7d083feb..f4bd28039686f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java @@ -344,7 +344,7 @@ public boolean hasMoreWindows() { } /** - * Returns the timestamp of the next window + * Returns the timestamp of the next window. */ public Instant nextTimestamp() { return windows.peek().maxTimestamp(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/BatchingShuffleEntryReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/BatchingShuffleEntryReader.java index 2a596c0d86f86..47cfa3646dec6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/BatchingShuffleEntryReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/BatchingShuffleEntryReader.java @@ -36,7 +36,7 @@ public final class BatchingShuffleEntryReader implements ShuffleEntryReader { private final ShuffleBatchReader batchReader; /** - * Constructs a {@link BatchingShuffleEntryReader} + * Constructs a {@link BatchingShuffleEntryReader}. * * @param batchReader supplies the underlying * {@link ShuffleBatchReader} to read batches of entries from diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java index e026c58102dae..fba8ec20b3fd0 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java @@ -67,7 +67,7 @@ public void setUp() { } /** - * Test for reading one entity from kind "food" + * Test for reading one entity from kind "food". */ @Test @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java index ca1e9502bf97d..206a13f70d171 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java @@ -235,7 +235,7 @@ public void testPropertyIsSetOnRegisteredPipelineOptionNotPartOfOriginalInterfac assertEquals("testProject", options.as(GcpOptions.class).getProject()); } - /** A test interface containing all the primitives */ + /** A test interface containing all the primitives. */ public static interface Primitives extends PipelineOptions { boolean getBoolean(); void setBoolean(boolean value); @@ -286,7 +286,7 @@ public void testBooleanShorthandArgument() { assertTrue(options.getBoolean()); } - /** A test interface containing all supported objects */ + /** A test interface containing all supported objects. */ public static interface Objects extends PipelineOptions { Boolean getBoolean(); void setBoolean(Boolean value); @@ -345,7 +345,7 @@ public void testMissingArgument() { assertNull(options.getString()); } - /** A test interface containing all supported array return types */ + /** A test interface containing all supported array return types. */ public static interface Arrays extends PipelineOptions { boolean[] getBoolean(); void setBoolean(boolean[] value); @@ -429,7 +429,7 @@ public void testOutOfOrderArrays() { assertArrayEquals(new char[] {'d', 'e', 'f'}, options.getChar()); } - /** A test interface containing all supported List return types */ + /** A test interface containing all supported List return types. */ public static interface Lists extends PipelineOptions { List getString(); void setString(List value); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsTest.java index 9db6a6b754221..1a11039504a6d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsTest.java @@ -28,7 +28,7 @@ /** Unit tests for {@link PipelineOptions}. */ @RunWith(JUnit4.class) public class PipelineOptionsTest { - /** Interface used for testing that {@link PipelineOptions#as(Class)} functions */ + /** Interface used for testing that {@link PipelineOptions#as(Class)} functions. */ public static interface TestOptions extends PipelineOptions { } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java index d33d42e4bf165..0b2dc0a9adfba 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java @@ -387,7 +387,7 @@ public void testJsonConversionForDefault() throws Exception { assertNotNull(serializeDeserialize(PipelineOptions.class, options)); } - /** Test interface for JSON conversion of simple types */ + /** Test interface for JSON conversion of simple types. */ private static interface SimpleTypes extends PipelineOptions { int getInteger(); void setInteger(int value); @@ -440,7 +440,7 @@ public void testJsonConversionForOverriddenSerializedValues() throws Exception { assertEquals("TestValue", options3.getString()); } - /** Test interface for JSON conversion of container types */ + /** Test interface for JSON conversion of container types. */ private static interface ContainerTypes extends PipelineOptions { List getList(); void setList(List values); @@ -465,7 +465,7 @@ public void testJsonConversionForContainerTypes() throws Exception { assertEquals(set, options2.getSet()); } - /** Test interface for conversion of inner types */ + /** Test interface for conversion of inner types. */ private static class InnerType { public double doubleField; @@ -483,7 +483,7 @@ && getClass().equals(obj.getClass()) } } - /** Test interface for conversion of generics and inner types */ + /** Test interface for conversion of generics and inner types. */ private static class ComplexType { public String stringField; public Integer intField; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java index b41bd1b2e2914..eacb1ef7661b5 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java @@ -64,12 +64,12 @@ */ @RunWith(JUnit4.class) public class GroupingShuffleSourceTest { - static final List>> NO_KVS = Collections.emptyList(); + private static final List>> NO_KVS = Collections.emptyList(); - static final Instant timestamp = new Instant(123000); - static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); + private static final Instant timestamp = new Instant(123000); + private static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); - static final List>> KVS = Arrays.asList( + private static final List>> KVS = Arrays.asList( KV.of(1, Arrays.asList("in 1a", "in 1b")), KV.of(2, Arrays.asList("in 2a", "in 2b")), KV.of(3, Arrays.asList("in 3")), @@ -77,7 +77,7 @@ public class GroupingShuffleSourceTest { KV.of(5, Arrays.asList("in 5"))); /** How many of the values with each key are to be read. */ - enum ValuesToRead { + private enum ValuesToRead { /** Don't even ask for the values iterator. */ SKIP_VALUES, /** Get the iterator, but don't read any values. */ @@ -88,7 +88,7 @@ enum ValuesToRead { READ_ALL_VALUES } - void runTestReadShuffleSource(List>> input, + private void runTestReadShuffleSource(List>> input, ValuesToRead valuesToRead) throws Exception { Coder> elemCoder = @@ -319,7 +319,7 @@ public void testReadFromShuffleSourceAndFailToUpdateStopPosition() int i = 0; for (; iter.hasNext(); ++i) { - KV> elem = iter.next().getValue(); + iter.next().getValue(); // ignored if (i == 0) { // First record byte[] key = CoderUtils.encodeToByteArray(BigEndianIntegerCoder.of(), i); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceTest.java index be8c972c5944a..2b88456011dc4 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceTest.java @@ -47,12 +47,12 @@ */ @RunWith(JUnit4.class) public class PartitioningShuffleSourceTest { - static final List>> NO_KVS = Collections.emptyList(); + private static final List>> NO_KVS = Collections.emptyList(); - static final Instant timestamp = new Instant(123000); - static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); + private static final Instant timestamp = new Instant(123000); + private static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); - static final List>> KVS = Arrays.asList( + private static final List>> KVS = Arrays.asList( WindowedValue.of(KV.of(1, "in 1a"), timestamp, Lists.newArrayList(window)), WindowedValue.of(KV.of(1, "in 1b"), timestamp, Lists.newArrayList(window)), WindowedValue.of(KV.of(2, "in 2a"), timestamp, Lists.newArrayList(window)), @@ -64,7 +64,7 @@ public class PartitioningShuffleSourceTest { WindowedValue.of(KV.of(4, "in 4d"), timestamp, Lists.newArrayList(window)), WindowedValue.of(KV.of(5, "in 5"), timestamp, Lists.newArrayList(window))); - void runTestReadShuffleSource(List>> expected) + private void runTestReadShuffleSource(List>> expected) throws Exception { Coder>> elemCoder = WindowedValue.getFullCoder( KvCoder.of(BigEndianIntegerCoder.of(), StringUtf8Coder.of()), diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java index 3e390b8966afa..b2aa533ce53f2 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java @@ -45,13 +45,13 @@ import java.util.List; /** - * Tests for ShuffleSink. + * Tests for {@link ShuffleSink}. */ @RunWith(JUnit4.class) public class ShuffleSinkTest { - static final List> NO_KVS = Collections.emptyList(); + private static final List> NO_KVS = Collections.emptyList(); - static final List> KVS = Arrays.asList( + private static final List> KVS = Arrays.asList( KV.of(1, "in 1a"), KV.of(1, "in 1b"), KV.of(2, "in 2a"), @@ -63,10 +63,10 @@ public class ShuffleSinkTest { KV.of(4, "in 4d"), KV.of(5, "in 5")); - static final List>> NO_SORTING_KVS = + private static final List>> NO_SORTING_KVS = Collections.emptyList(); - static final List>> SORTING_KVS = + private static final List>> SORTING_KVS = Arrays.asList( KV.of(1, KV.of("in 1a", 3)), KV.of(1, KV.of("in 1b", 9)), @@ -79,10 +79,10 @@ public class ShuffleSinkTest { KV.of(4, KV.of("in 4d", 1)), KV.of(5, KV.of("in 5", 666))); - static final Instant timestamp = new Instant(123000); - static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); + private static final Instant timestamp = new Instant(123000); + private static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); - void runTestWriteUngroupingShuffleSink(List expected) + private void runTestWriteUngroupingShuffleSink(List expected) throws Exception { Coder> windowedValueCoder = WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), new GlobalWindow().windowCoder()); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java index 3a360d8d24add..bf9a15c05b343 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java @@ -43,20 +43,20 @@ */ @RunWith(JUnit4.class) public class UngroupedShuffleSourceTest { - static final Instant timestamp = new Instant(123000); - static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); + private static final Instant timestamp = new Instant(123000); + private static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); - byte[] asShuffleKey(long seqNum) throws Exception { + private byte[] asShuffleKey(long seqNum) throws Exception { return CoderUtils.encodeToByteArray(BigEndianLongCoder.of(), seqNum); } - byte[] asShuffleValue(Integer value) throws Exception { + private byte[] asShuffleValue(Integer value) throws Exception { return CoderUtils.encodeToByteArray( WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), IntervalWindow.getCoder()), WindowedValue.of(value, timestamp, Lists.newArrayList(window))); } - void runTestReadShuffleSource(List expected) throws Exception { + private void runTestReadShuffleSource(List expected) throws Exception { UngroupedShuffleSource> shuffleSource = new UngroupedShuffleSource<>( PipelineOptionsFactory.create(), diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java index 406fb0730d023..808ef4a23b6a8 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java @@ -25,7 +25,6 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.runners.DirectPipeline; -import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.ApproximateQuantiles.ApproximateQuantilesCombineFn; @@ -45,7 +44,7 @@ import java.util.List; /** - * Tests for ApproximateQuantiles + * Tests for {@link ApproximateQuantiles}. */ @RunWith(JUnit4.class) @SuppressWarnings("serial") @@ -74,7 +73,7 @@ public void testQuantilesGlobally() { PCollection> quantiles = input.apply(ApproximateQuantiles.globally(5)); - DirectPipelineRunner.EvaluationResults results = p.run(); + p.run(); DataflowAssert.that(quantiles) .containsInAnyOrder(Arrays.asList(0, 25, 50, 75, 100)); @@ -89,7 +88,7 @@ public void testQuantilesGobally_comparable() { input.apply( ApproximateQuantiles.globally(5, new DescendingIntComparator())); - DirectPipelineRunner.EvaluationResults results = p.run(); + p.run(); DataflowAssert.that(quantiles) .containsInAnyOrder(Arrays.asList(100, 75, 50, 25, 0)); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java index c0d8e21b15e67..baf8a24756cc7 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java @@ -406,7 +406,7 @@ public Coder getAccumulatorCoder( } /** - * A Coder for CountSum + * A {@link Coder} for {@link CountSum}. */ public class CountSumCoder extends CustomCoder { @Override @@ -450,7 +450,7 @@ public static class TestCounter extends Combine.AccumulatingCombineFn< Integer, TestCounter.Counter, Iterable> { - /** An accumulator that observes its merges and outputs */ + /** An accumulator that observes its merges and outputs. */ public class Counter implements Combine.AccumulatingCombineFn.Accumulator>, Serializable { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java index c129b8e6b912e..a4de8c80b7fb1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java @@ -39,7 +39,7 @@ import java.util.List; /** - * Tests for Partition + * Tests for {@link Partition}. */ @RunWith(JUnit4.class) @SuppressWarnings("serial") @@ -93,8 +93,7 @@ public void testOutOfBoundsPartitions() { PCollection input = createInts(p, Arrays.asList(-1)); - PCollectionList outputs = - input.apply(Partition.of(5, new IdentityFn())); + input.apply(Partition.of(5, new IdentityFn())); try { p.run(); @@ -111,8 +110,7 @@ public void testZeroNumPartitions() { PCollection input = createInts(p, Arrays.asList(591)); try { - PCollectionList outputs = - input.apply(Partition.of(0, new IdentityFn())); + input.apply(Partition.of(0, new IdentityFn())); fail("should have failed"); } catch (IllegalArgumentException exn) { assertThat(exn.toString(), containsString("numPartitions must be > 0")); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java index 561e17b871e7e..fced952d90670 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java @@ -42,7 +42,7 @@ import java.util.Comparator; import java.util.List; -/** Tests for Top */ +/** Tests for Top. */ @RunWith(JUnit4.class) @SuppressWarnings("serial") public class TopTest { @@ -175,15 +175,14 @@ public void testTopZero() { @Test public void testPerKeySerializabilityRequirement() { DirectPipeline p = DirectPipeline.createForTest(); - PCollection input = - p.apply(Create.of(Arrays.asList(COLLECTION))) + p.apply(Create.of(Arrays.asList(COLLECTION))) .setCoder(StringUtf8Coder.of()); - PCollection>> top1 = createInputTable(p) + createInputTable(p) .apply(Top.perKey(1, new IntegerComparator())); - PCollection>> top2 = createInputTable(p) + createInputTable(p) .apply(Top.perKey(1, new IntegerComparator2())); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/KVTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/KVTest.java index dae544fb033af..6dc77eba1475e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/KVTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/KVTest.java @@ -29,7 +29,7 @@ */ @RunWith(JUnit4.class) public class KVTest { - static final Integer testValues[] = + private static final Integer TEST_VALUES[] = {null, Integer.MIN_VALUE, -1, 0, 1, Integer.MAX_VALUE}; // Wrapper around Integer.compareTo() to support null values. @@ -44,10 +44,10 @@ private int compareInt(Integer a, Integer b) { @Test public void testOrderByKey() { Comparator> orderByKey = new KV.OrderByKey<>(); - for (Integer key1 : testValues) { - for (Integer val1 : testValues) { - for (Integer key2 : testValues) { - for (Integer val2 : testValues) { + for (Integer key1 : TEST_VALUES) { + for (Integer val1 : TEST_VALUES) { + for (Integer key2 : TEST_VALUES) { + for (Integer val2 : TEST_VALUES) { assertEquals(compareInt(key1, key2), orderByKey.compare(KV.of(key1, val1), KV.of(key2, val2))); } @@ -59,10 +59,10 @@ public void testOrderByKey() { @Test public void testOrderByValue() { Comparator> orderByValue = new KV.OrderByValue<>(); - for (Integer key1 : testValues) { - for (Integer val1 : testValues) { - for (Integer key2 : testValues) { - for (Integer val2 : testValues) { + for (Integer key1 : TEST_VALUES) { + for (Integer val1 : TEST_VALUES) { + for (Integer key2 : TEST_VALUES) { + for (Integer val2 : TEST_VALUES) { assertEquals(compareInt(val1, val2), orderByValue.compare(KV.of(key1, val1), KV.of(key2, val2))); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PDoneTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PDoneTest.java index f41fef6d840bf..5d75100f58f5e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PDoneTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/PDoneTest.java @@ -24,8 +24,10 @@ import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.PTransform; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -72,9 +74,10 @@ public PDone apply(PBegin begin) { // TODO: This test doesn't work, because we can't handle composite // transforms that contain no nested transforms. - // @Test - // @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) - public void DISABLED_testEmptyTransform() { + @Ignore + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testEmptyTransform() { Pipeline p = TestPipeline.create(); p.begin().apply(new EmptyTransform()); From dfa53aab308e14a1fdf7f572b70a90821d922dcc Mon Sep 17 00:00:00 2001 From: relax Date: Mon, 15 Dec 2014 14:47:08 -0800 Subject: [PATCH 0020/1541] Revert change to set the sdk major_version to 0. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82178016 --- .../cloud/dataflow/sdk/runners/DataflowPipelineRunner.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index 1a0d873307375..a2574774d3315 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -196,7 +196,9 @@ public DataflowPipelineJob run(Pipeline pipeline) { // Requirements about the service. Map environmentVersion = new HashMap<>(); - environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, ENVIRONMENT_MAJOR_VERSION); + // TODO: Specify the environment major version. + // environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, + // ENVIRONMENT_MAJOR_VERSION); newJob.getEnvironment().setVersion(environmentVersion); // Default jobType is DATA_PARALLEL which is for java batch. String jobType = "DATA_PARALLEL"; From 622c8587675b6c197ab2f183d1172d56c1dffa45 Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 15 Dec 2014 15:51:49 -0800 Subject: [PATCH 0021/1541] Canonicalize temp folder path before globbing in IOFactoryTest. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82183712 --- .../java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java index fbf2f70b22355..89ca01ef724d2 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java @@ -52,7 +52,7 @@ public void testLocalFileIO() throws Exception { tmpFolder.newFile("barf").createNewFile(); FileIOChannelFactory factory = new FileIOChannelFactory(); - Collection paths = factory.match(tmpFolder.getRoot() + "/f*"); + Collection paths = factory.match(tmpFolder.getRoot().getCanonicalPath() + "/f*"); Assert.assertEquals(2, paths.size()); Assert.assertTrue(paths.contains(foo1.getCanonicalPath())); From 904ffe9367f1fc3fb1d6083949f5cf5a6d1693ae Mon Sep 17 00:00:00 2001 From: davor Date: Mon, 15 Dec 2014 16:32:20 -0800 Subject: [PATCH 0022/1541] Dataflow launch: provide build and run instructions in README.md on GitHub. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82187443 --- README.md | 75 +++++++++++++++++++++++++++++++++++++++++++++++++------ 1 file changed, 68 insertions(+), 7 deletions(-) diff --git a/README.md b/README.md index 12eba01f5e831..086c0aae78a14 100644 --- a/README.md +++ b/README.md @@ -11,9 +11,11 @@ The key concepts in this programming model are: * [PCollection](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java): represents a collection of data, which could be bounded or unbounded in size. * [PTransform](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java): -represents a computation that transform input PCollections into output PCollections. +represents a computation that transform input PCollections into output +PCollections. * [Pipeline](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java): -manages a directed acyclic graph of PTransforms and PCollections, which is ready for excution. +manages a directed acyclic graph of PTransforms and PCollections, which is ready +for execution. * [PipelineRunner](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java): specifies where and how the pipeline should execute. @@ -23,18 +25,77 @@ Currently there are three runners: runs the pipeline on your local machine. 2. The [DataflowPipelineRunner](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java) -submits the pipeline to the Dataflow Service**\***, where it runs using managed +submits the pipeline to the Dataflow Service, where it runs using managed resources in the [Google Cloud Platform](http://cloud.google.com). 3. The [BlockingDataflowPipelineRunner](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunner.java) -submits the pipeline to the Dataflow Service**\*** via the DataflowPipelineRunner and then prints messages -about the job status until execution is complete. +submits the pipeline to the Dataflow Service via the DataflowPipelineRunner and +then prints messages about the job status until execution is complete. -**\***_The Dataflow Service is currently in the Alpha phase of development and access -is limited to whitelisted users._ +_The Dataflow Service is currently in the Alpha phase of development and +access is limited to whitelisted users._ ## Getting Started +This repository consists of two modules: + +* [Java SDK](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk) +module provides a set of basic Java APIs to program against. +* [Examples](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples) +module provides a few samples to get started. We recommend starting with the +WordCount example. + +The following command will build both modules and install them in your local +Maven repository: + + mvn clean install + +You can speed up the build and install process by using the following options: + + 1. To skip execution of the unit tests, run: + + mvn install -DskipTests + + 2. While iterating on a specific module, use the following command to compile + and reinstall it. For example, to reinstall the 'examples' module, run: + + mvn install -pl examples + + Be careful, however, as this command will use the most recently installed SDK + from the local repository (or Maven Central) even if you have changed it + locally. + + 3. To run Maven using multiple threads, run: + + mvn -T 4 install + +After building and installing, the following command will execute the WordCount +example using the DirectPipelineRunner on your local machine: + + mvn exec:java -pl examples \ + -Dexec.mainClass=com.google.cloud.dataflow.examples.WordCount \ + -Dexec.args="--input= --output=" + +If you have been whitelisted for Alpha access to the Dataflow Service and +followed the [developer setup](https://cloud.google.com/dataflow/java-sdk/getting-started#DeveloperSetup) +steps, you can use the BlockingDataflowPipelineRunner to run the same program in +the Google Cloud Platform (GCP): + + mvn exec:java -pl examples \ + -Dexec.mainClass=com.google.cloud.dataflow.examples.WordCount \ + -Dexec.args="--project= --stagingLocation= --runner=BlockingDataflowPipelineRunner" + +Google Cloud Storage (GCS) location should be entered in the form of +gs://bucket/path/to/staging/directory. Google Cloud Platform (GCP) project +refers to its name (not number), which has been whitelisted for Cloud Dataflow. +Refer [here](https://cloud.google.com/) for instructions to get started with +Google Cloud Platform. + +Other examples can be run similarly by replacing the WordCount class name with +BigQueryTornadoes, DatastoreWordCount, TfIdf, TopWikipediaSessions, etc. and +adjusting runtime options under Dexec.args parameter, as specified in the +example itself. + ## More Information * [Google Cloud Dataflow](http://cloud.google.com/dataflow) From 4d91458aaf85239f25621131d0e25374dbf2bb3f Mon Sep 17 00:00:00 2001 From: davor Date: Mon, 15 Dec 2014 16:44:33 -0800 Subject: [PATCH 0023/1541] Fix FileIOChannelFactory to allow specifying filename alone (that is, without the directory information) for inputs and outputs. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82188346 --- .../google/cloud/dataflow/sdk/util/FileIOChannelFactory.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java index 71f66ed2f6db5..1bb7566a526b5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java @@ -47,13 +47,13 @@ public class FileIOChannelFactory implements IOChannelFactory { public Collection match(String spec) throws IOException { File file = new File(spec); - File parent = file.getParentFile(); + File parent = file.getAbsoluteFile().getParentFile(); if (!parent.exists()) { throw new IOException("Unable to find parent directory of " + spec); } final PathMatcher matcher = - FileSystems.getDefault().getPathMatcher("glob:" + spec); + FileSystems.getDefault().getPathMatcher("glob:" + file.getAbsolutePath()); File[] files = parent.listFiles(new FileFilter() { @Override public boolean accept(File pathname) { From 43a04eb35a25bdbd5a269005cb1533afa3373d32 Mon Sep 17 00:00:00 2001 From: davor Date: Tue, 16 Dec 2014 09:35:44 -0800 Subject: [PATCH 0024/1541] Dataflow launch: enable Embed-Transitive option when bundling. The default behavior of the bundle plugin is to embed only direct dependencies, i.e., it does not embed transitive dependencies. Setting this option enables transitive dependency embedding. So far, this option has been enabled on SDK's module, but not on other modules, such as examples. Now, this option is enabled across all modules. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82240765 --- examples/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/examples/pom.xml b/examples/pom.xml index ad847af861d8b..9ff72266c6d25 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -111,6 +111,7 @@ ${project.artifactId}-bundled-${project.version} + true *;scope=compile|runtime;inline=true From 170351e2313aa0713cedfc6598483723b8c6a57a Mon Sep 17 00:00:00 2001 From: fjp Date: Tue, 16 Dec 2014 09:57:25 -0800 Subject: [PATCH 0025/1541] Update documentation links. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82242478 --- .../java/com/google/cloud/dataflow/examples/WordCount.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/ParDo.java | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java b/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java index 6b1623005785d..bdd5fb6208fdf 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java @@ -37,7 +37,7 @@ /** * An example that counts words in Shakespeare. For a detailed walkthrough of this * example see: - * https://developers.google.com/cloud-dataflow/java-sdk/wordcount-example + * https://cloud.google.com/dataflow/java-sdk/wordcount-example * *

Concepts: Reading/writing text files; counting a PCollection; user-defined PTransforms * diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java index fad1235de0b90..8c9c43fb4e51f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java @@ -434,9 +434,8 @@ * Dataflow service's optimizer to "flatten out" all the compositions * into highly optimized stages. * - * @see Using ParDo + * @see Parallel + * Processing with ParDo */ public class ParDo { From 9e90654cd39af8d2b91290055b5b5066d07c6d94 Mon Sep 17 00:00:00 2001 From: davor Date: Tue, 16 Dec 2014 09:59:31 -0800 Subject: [PATCH 0026/1541] Dataflow launch: update pom.xml for certain modules. In the context of bringing in dependencies from com.google.apis group, move exclusion of Guava transitive dependency to the first instance of the dependency in the POM file. Notes: * Artifacts from com.google.apis, version 1.19 in particular, brings in an old version of Guava, which is not compatible with the SDK content. * We need to exclude this transitive dependency to ensure build works. * This seems to happen when the exclusion in set on the first instance of a dependency from com.google.apis. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82242666 --- examples/pom.xml | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 9ff72266c6d25..eef3d4003c35c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -151,12 +151,6 @@ com.google.apis google-api-services-storage v1-rev11-1.19.0 - - - - com.google.apis - google-api-services-bigquery - v2-rev167-1.19.0 @@ -167,6 +161,12 @@ + + com.google.apis + google-api-services-bigquery + v2-rev167-1.19.0 + + com.google.http-client google-http-client-jackson2 From 6ad521097f50d6390ba99489d0dffb87c9991715 Mon Sep 17 00:00:00 2001 From: davor Date: Tue, 16 Dec 2014 10:37:48 -0800 Subject: [PATCH 0027/1541] Dataflow launch: update pom.xml for certain modules. In the context of bringing in dependencies from com.google.apis group, add exclusion of Guava transitive dependency to all com.google.apis dependencies. Notes: * Artifacts from com.google.apis, version 1.19 in particular, brings in an old version of Guava, which is not compatible with the SDK content. * We need to exclude this transitive dependency to ensure build works. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82246218 --- examples/pom.xml | 16 ++++++++++++++ sdk/pom.xml | 56 ++++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 72 insertions(+) diff --git a/examples/pom.xml b/examples/pom.xml index eef3d4003c35c..6604a90e531ef 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -165,12 +165,28 @@ com.google.apis google-api-services-bigquery v2-rev167-1.19.0 + + + + com.google.guava + guava-jdk5 + + com.google.http-client google-http-client-jackson2 1.19.0 + + + + com.google.guava + guava-jdk5 + + diff --git a/sdk/pom.xml b/sdk/pom.xml index 93a8f277a8370..921fa8675ac80 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -203,42 +203,98 @@ com.google.apis google-api-services-bigquery v2-rev167-1.19.0 + + + + com.google.guava + guava-jdk5 + + com.google.apis google-api-services-compute v1-rev34-1.19.0 + + + + com.google.guava + guava-jdk5 + + com.google.apis google-api-services-pubsub v1beta1-rev9-1.19.0 + + + + com.google.guava + guava-jdk5 + + com.google.apis google-api-services-storage v1-rev11-1.19.0 + + + + com.google.guava + guava-jdk5 + + com.google.http-client google-http-client-jackson2 1.19.0 + + + + com.google.guava + guava-jdk5 + + com.google.oauth-client google-oauth-client-java6 1.19.0 + + + + com.google.guava + guava-jdk5 + + com.google.apis google-api-services-datastore-protobuf v1beta2-rev1-2.1.0 + + + + com.google.guava + guava-jdk5 + + From 83b44d548641a098c017273b2bb79534b2b1525a Mon Sep 17 00:00:00 2001 From: Frances Perry Date: Wed, 17 Dec 2014 16:52:45 -0800 Subject: [PATCH 0028/1541] Update README.md --- README.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/README.md b/README.md index 086c0aae78a14..0aec8a0a251b4 100644 --- a/README.md +++ b/README.md @@ -4,6 +4,18 @@ provides a simple, powerful programming model for building both batch and streaming parallel data processing pipelines. +## Status + +The Cloud Dataflow SDK is used to access the Google Cloud Dataflow service, +which is currently in Alpha and restricted to whitelisted users. + +The SDK is publicly available and can be used for local execution by anyone. +Note, however, that the SDK is also an Alpha release and may change +significantly over time. The SDK is built to be extensible and support +additional execution environments ("runners") beyond local execution and the +Google Cloud Dataflow service. As the product matures, we look forward to +working with you to improve Cloud Dataflow. + ## Overview The key concepts in this programming model are: From 9161131f4397f077ce2967e1280067214aaea954 Mon Sep 17 00:00:00 2001 From: Sean O'Keefe Date: Fri, 19 Dec 2014 08:56:32 -0800 Subject: [PATCH 0029/1541] Updating links to canonicals --- README.md | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index 0aec8a0a251b4..f88d9622d2c92 100644 --- a/README.md +++ b/README.md @@ -1,6 +1,6 @@ # Cloud Dataflow Java SDK (Alpha) -[Google Cloud Dataflow](http://cloud.google.com/dataflow) +[Google Cloud Dataflow](https://cloud.google.com/dataflow/) provides a simple, powerful programming model for building both batch and streaming parallel data processing pipelines. @@ -97,11 +97,10 @@ the Google Cloud Platform (GCP): -Dexec.mainClass=com.google.cloud.dataflow.examples.WordCount \ -Dexec.args="--project= --stagingLocation= --runner=BlockingDataflowPipelineRunner" -Google Cloud Storage (GCS) location should be entered in the form of +[Google Cloud Storage](https://cloud.google.com/storage/) (GCS) location should be entered in the form of gs://bucket/path/to/staging/directory. Google Cloud Platform (GCP) project refers to its name (not number), which has been whitelisted for Cloud Dataflow. -Refer [here](https://cloud.google.com/) for instructions to get started with -Google Cloud Platform. +Refer to [Google Cloud Platform](https://cloud.google.com/) for instructions on getting started. Other examples can be run similarly by replacing the WordCount class name with BigQueryTornadoes, DatastoreWordCount, TfIdf, TopWikipediaSessions, etc. and @@ -110,6 +109,6 @@ example itself. ## More Information -* [Google Cloud Dataflow](http://cloud.google.com/dataflow) +* [Google Cloud Dataflow](https://cloud.google.com/dataflow/) * [Dataflow Concepts and Programming Model](https://cloud.google.com/dataflow/java-sdk/building-a-pipeline) * [Javadoc](https://cloud.google.com/dataflow/java-sdk/JavaDoc/index) From 60035b4e1f2a31485298f936d9715fcff95bc4ea Mon Sep 17 00:00:00 2001 From: fjp Date: Tue, 16 Dec 2014 12:18:22 -0800 Subject: [PATCH 0030/1541] Add a missing package-info.java for windowing. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82255231 --- .../transforms/windowing/package-info.java | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/package-info.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/package-info.java new file mode 100644 index 0000000000000..cab217d3e9698 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/package-info.java @@ -0,0 +1,39 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Defines the {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} transform + * for dividing the elements in a PCollection into windows. + * + *

{@code Window} logically divides up or groups the elements of a + * {@link com.google.cloud.dataflow.sdk.values.PCollection} into finite windows according to a + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn}. + * The output of {@code Window} contains the same elements as input, but they + * have been logically assigned to windows. The next + * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}s, including one + * within composite transforms, will group by the combination of keys and + * windows. + * + *

Windowing a {@code PCollection} allows chunks of it to be processed + * individually, before the entire {@code PCollection} is available. This is + * especially important for {@code PCollection}s with unbounded size, since the full + * {@code PCollection} is never available at once. + * + *

For {@code PCollection}s with a bounded size, by default, all data is implicitly in a + * single window, and this replicates conventional batch mode. However, windowing can still be a + * convenient way to express time-sliced algorithms over bounded {@code PCollection}s. + */ +package com.google.cloud.dataflow.sdk.transforms.windowing; From aab6b6d2c0b27b2a925d0dd6cd420bf56f41f795 Mon Sep 17 00:00:00 2001 From: peihe Date: Thu, 18 Dec 2014 11:58:18 -0800 Subject: [PATCH 0031/1541] Maven: turn off trimStackTrace. Before the fix, only top level Exceptions are shown, e.t.c. InvocationTargetException and the root Exception is trimmed. Currently, it shows the full stack trace. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82451245 --- pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/pom.xml b/pom.xml index fd5b04376e43e..39eec27b528b2 100644 --- a/pom.xml +++ b/pom.xml @@ -169,6 +169,7 @@ ${dataflowProjectName} false + false From 96b0834a952803e1b09b7429464041d7b0ded4c7 Mon Sep 17 00:00:00 2001 From: millsd Date: Fri, 19 Dec 2014 11:24:19 -0800 Subject: [PATCH 0032/1541] Fix typos in README.md. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82529793 --- README.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 0aec8a0a251b4..18f78fc9f99d6 100644 --- a/README.md +++ b/README.md @@ -23,7 +23,7 @@ The key concepts in this programming model are: * [PCollection](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java): represents a collection of data, which could be bounded or unbounded in size. * [PTransform](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java): -represents a computation that transform input PCollections into output +represents a computation that transforms input PCollections into output PCollections. * [Pipeline](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java): manages a directed acyclic graph of PTransforms and PCollections, which is ready @@ -31,7 +31,7 @@ for execution. * [PipelineRunner](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java): specifies where and how the pipeline should execute. -Currently there are three runners: +Currently there are three PipelineRunners: 1. The [DirectPipelineRunner](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java) runs the pipeline on your local machine. @@ -105,7 +105,7 @@ Google Cloud Platform. Other examples can be run similarly by replacing the WordCount class name with BigQueryTornadoes, DatastoreWordCount, TfIdf, TopWikipediaSessions, etc. and -adjusting runtime options under Dexec.args parameter, as specified in the +adjusting runtime options under the Dexec.args parameter, as specified in the example itself. ## More Information From df872084cd83178569ed60f58a4a9c18a8b24195 Mon Sep 17 00:00:00 2001 From: davor Date: Mon, 22 Dec 2014 10:51:49 -0800 Subject: [PATCH 0033/1541] Dataflow open-sourcing: Add file explaining contribution rules. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82665181 --- CONTRIBUTING.md | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) create mode 100644 CONTRIBUTING.md diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md new file mode 100644 index 0000000000000..26147ec273b71 --- /dev/null +++ b/CONTRIBUTING.md @@ -0,0 +1,26 @@ +Want to contribute? Great! First, read this page (including the small print at +the end). + +### Before you contribute +Before we can use your code, you must sign the +[Google Individual Contributor License Agreement](https://developers.google.com/open-source/cla/individual?csw=1) +(CLA), which you can do online. The CLA is necessary mainly because you own the +copyright to your changes, even after your contribution becomes part of our +codebase, so we need your permission to use and distribute your code. We also +need to be sure of various other things. For instance that you'll tell us if you +know that your code infringes on other people's patents. You don't have to sign +the CLA until after you've submitted your code for review and a member has +approved it, but you must do it before we can put your code into our codebase. + +Before you start working on a larger contribution, we recommend to get in touch +with us first through the issue tracker with your idea so that we can help out +and possibly guide you. Coordinating up front makes it much easier to avoid +frustration later on. + +### Code reviews +All submissions, including submissions by project members, require review. We +use Github pull requests for this purpose. + +### The small print +Contributions made by corporations are covered by a different agreement than +the one above, the Software Grant and Corporate Contributor License Agreement. From 3db16e53bb153c04fba55423f7ebe90ffa5d737f Mon Sep 17 00:00:00 2001 From: earhart Date: Fri, 19 Dec 2014 13:39:17 -0800 Subject: [PATCH 0034/1541] In GroupAlsoByWindowsDoFnTest, match values in arbitrary order. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82539405 --- .../cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java index d482d2c4d345a..d01fe1008fa39 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java @@ -100,7 +100,7 @@ public class GroupAlsoByWindowsDoFnTest { WindowedValue>> item0 = result.get(0); assertEquals("k", item0.getValue().getKey()); - assertThat(item0.getValue().getValue(), Matchers.contains("v1", "v2")); + assertThat(item0.getValue().getValue(), Matchers.containsInAnyOrder("v1", "v2")); assertEquals(new Instant(9), item0.getTimestamp()); assertThat(item0.getWindows(), Matchers.contains(window(0, 10))); @@ -146,7 +146,7 @@ public class GroupAlsoByWindowsDoFnTest { WindowedValue>> item1 = result.get(1); assertEquals("k", item1.getValue().getKey()); - assertThat(item1.getValue().getValue(), Matchers.contains("v1", "v2")); + assertThat(item1.getValue().getValue(), Matchers.containsInAnyOrder("v1", "v2")); assertEquals(new Instant(19), item1.getTimestamp()); assertThat(item1.getWindows(), Matchers.contains(window(0, 20))); @@ -189,7 +189,7 @@ public class GroupAlsoByWindowsDoFnTest { WindowedValue>> item0 = result.get(0); assertEquals("k", item0.getValue().getKey()); - assertThat(item0.getValue().getValue(), Matchers.contains("v1", "v2")); + assertThat(item0.getValue().getValue(), Matchers.containsInAnyOrder("v1", "v2")); assertEquals(new Instant(14), item0.getTimestamp()); assertThat(item0.getWindows(), Matchers.contains(window(0, 15))); From 45b78eaa6abfc7fcd0d173e111b60b66c811dbe4 Mon Sep 17 00:00:00 2001 From: Kelly Westbrooks Date: Fri, 9 Jan 2015 10:57:36 -0800 Subject: [PATCH 0035/1541] Generalize type signature of KV.of --- .../main/java/com/google/cloud/dataflow/sdk/values/KV.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java index febb1302bd8c1..81b6806a1125f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java @@ -35,8 +35,8 @@ public class KV implements Serializable { private static final long serialVersionUID = 0; /** Returns a KV with the given key and value. */ - public static KV of(K key, V value) { - return new KV<>(key, value); + public static KV of(SK key, SV value) { + return new KV(key, value); } /** Returns the key of this KV. */ From b2870737bf3ffaed94eaa39aa5eebed8c1bf5a3c Mon Sep 17 00:00:00 2001 From: peihe Date: Wed, 17 Dec 2014 11:14:46 -0800 Subject: [PATCH 0036/1541] Cleanup: clarify fake exceptions and error messages in Tests, it helps to find the real tests failure when people go through logs. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82346418 --- .../sdk/runners/worker/DataflowWorkerTest.java | 3 ++- .../cloud/dataflow/sdk/testing/ExpectedLogsTest.java | 11 +++-------- .../cloud/dataflow/sdk/util/PackageUtilTest.java | 6 +++--- 3 files changed, 8 insertions(+), 12 deletions(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java index 2d51fb2838954..037946c29a917 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java @@ -63,7 +63,8 @@ public void testWhenNoWorkThatWeReturnFalse() throws Exception { @Test public void testWhenProcessingWorkUnitFailsWeReportStatus() throws Exception { DataflowWorker worker = new DataflowWorker(mockWorkUnitClient, options); - when(mockWorkUnitClient.getWorkItem()).thenReturn(new WorkItem().setId(1L)).thenReturn(null); + when(mockWorkUnitClient.getWorkItem()).thenReturn( + new WorkItem().setId(1L).setJobId("Expected to fail the job")).thenReturn(null); assertFalse(worker.getAndPerformWork()); verify(mockWorkUnitClient).reportWorkItemStatus(argThat(cloudWorkHasErrors())); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java index 4d9cd0e76639a..bcd96cefffe45 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java @@ -54,7 +54,6 @@ public void testWhenExpectationIsMatchedFully() throws Throwable { expectedLogs.after(); } - @Test public void testWhenExpectationIsMatchedPartially() throws Throwable { String expected = generateRandomString(); @@ -69,7 +68,7 @@ public void testWhenExpectationIsMatchedWithExceptionBeingLogged() throws Throwa String expected = generateRandomString(); expectedLogs.before(); expectedLogs.expectError(expected); - LOG.error(expected, new IOException()); + LOG.error(expected, new IOException("Fake Exception")); expectedLogs.after(); } @@ -90,13 +89,9 @@ public void testLogCaptureOccursAtLowestLogLevel() throws Throwable { expectedLogs.after(); } - // Generates random strings of 10 characters. + // Generates a random fake error message. private static String generateRandomString() { Random random = new Random(); - StringBuilder builder = new StringBuilder(); - for (int i = 0; i < 10; i++) { - builder.append('a' + (char) random.nextInt(26)); - } - return builder.toString(); + return "Fake error message: " + random.nextInt(); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java index 7d923c2fcdb7f..e49782f6b221b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java @@ -244,7 +244,7 @@ public void testPackageUploadFailsWhenIOExceptionThrown() throws Exception { GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(-1L); when(mockGcsUtil.create(any(GcsPath.class), anyString())) - .thenThrow(new IOException("Upload error")); + .thenThrow(new IOException("Fake Exception: Upload error")); try { PackageUtil.stageClasspathElementsToGcs(mockGcsUtil, @@ -264,8 +264,8 @@ public void testPackageUploadEventuallySucceeds() throws Exception { GcsPath gcsStaging = GcsPath.fromComponents("somebucket", "base/path"); when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(-1L); when(mockGcsUtil.create(any(GcsPath.class), anyString())) - .thenThrow(new IOException("410 Gone")) // First attempt fails - .thenReturn(pipe.sink()); // second attempt succeeds + .thenThrow(new IOException("Fake Exception: 410 Gone")) // First attempt fails + .thenReturn(pipe.sink()); // second attempt succeeds try { PackageUtil.stageClasspathElementsToGcs(mockGcsUtil, From 5cf3ed13d19335831034628a4b852d2b21e85630 Mon Sep 17 00:00:00 2001 From: lcwik Date: Wed, 17 Dec 2014 11:43:31 -0800 Subject: [PATCH 0037/1541] Add support for enum defaults on PipelineOptions using the enum name. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82349417 --- .../DataflowPipelineWorkerPoolOptions.java | 22 ++----------------- .../cloud/dataflow/sdk/options/Default.java | 10 +++++++++ .../sdk/options/ProxyInvocationHandler.java | 3 +++ .../options/ProxyInvocationHandlerTest.java | 19 ++++++++++++++++ 4 files changed, 34 insertions(+), 20 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java index 0193ddaac1077..da0a6e588605f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java @@ -61,19 +61,10 @@ public String getAlgorithm() { } @Description("(experimental) The autoscaling algorithm to use for the workerpool.") - @Default.InstanceFactory(AutoscalingAlgorithmTypeFactory.class) + @Default.Enum("NONE") AutoscalingAlgorithmType getAutoscalingAlgorithm(); void setAutoscalingAlgorithm(AutoscalingAlgorithmType value); - /** Returns the default NONE AutoscalingAlgorithmType. */ - public static class AutoscalingAlgorithmTypeFactory implements - DefaultValueFactory { - @Override - public AutoscalingAlgorithmType create(PipelineOptions options) { - return AutoscalingAlgorithmType.NONE; - } - } - /** * Max number of workers to use when using workerpool autoscaling. * This option is experimental and subject to change. @@ -120,19 +111,10 @@ public String getApiServiceName() { } @Description("Type of API for handling cluster management,i.e. resizing, healthchecking, etc.") - @Default.InstanceFactory(ClusterManagerApiTypeFactory.class) + @Default.Enum("COMPUTE_ENGINE") ClusterManagerApiType getClusterManagerApi(); void setClusterManagerApi(ClusterManagerApiType value); - /** Returns the default COMPUTE_ENGINE ClusterManagerApiType. */ - public static class ClusterManagerApiTypeFactory implements - DefaultValueFactory { - @Override - public ClusterManagerApiType create(PipelineOptions options) { - return ClusterManagerApiType.COMPUTE_ENGINE; - } - } - /** * Machine type to create worker VMs as. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Default.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Default.java index 321fe744ca49b..c295bde5eaf71 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Default.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Default.java @@ -116,6 +116,16 @@ double value(); } + /** + * This represents that the default of the option is the specified enum. + * The value should equal the enum's {@link java.lang.Enum#name() name}. + */ + @Target(ElementType.METHOD) + @Retention(RetentionPolicy.RUNTIME) + public @interface Enum { + java.lang.String value(); + } + /** * Value must be of type {@link DefaultValueFactory} and have a default constructor. * Value is instantiated and then used as a type factory to generate the default. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java index cbe9c422b325b..39ab988182370 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java @@ -252,6 +252,9 @@ private Object getDefault(PipelineOptions proxy, Method method) { return ((Default.String) annotation).value(); } else if (annotation instanceof Default.String) { return ((Default.String) annotation).value(); + } else if (annotation instanceof Default.Enum) { + return Enum.valueOf((Class) method.getReturnType(), + ((Default.Enum) annotation).value()); } else if (annotation instanceof Default.InstanceFactory) { return InstanceBuilder.ofType(((Default.InstanceFactory) annotation).value()) .build() diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java index 0b2dc0a9adfba..01085322f26e4 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java @@ -115,6 +115,21 @@ public String create(PipelineOptions options) { } } + /** A test enum for testing {@link Default.Enum @Default.Enum}. */ + public enum EnumType { + MyEnum("MyTestEnum"); + + private final String value; + private EnumType(String value) { + this.value = value; + } + + @Override + public String toString() { + return value; + } + } + /** A test interface containing all the {@link Default} annotations. */ public static interface DefaultAnnotations extends PipelineOptions { @Default.Boolean(true) @@ -147,6 +162,9 @@ public static interface DefaultAnnotations extends PipelineOptions { @Default.Class(DefaultAnnotations.class) Class getClassOption(); void setClassOption(Class value); + @Default.Enum("MyEnum") + EnumType getEnum(); + void setEnum(EnumType value); @Default.InstanceFactory(TestOptionFactory.class) String getComplex(); void setComplex(String value); @@ -166,6 +184,7 @@ public void testAnnotationDefaults() throws Exception { assertEquals(9d, proxy.getDouble(), 0d); assertEquals("testString", proxy.getString()); assertEquals(DefaultAnnotations.class, proxy.getClassOption()); + assertEquals(EnumType.MyEnum, proxy.getEnum()); assertEquals("testOptionFactory", proxy.getComplex()); } From c4c96bb0e84ed7aa9c0c39c25a2e6b45b81a6dbb Mon Sep 17 00:00:00 2001 From: peihe Date: Wed, 17 Dec 2014 12:19:43 -0800 Subject: [PATCH 0038/1541] Testing: add CombineTest cases for windowing. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82352893 --- .../dataflow/sdk/transforms/CombineTest.java | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java index baf8a24756cc7..4307619d3f93d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java @@ -183,6 +183,33 @@ private void runTestAccumulatingCombine(KV[] table, p.run(); } + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testWindowedCombine() { + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.timestamped(Arrays.asList(TABLE), + Arrays.asList(0L, 1L, 6L, 7L, 8L))) + .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())) + .apply(Window.>into(FixedWindows.of(Duration.millis(2)))); + + PCollection sum = input + .apply(Values.create()) + .apply(Combine.globally(new SumInts())); + + PCollection> sumPerKey = input + .apply(Combine.perKey(new SumInts())); + + DataflowAssert.that(sum).containsInAnyOrder(2, 5, 13); + DataflowAssert.that(sumPerKey).containsInAnyOrder( + KV.of("a", 2), + KV.of("a", 4), + KV.of("b", 1), + KV.of("b", 13)); + p.run(); + } + @Test @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) public void testWindowedCombineEmpty() { @@ -510,6 +537,7 @@ public boolean equals(Object otherObj) { return false; } + @Override public String toString() { return sum + ":" + inputs + ":" + merges + ":" + outputs; } From 73251c1265ef61afb0fb1f67f3ef23a446549217 Mon Sep 17 00:00:00 2001 From: lcwik Date: Wed, 17 Dec 2014 13:02:04 -0800 Subject: [PATCH 0039/1541] Add support for using ServiceLoader to register PipelineOptions and PipelineRunners with the SDK. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82356725 --- sdk/pom.xml | 9 +++ .../sdk/options/PipelineOptionsFactory.java | 65 ++++++++++------- .../sdk/options/PipelineOptionsRegistrar.java | 36 ++++++++++ .../runners/DataflowPipelineRegistrar.java | 58 +++++++++++++++ .../sdk/runners/DirectPipelineRegistrar.java | 53 ++++++++++++++ .../sdk/runners/PipelineRunnerRegistrar.java | 37 ++++++++++ .../options/PipelineOptionsFactoryTest.java | 14 +++- .../DataflowPipelineRegistrarTest.java | 72 +++++++++++++++++++ .../runners/DirectPipelineRegistrarTest.java | 69 ++++++++++++++++++ 9 files changed, 388 insertions(+), 25 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsRegistrar.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrar.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRegistrar.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerRegistrar.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrarTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRegistrarTest.java diff --git a/sdk/pom.xml b/sdk/pom.xml index 921fa8675ac80..1e9633695d10d 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -144,6 +144,7 @@ ${project.artifactId}-bundled-${project.version} + META-INF=target/classes/META-INF !${dataflow}.sdk.runners.worker.*, !${dataflow}.sdk.streaming.*, @@ -346,6 +347,14 @@ 2.4 + + + com.google.auto.service + auto-service + 1.0-rc2 + true + + org.hamcrest diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java index 176ff8c96e509..250baa62af277 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java @@ -16,13 +16,9 @@ package com.google.cloud.dataflow.sdk.options; -import com.google.cloud.dataflow.sdk.PipelineResult; -import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner; -import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner; -import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.runners.PipelineRunner; +import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar; import com.google.cloud.dataflow.sdk.runners.worker.DataflowWorkerHarness; -import com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineOptions; import com.google.common.base.Equivalence; import com.google.common.base.Function; import com.google.common.base.Preconditions; @@ -63,6 +59,7 @@ import java.util.List; import java.util.Map; import java.util.Queue; +import java.util.ServiceLoader; import java.util.Set; import java.util.SortedMap; import java.util.SortedSet; @@ -283,18 +280,7 @@ Class getProxyClass() { @SuppressWarnings("rawtypes") private static final Class[] EMPTY_CLASS_ARRAY = new Class[0]; private static final ObjectMapper MAPPER = new ObjectMapper(); - - // TODO: Add dynamic registration of pipeline runners. - private static final Map>> - SUPPORTED_PIPELINE_RUNNERS = - ImmutableMap.>>builder() - .put(DirectPipelineRunner.class.getSimpleName(), - DirectPipelineRunner.class) - .put(DataflowPipelineRunner.class.getSimpleName(), - DataflowPipelineRunner.class) - .put(BlockingDataflowPipelineRunner.class.getSimpleName(), - BlockingDataflowPipelineRunner.class) - .build(); + private static final Map>> SUPPORTED_PIPELINE_RUNNERS; /** Methods which are ignored when validating the proxy class. */ private static final Set IGNORED_METHODS; @@ -327,12 +313,31 @@ Class getProxyClass() { throw new ExceptionInInitializerError(e); } - // TODO Add support for dynamically loading and registering the options interfaces. + // Store the list of all available pipeline runners. + ImmutableMap.Builder>> builder = + new ImmutableMap.Builder<>(); + Set pipelineRunnerRegistrars = + Sets.newTreeSet(ObjectsClassComparator.INSTANCE); + pipelineRunnerRegistrars.addAll( + Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class))); + for (PipelineRunnerRegistrar registrar : pipelineRunnerRegistrars) { + for (Class> klass : registrar.getPipelineRunners()) { + builder.put(klass.getSimpleName(), klass); + } + } + SUPPORTED_PIPELINE_RUNNERS = builder.build(); + + // Load and register the list of all classes that extend PipelineOptions. register(PipelineOptions.class); - register(DirectPipelineOptions.class); - register(DataflowPipelineOptions.class); - register(BlockingDataflowPipelineOptions.class); - register(TestDataflowPipelineOptions.class); + Set pipelineOptionsRegistrars = + Sets.newTreeSet(ObjectsClassComparator.INSTANCE); + pipelineOptionsRegistrars.addAll( + Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class))); + for (PipelineOptionsRegistrar registrar : pipelineOptionsRegistrars) { + for (Class klass : registrar.getPipelineOptions()) { + register(klass); + } + } } /** @@ -430,12 +435,15 @@ public static Set> getRegisteredOptions() { return Collections.unmodifiableSet(REGISTERED_OPTIONS); } + static Map>> getRegisteredRunners() { + return SUPPORTED_PIPELINE_RUNNERS; + } + static List getPropertyDescriptors( Set> interfaces) { return COMBINED_CACHE.get(interfaces).getPropertyDescriptors(); } - /** * Creates a set of {@link DataflowWorkerHarnessOptions} based of a set of known system * properties. This is meant to only be used from the {@link DataflowWorkerHarness} as a method to @@ -701,6 +709,15 @@ private static void validateClass(Class iface, iface.getName()); } + /** A {@link Comparator} which uses the classes canonical name to compare them. */ + private static class ObjectsClassComparator implements Comparator { + static final ObjectsClassComparator INSTANCE = new ObjectsClassComparator(); + @Override + public int compare(Object o1, Object o2) { + return o1.getClass().getCanonicalName().compareTo(o2.getClass().getCanonicalName()); + } + } + /** A {@link Comparator} which uses the generic method signature to sort them. */ private static class MethodComparator implements Comparator { static final MethodComparator INSTANCE = new MethodComparator(); @@ -850,7 +867,7 @@ private static Map parseObjects( String runner = Iterables.getOnlyElement(entry.getValue()); Preconditions.checkArgument(SUPPORTED_PIPELINE_RUNNERS.containsKey(runner), "Unknown 'runner' specified %s, supported pipeline runners %s", - runner, SUPPORTED_PIPELINE_RUNNERS.keySet()); + runner, Sets.newTreeSet(SUPPORTED_PIPELINE_RUNNERS.keySet())); convertedOptions.put("runner", SUPPORTED_PIPELINE_RUNNERS.get(runner)); } else if (method.getReturnType().isArray() || Collection.class.isAssignableFrom(method.getReturnType())) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsRegistrar.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsRegistrar.java new file mode 100644 index 0000000000000..4235ec7dc056b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsRegistrar.java @@ -0,0 +1,36 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.auto.service.AutoService; + +import java.util.ServiceLoader; + +/** + * {@link PipelineOptions} creators have the ability to automatically have their + * {@link PipelineOptions} registered with this SDK by creating a {@link ServiceLoader} entry + * and a concrete implementation of this interface. + *

+ * Note that automatic registration of any {@PipelineOptions} requires users + * conform to the limitations discussed on {@link PipelineOptionsFactory#register(Class)}. + *

+ * 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. + */ +public interface PipelineOptionsRegistrar { + Iterable> getPipelineOptions(); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrar.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrar.java new file mode 100644 index 0000000000000..8e8f3c5017f7e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrar.java @@ -0,0 +1,58 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import com.google.auto.service.AutoService; +import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar; +import com.google.common.collect.ImmutableList; + +/** + * Contains the {@link PipelineOptionsRegistrar} and {@link PipelineRunnerRegistrar} for + * the {@link DataflowPipeline}. + */ +public class DataflowPipelineRegistrar { + private DataflowPipelineRegistrar() { } + + /** + * Register the {@link DataflowPipelineOptions} and {@link BlockingDataflowPipelineOptions}. + */ + @AutoService(PipelineOptionsRegistrar.class) + public static class Options implements PipelineOptionsRegistrar { + @Override + public Iterable> getPipelineOptions() { + return ImmutableList.>of( + DataflowPipelineOptions.class, + BlockingDataflowPipelineOptions.class); + } + } + + /** + * Register the {@link DataflowPipelineRunner} and {@link BlockingDataflowPipelineRunner}. + */ + @AutoService(PipelineRunnerRegistrar.class) + public static class Runner implements PipelineRunnerRegistrar { + @Override + public Iterable>> getPipelineRunners() { + return ImmutableList.>>of( + DataflowPipelineRunner.class, + BlockingDataflowPipelineRunner.class); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRegistrar.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRegistrar.java new file mode 100644 index 0000000000000..03be8546f2846 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRegistrar.java @@ -0,0 +1,53 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import com.google.auto.service.AutoService; +import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar; +import com.google.common.collect.ImmutableList; + +/** + * Contains the {@link PipelineOptionsRegistrar} and {@link PipelineRunnerRegistrar} for + * the {@link DirectPipeline}. + */ +public class DirectPipelineRegistrar { + private DirectPipelineRegistrar() { } + + /** + * Register the {@link DirectPipelineOptions}. + */ + @AutoService(PipelineRunnerRegistrar.class) + public static class Runner implements PipelineRunnerRegistrar { + @Override + public Iterable>> getPipelineRunners() { + return ImmutableList.>>of(DirectPipelineRunner.class); + } + } + + /** + * Register the {@link DirectPipelineRunner}. + */ + @AutoService(PipelineOptionsRegistrar.class) + public static class Options implements PipelineOptionsRegistrar { + @Override + public Iterable> getPipelineOptions() { + return ImmutableList.>of(DirectPipelineOptions.class); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerRegistrar.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerRegistrar.java new file mode 100644 index 0000000000000..aea6b5151747c --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerRegistrar.java @@ -0,0 +1,37 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import com.google.auto.service.AutoService; + +import java.util.ServiceLoader; + +/** + * {@link PipelineRunner} creators have the ability to automatically have their + * {@link PipelineRunner} registered with this SDK by creating a {@link ServiceLoader} entry + * and a concrete implementation of this interface. + *

+ * Note that automatic registration of any {@PipelineOptions} requires users + * conform to the limit that each {@link PipelineRunner}'s {@link Class#getSimpleName() simple name} + * must be unique. + *

+ * 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. + */ +public interface PipelineRunnerRegistrar { + public Iterable>> getPipelineRunners(); +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java index 206a13f70d171..872f31dae3554 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -44,6 +45,17 @@ public class PipelineOptionsFactoryTest { @Rule public ExpectedException expectedException = ExpectedException.none(); @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); + @Test + public void testAutomaticRegistrationOfPipelineOptions() { + assertTrue(PipelineOptionsFactory.getRegisteredOptions().contains(DirectPipelineOptions.class)); + } + + @Test + public void testAutomaticRegistrationOfRunners() { + assertEquals(DirectPipelineRunner.class, + PipelineOptionsFactory.getRegisteredRunners().get("DirectPipelineRunner")); + } + @Test public void testCreationFromSystemProperties() { System.getProperties().putAll(ImmutableMap @@ -493,7 +505,7 @@ public void testSettingRunner() { public void testSettingUnknownRunner() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Unknown 'runner' specified UnknownRunner, supported pipeline " - + "runners [DirectPipelineRunner, DataflowPipelineRunner, BlockingDataflowPipelineRunner]"); + + "runners [BlockingDataflowPipelineRunner, DataflowPipelineRunner, DirectPipelineRunner]"); String[] args = new String[] {"--runner=UnknownRunner"}; PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrarTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrarTest.java new file mode 100644 index 0000000000000..13c20d41d6805 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRegistrarTest.java @@ -0,0 +1,72 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import com.google.cloud.dataflow.sdk.options.BlockingDataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ServiceLoader; + +/** Tests for {@link DataflowPipelineRegistrar}. */ +@RunWith(JUnit4.class) +public class DataflowPipelineRegistrarTest { + @Test + public void testCorrectOptionsAreReturned() { + assertEquals(ImmutableList.of(DataflowPipelineOptions.class, + BlockingDataflowPipelineOptions.class), + new DataflowPipelineRegistrar.Options().getPipelineOptions()); + } + + @Test + public void testCorrectRunnersAreReturned() { + assertEquals(ImmutableList.of(DataflowPipelineRunner.class, + BlockingDataflowPipelineRunner.class), + new DataflowPipelineRegistrar.Runner().getPipelineRunners()); + } + + @Test + public void testServiceLoaderForOptions() { + for (PipelineOptionsRegistrar registrar : + Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) { + if (registrar instanceof DataflowPipelineRegistrar.Options) { + return; + } + } + fail("Expected to find " + DataflowPipelineRegistrar.Options.class); + } + + @Test + public void testServiceLoaderForRunner() { + for (PipelineRunnerRegistrar registrar : + Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) { + if (registrar instanceof DataflowPipelineRegistrar.Runner) { + return; + } + } + fail("Expected to find " + DataflowPipelineRegistrar.Runner.class); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRegistrarTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRegistrarTest.java new file mode 100644 index 0000000000000..6f59ff641d1ac --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRegistrarTest.java @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import com.google.cloud.dataflow.sdk.options.DirectPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsRegistrar; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ServiceLoader; + +/** Tests for {@link DirectPipelineRegistrar}. */ +@RunWith(JUnit4.class) +public class DirectPipelineRegistrarTest { + @Test + public void testCorrectOptionsAreReturned() { + assertEquals(ImmutableList.of(DirectPipelineOptions.class), + new DirectPipelineRegistrar.Options().getPipelineOptions()); + } + + @Test + public void testCorrectRunnersAreReturned() { + assertEquals(ImmutableList.of(DirectPipelineRunner.class), + new DirectPipelineRegistrar.Runner().getPipelineRunners()); + } + + @Test + public void testServiceLoaderForOptions() { + for (PipelineOptionsRegistrar registrar : + Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) { + if (registrar instanceof DirectPipelineRegistrar.Options) { + return; + } + } + fail("Expected to find " + DirectPipelineRegistrar.Options.class); + } + + @Test + public void testServiceLoaderForRunner() { + for (PipelineRunnerRegistrar registrar : + Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) { + if (registrar instanceof DirectPipelineRegistrar.Runner) { + return; + } + } + fail("Expected to find " + DirectPipelineRegistrar.Runner.class); + } +} From 7c9b182ff58b3c1d390fc4dfbd62b24ed97a31ea Mon Sep 17 00:00:00 2001 From: chernyak Date: Wed, 17 Dec 2014 14:18:13 -0800 Subject: [PATCH 0040/1541] Insert reshard for RequiresKeyedState for DirectPipelineRunner [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82364001 --- .../google/cloud/dataflow/sdk/transforms/ParDo.java | 12 +++++++++++- .../cloud/dataflow/sdk/transforms/ParDoTest.java | 2 +- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java index 8c9c43fb4e51f..4ddca9ef7dd12 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java @@ -23,6 +23,7 @@ import com.google.cloud.dataflow.sdk.util.PTuple; import com.google.cloud.dataflow.sdk.util.StringUtils; import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionTuple; import com.google.cloud.dataflow.sdk.values.PCollectionView; @@ -1044,7 +1045,16 @@ private static DoFnRunner evaluateHelper( for (DirectPipelineRunner.ValueWithMetadata elem : context.getPCollectionValuesWithMetadata(input)) { - executionContext.setKey(elem.getKey()); + if (doFn instanceof DoFn.RequiresKeyedState) { + // If the DoFn needs keyed state, set the implicit keys to the keys in the input elements. + if (!(elem.getValue() instanceof KV)) { + throw new IllegalStateException( + name + " marked as 'RequiresKeyedState' but input elements were not of type KV."); + } + executionContext.setKey(((KV) elem.getValue()).getKey()); + } else { + executionContext.setKey(elem.getKey()); + } fnRunner.processElement((WindowedValue) elem.getWindowedValue()); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java index a4fed2ada20b6..7034d1c53647c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java @@ -735,7 +735,7 @@ public void testParDoKeyedStateDoFnWithNonKvInput() { fail("should have failed"); } catch (RuntimeException exn) { assertThat(exn.toString(), - containsString("Keyed state is only available")); + containsString("'RequiresKeyedState' but input elements were not of type KV")); } } From d255b28d7a50f09f5cae09b516d8317456b9adab Mon Sep 17 00:00:00 2001 From: tudorm Date: Wed, 17 Dec 2014 22:43:59 -0800 Subject: [PATCH 0041/1541] Do not advance the iteration in getProgress() and updateStopPosition(); the two are called from a different thread than the read loop, and race with the ValuesIterator when using the same non-thread safe shuffle client. To this end, the GroupingShuffleSourceIterator maintains a monotonically increasing promisePosition and guarantees not to stop before reaching it. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82399182 --- .../runners/worker/GroupingShuffleSource.java | 72 +++++++------------ .../worker/GroupingShuffleSourceTest.java | 38 ++++++---- 2 files changed, 49 insertions(+), 61 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java index ac2bd5c3a78fe..19ce35800c6ee 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java @@ -20,7 +20,6 @@ import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToSourcePosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; -import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; import com.google.api.client.util.Preconditions; @@ -43,7 +42,6 @@ import com.google.cloud.dataflow.sdk.util.common.worker.Source; import com.google.cloud.dataflow.sdk.values.KV; -import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -158,15 +156,27 @@ private final class GroupingShuffleSourceIterator */ private ByteArrayShufflePosition stopPosition = null; + /** + * Position that this @GroupingShuffleSourceIterator is guaranteed + * not to stop before reaching (inclusive); @promisedPosition can + * only increase monotonically and is updated when advancing to a + * new group of records (either in the most recent call to next() + * or when peeked at in hasNext()). + */ + private ByteArrayShufflePosition promisedPosition = null; + /** The next group to be consumed, if available. */ private KeyGroupedShuffleEntries nextGroup = null; public GroupingShuffleSourceIterator(ShuffleEntryReader reader) { + promisedPosition = ByteArrayShufflePosition.fromBase64( + startShufflePosition); + if (promisedPosition == null) { + promisedPosition = new ByteArrayShufflePosition(new byte[0]); + } stopPosition = ByteArrayShufflePosition.fromBase64(stopShufflePosition); - this.groups = - new GroupingShuffleEntryIterator(reader.read( - ByteArrayShufflePosition.fromBase64(startShufflePosition), - stopPosition)) { + this.groups = new GroupingShuffleEntryIterator(reader.read( + promisedPosition, stopPosition)) { @Override protected void notifyElementRead(long byteSize) { GroupingShuffleSource.this.notifyElementRead(byteSize); @@ -177,26 +187,18 @@ protected void notifyElementRead(long byteSize) { private void advanceIfNecessary() { if (nextGroup == null && groups.hasNext()) { nextGroup = groups.next(); + promisedPosition = ByteArrayShufflePosition.of(nextGroup.position); } } @Override public boolean hasNext() throws IOException { - return hasNextInternal(); - } - - /** - * Returns false if the next group does not exist (i.e., no more - * records available) or the group is beyond @stopPosition. - */ - private boolean hasNextInternal() { advanceIfNecessary(); if (nextGroup == null) { return false; } - ByteArrayShufflePosition current = - ByteArrayShufflePosition.of(nextGroup.position); - return stopPosition == null || current.compareTo(stopPosition) < 0; + return stopPosition == null + || promisedPosition.compareTo(stopPosition) < 0; } @Override @@ -223,27 +225,11 @@ public WindowedValue>> next() throws IOException { */ @Override public Progress getProgress() { - com.google.api.services.dataflow.model.Position currentPosition = + com.google.api.services.dataflow.model.Position position = new com.google.api.services.dataflow.model.Position(); ApproximateProgress progress = new ApproximateProgress(); - if (hasNextInternal()) { - ByteArrayShufflePosition current = - ByteArrayShufflePosition.of(nextGroup.position); - currentPosition.setShufflePosition(current.encodeBase64()); - } else { - if (stopPosition != null) { - currentPosition.setShufflePosition(stopPosition.encodeBase64()); - } else { - // The original stop position described the end of the - // shuffle-position-space (or infinity) and all records have - // been consumed. - progress.setPercentComplete((float) 1.0); - progress.setRemainingTime(toCloudDuration(Duration.ZERO)); - return cloudProgressToSourceProgress(progress); - } - } - - progress.setPosition(currentPosition); + position.setShufflePosition(promisedPosition.encodeBase64()); + progress.setPosition(position); return cloudProgressToSourceProgress(progress); } @@ -272,18 +258,10 @@ public Position updateStopPosition(Progress proposedStopPosition) { ByteArrayShufflePosition newStopPosition = ByteArrayShufflePosition.fromBase64(stopCloudPosition.getShufflePosition()); - if (!hasNextInternal()) { - LOG.warn("Cannot update stop position to " - + stopCloudPosition.getShufflePosition() - + " since all input was consumed."); - return null; - } - ByteArrayShufflePosition current = - ByteArrayShufflePosition.of(nextGroup.position); - if (newStopPosition.compareTo(current) <= 0) { + if (newStopPosition.compareTo(promisedPosition) <= 0) { LOG.warn("Proposed stop position: " - + stopCloudPosition.getShufflePosition() + " <= current position: " - + current.encodeBase64()); + + stopCloudPosition.getShufflePosition() + " <= promised position: " + + promisedPosition.encodeBase64()); return null; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java index eacb1ef7661b5..df75774d9ccd6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java @@ -20,7 +20,6 @@ import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; -import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudDuration; import com.google.api.services.dataflow.model.ApproximateProgress; import com.google.api.services.dataflow.model.Position; @@ -43,7 +42,6 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.common.collect.Lists; -import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; import org.junit.Test; @@ -265,14 +263,29 @@ public void testReadFromEmptyShuffleSourceAndUpdateStopPosition() try (Source.SourceIterator>>> iter = shuffleSource.iterator(shuffleReader)) { + + // Can update the stop position, the source range spans all interval Position proposedStopPosition = new Position(); String stop = encodeBase64URLSafeString(fabricatePosition(0, null)); proposedStopPosition.setShufflePosition(stop); - // Cannot update stop position since all input was consumed. + Assert.assertEquals( + stop, + sourcePositionToCloudPosition( + iter.updateStopPosition( + cloudProgressToSourceProgress( + createApproximateProgress(proposedStopPosition)))) + .getShufflePosition()); + + + // Cannot update stop position to a position >= the current stop position + stop = encodeBase64URLSafeString(fabricatePosition(1, null)); + proposedStopPosition.setShufflePosition(stop); + Assert.assertEquals(null, iter.updateStopPosition( - cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); - } + cloudProgressToSourceProgress( + createApproximateProgress(proposedStopPosition)))); + } } @Test @@ -430,10 +443,6 @@ public void testReadFromShuffleSourceAndUpdateStopPosition() Assert.assertEquals(j, 1); ++i; } - - ApproximateProgress progress = - sourceProgressToCloudProgress(iter.getProgress()); - Assert.assertEquals(stop, progress.getPosition().getShufflePosition()); } Assert.assertEquals(i, kNumRecords); } @@ -484,11 +493,12 @@ public void testGetApproximateProgress() throws Exception { } Assert.assertFalse(sourceIterator.hasNext()); - ApproximateProgress finalProgress = - sourceProgressToCloudProgress(sourceIterator.getProgress()); - Assert.assertEquals(1.0, - (float) finalProgress.getPercentComplete(), 0.000000001); - Assert.assertEquals(Duration.ZERO, fromCloudDuration(finalProgress.getRemainingTime())); + // Cannot update stop position since all input was consumed. + Position proposedStopPosition = new Position(); + String stop = encodeBase64URLSafeString(fabricatePosition(0, null)); + proposedStopPosition.setShufflePosition(stop); + Assert.assertEquals(null, sourceIterator.updateStopPosition( + cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); } } From e8c361484b5c6793550f894c97b7a6bb2806408e Mon Sep 17 00:00:00 2001 From: klk Date: Thu, 18 Dec 2014 08:51:33 -0800 Subject: [PATCH 0042/1541] Infer coders lazily, when requested via getCoder(). This makes impossible a current problem where coder inference fails before a user has a change to set the coder explicitly. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82434275 --- .../dataflow/sdk/values/TypedPValue.java | 11 ++-- .../dataflow/sdk/transforms/ParDoTest.java | 52 ++++++++++++++++++- 2 files changed, 53 insertions(+), 10 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypedPValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypedPValue.java index 95b9b45f53770..2bde2121a9fe2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypedPValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypedPValue.java @@ -41,10 +41,7 @@ public abstract class TypedPValue extends PValueBase implements PValue { */ public Coder getCoder() { if (coder == null) { - throw new IllegalStateException( - "coder for " + this + " not set, and couldn't be inferred; " - + "either register a default Coder for its element type, " - + "or use setCoder() to specify one explicitly"); + inferCoderOrFail(); } return coder; } @@ -131,15 +128,13 @@ public TypedPValue setTypeTokenInternal(TypeToken typeToken) { return this; } - /** * If the coder is not explicitly set, this sets the coder for * this {@code TypedPValue} to the best coder that can be inferred * based upon the known {@code TypeToken}. By default, this is null, * but can and should be improved by subclasses. */ - @Override - public void finishSpecifyingOutput() { + private void inferCoderOrFail() { if (coder == null) { TypeToken token = getTypeToken(); CoderRegistry registry = getProducingTransformInternal() @@ -161,7 +156,7 @@ public void finishSpecifyingOutput() { + "or use setCoder() to specify one explicitly. " + "If a default coder is registered, it may not be found " + "due to type erasure; again, use setCoder() to specify " - + "a Coder explicitly"); + + "a Coder explicitly."); } } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java index 7034d1c53647c..7baf0daa951e0 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java @@ -27,6 +27,7 @@ import static org.hamcrest.core.IsEqual.equalTo; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -974,9 +975,15 @@ public void testSideOutputUnregisteredExplicitCoder() { PCollectionTuple outputTuple = input.apply(ParDo.of(new SideOutputDummyFn(sideTag)) .withOutputTags(mainTag, TupleTagList.of(sideTag))); - outputTuple.get(sideTag) - .setCoder(new TestDummyCoder()); + assertNull(pipeline.getCoderRegistry().getDefaultCoder(TestDummy.class)); + + outputTuple.get(sideTag).setCoder(new TestDummyCoder()); + outputTuple.get(sideTag).apply(View.asSingleton()); + + assertEquals(new TestDummyCoder(), outputTuple.get(sideTag).getCoder()); + outputTuple.get(sideTag).finishSpecifyingOutput(); // Check for crashes + assertEquals(new TestDummyCoder(), outputTuple.get(sideTag).getCoder()); // Check for corruption pipeline.run(); } @@ -997,6 +1004,47 @@ public void testMainOutputUnregisteredExplicitCoder() { pipeline.run(); } + @Test + public void testMainOutputApplySideOutputNoCoder() { + // Regression test: applying a transform to the main output + // should not cause a crash based on lack of a coder for the + // side output. + + Pipeline pipeline = TestPipeline.create(); + final TupleTag mainOutputTag = new TupleTag(); + final TupleTag sideOutputTag = new TupleTag(); + PCollectionTuple tuple = pipeline + .apply(Create.of(new TestDummy())) + .setCoder(TestDummyCoder.of()) + .apply(ParDo + .withOutputTags(mainOutputTag, TupleTagList.of(sideOutputTag)) + .of( + new DoFn() { + @Override public void processElement(ProcessContext context) { + TestDummy element = context.element(); + context.output(element); + context.sideOutput(sideOutputTag, element); + } + }) + ); + + // Before fix, tuple.get(mainOutputTag).apply(...) would indirectly trigger + // tuple.get(sideOutputTag).finishSpecifyingOutput() which would crash + // on a missing coder. + PCollection foo = tuple + .get(mainOutputTag) + .setCoder(TestDummyCoder.of()) + .apply(ParDo.of(new DoFn() { + public void processElement(ProcessContext context) { + context.output(1); + } + })); + + tuple.get(sideOutputTag).setCoder(TestDummyCoder.of()); + + pipeline.run(); + } + @Test public void testParDoOutputWithTimestamp() { Pipeline p = TestPipeline.create(); From f94256324259d5af106bd8281cfea039213ef072 Mon Sep 17 00:00:00 2001 From: jlewi Date: Thu, 18 Dec 2014 16:34:46 -0800 Subject: [PATCH 0043/1541] Add a teardown policy flag to the SDK. This is very useful for troubleshooting because it ensures VM's stay alive for manual inspection. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82474620 --- .../DataflowPipelineWorkerPoolOptions.java | 27 +++++++++++++++++++ .../runners/DataflowPipelineTranslator.java | 3 +++ 2 files changed, 30 insertions(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java index da0a6e588605f..a9be13a16df4a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java @@ -129,6 +129,33 @@ public String getApiServiceName() { String getMachineType(); void setMachineType(String value); + /** + * The policy for tearing down the workers spun up by the service. + */ + public enum TeardownPolicy { + TEARDOWN_ALWAYS("TEARDOWN_ALWAYS"), + TEARDOWN_NEVER("TEARDOWN_NEVER"); + + private final String teardownPolicy; + + private TeardownPolicy(String teardownPolicy) { + this.teardownPolicy = teardownPolicy; + } + + public String getTeardownPolicyName() { + return this.teardownPolicy; + } + } + + /** + * Teardown policy for the VMs. + * + *

By default this is left unset and the service sets the default policy. + */ + @Description("The teardown policy for the VMs.") + TeardownPolicy getTeardownPolicy(); + void setTeardownPolicy(TeardownPolicy value); + /** * List of local files to make available to workers. *

diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index 52305ccd54966..2fe5d1d8d147e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -348,6 +348,9 @@ public Job translate(List packages) { WorkerPool workerPool = new WorkerPool(); workerPool.setKind(HARNESS_WORKER_POOL); + if (options.getTeardownPolicy() != null) { + workerPool.setTeardownPolicy(options.getTeardownPolicy().getTeardownPolicyName()); + } // Pass the URL and endpoint to use to the worker pool. WorkerSettings workerSettings = new WorkerSettings(); From 623b6b781f95de7b24972c5c22efaca79e5a0f89 Mon Sep 17 00:00:00 2001 From: kirpichov Date: Fri, 19 Dec 2014 10:42:41 -0800 Subject: [PATCH 0044/1541] * Raise logging level of non-retryable HTTP request URLs to warning. * RetryHttpRequestInitializer is configurable with a set of HTTP codes to skip logging. * In case of GCS, skip logging for 404 because it's a normal situation during file staging, and if it's not normal in some other use case, the caller should be able to just handle the exception. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82526469 --- .../sdk/util/RetryHttpRequestInitializer.java | 23 ++++++++++++------- .../cloud/dataflow/sdk/util/Transport.java | 8 ++++++- .../util/RetryHttpRequestInitializerTest.java | 3 ++- 3 files changed, 24 insertions(+), 10 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java index c673b10204bc1..f5e660fa156bc 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java @@ -32,6 +32,8 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; import java.util.HashSet; import java.util.Set; @@ -52,7 +54,7 @@ public class RetryHttpRequestInitializer implements HttpRequestInitializer { /** * Http response codes that should be silently ignored. */ - private static final Set IGNORED_RESPONSE_CODES = new HashSet<>( + private static final Set DEFAULT_IGNORED_RESPONSE_CODES = new HashSet<>( Arrays.asList(307 /* Redirect, handled by Apiary client */, 308 /* Resume Incomplete, handled by Apiary client */)); @@ -74,7 +76,7 @@ public boolean handleIOException(HttpRequest request, boolean supportsRetry) if (willRetry) { LOG.debug("Request failed with IOException, will retry: {}", request.getUrl()); } else { - LOG.debug("Request failed with IOException, will NOT retry: {}", request.getUrl()); + LOG.warn("Request failed with IOException, will NOT retry: {}", request.getUrl()); } return willRetry; } @@ -83,9 +85,11 @@ public boolean handleIOException(HttpRequest request, boolean supportsRetry) private static class LoggingHttpBackoffUnsuccessfulResponseHandler implements HttpUnsuccessfulResponseHandler { private final HttpBackOffUnsuccessfulResponseHandler handler; + private final Set ignoredResponseCodes; public LoggingHttpBackoffUnsuccessfulResponseHandler(BackOff backoff, - Sleeper sleeper) { + Sleeper sleeper, Set ignoredResponseCodes) { + this.ignoredResponseCodes = ignoredResponseCodes; handler = new HttpBackOffUnsuccessfulResponseHandler(backoff); handler.setSleeper(sleeper); handler.setBackOffRequired( @@ -107,8 +111,8 @@ public boolean handleResponse(HttpRequest request, HttpResponse response, LOG.debug("Request failed with code {} will retry: {}", response.getStatusCode(), request.getUrl()); - } else if (!IGNORED_RESPONSE_CODES.contains(response.getStatusCode())) { - LOG.debug("Request failed with code {}, will NOT retry: {}", + } else if (!ignoredResponseCodes.contains(response.getStatusCode())) { + LOG.warn("Request failed with code {}, will NOT retry: {}", response.getStatusCode(), request.getUrl()); } @@ -122,19 +126,22 @@ public boolean handleResponse(HttpRequest request, HttpResponse response, private final Sleeper sleeper; // used for testing + private Set ignoredResponseCodes = new HashSet<>(DEFAULT_IGNORED_RESPONSE_CODES); + /** * @param chained a downstream HttpRequestInitializer, which will also be * applied to HttpRequest initialization. May be null. */ public RetryHttpRequestInitializer(@Nullable HttpRequestInitializer chained) { - this(chained, NanoClock.SYSTEM, Sleeper.DEFAULT); + this(chained, NanoClock.SYSTEM, Sleeper.DEFAULT, Collections.emptyList()); } public RetryHttpRequestInitializer(@Nullable HttpRequestInitializer chained, - NanoClock nanoClock, Sleeper sleeper) { + NanoClock nanoClock, Sleeper sleeper, Collection ignoredResponseCodes) { this.chained = chained; this.nanoClock = nanoClock; this.sleeper = sleeper; + this.ignoredResponseCodes = new HashSet<>(ignoredResponseCodes); } @Override @@ -155,7 +162,7 @@ public void initialize(HttpRequest request) throws IOException { new LoggingHttpBackoffUnsuccessfulResponseHandler( new ExponentialBackOff.Builder().setNanoClock(nanoClock) .setMultiplier(2).build(), - sleeper)); + sleeper, ignoredResponseCodes)); // Retry immediately on IOExceptions. LoggingHttpBackOffIOExceptionHandler loggingBackoffHandler = diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java index e27f7fcc4f885..d1accb2ac191f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java @@ -20,6 +20,8 @@ import com.google.api.client.http.HttpTransport; import com.google.api.client.json.JsonFactory; import com.google.api.client.json.jackson2.JacksonFactory; +import com.google.api.client.util.NanoClock; +import com.google.api.client.util.Sleeper; import com.google.api.services.bigquery.Bigquery; import com.google.api.services.dataflow.Dataflow; import com.google.api.services.pubsub.Pubsub; @@ -34,6 +36,7 @@ import java.net.MalformedURLException; import java.net.URL; import java.security.GeneralSecurityException; +import java.util.Arrays; /** * Helpers for cloud communication. @@ -135,7 +138,10 @@ public static Dataflow.Builder newDataflowClient(DataflowPipelineOptions options public static Storage.Builder newStorageClient(GcsOptions options) { return new Storage.Builder(getTransport(), getJsonFactory(), - new RetryHttpRequestInitializer(options.getGcpCredential())) + new RetryHttpRequestInitializer( + // Do not log the code 404. Code up the stack will deal with 404's if needed, and + // logging it by default clutters the output during file staging. + options.getGcpCredential(), NanoClock.SYSTEM, Sleeper.DEFAULT, Arrays.asList(404))) .setApplicationName(options.getAppName()); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializerTest.java index 45924560630ba..09998a36a2fe6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializerTest.java @@ -53,6 +53,7 @@ import java.io.IOException; import java.security.PrivateKey; +import java.util.Arrays; /** * Tests for RetryHttpRequestInitializer. @@ -99,7 +100,7 @@ protected LowLevelHttpRequest buildRequest(String method, String url) mockCredential, new MockNanoClock(), new Sleeper() { @Override public void sleep(long millis) throws InterruptedException {} - }); + }, Arrays.asList(418 /* I'm a teapot */)); storage = new Storage.Builder(lowLevelTransport, jsonFactory, initializer) .setApplicationName("test").build(); } From c0ea51817e673b47d06653dbcc5c27563bc0d667 Mon Sep 17 00:00:00 2001 From: millsd Date: Fri, 19 Dec 2014 11:10:22 -0800 Subject: [PATCH 0045/1541] Add DoFn.Context.sideOutputWithTimestamp [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82528619 --- .../cloud/dataflow/sdk/transforms/DoFn.java | 24 +++++++++++++- .../dataflow/sdk/transforms/RateLimiting.java | 7 +++++ .../cloud/dataflow/sdk/util/DoFnContext.java | 5 +++ .../dataflow/sdk/util/DoFnProcessContext.java | 18 +++++++---- .../dataflow/sdk/transforms/ParDoTest.java | 31 +++++++++++++++++++ 5 files changed, 78 insertions(+), 7 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java index c18a646e4278a..d6057d5b29c65 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java @@ -128,7 +128,29 @@ public abstract class Context { */ public abstract void sideOutput(TupleTag tag, T output); - // TODO: add sideOutputWithTimestamp[AndWindows] + /** + * Adds the given element to the specified side output {@code PCollection}, + * with the given timestamp. + * + *

If invoked from {@link DoFn#processElement}), the timestamp + * must not be older than the input element's timestamp minus + * {@link DoFn#getAllowedTimestampSkew}. The output element will + * be in the same windows as the input element. + * + *

Is is illegal to invoke this from {@link #startBundle} or + * {@link #finishBundle} unless the input {@code PCollection} is + * windowed by the + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. + * If this is the case, the output element's timestamp will be + * the given timestamp and its window will be the + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. + * + * @throws IllegalArgumentException if the number of outputs exceeds + * the limit of 1,000 outputs per DoFn + * @see ParDo#withOutputTags + */ + public abstract void sideOutputWithTimestamp( + TupleTag tag, T output, Instant timestamp); /** * Returns an aggregator with aggregation logic specified by the CombineFn diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java index fd32303613bcb..d5f177ac83c8e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java @@ -296,6 +296,13 @@ public void sideOutput(TupleTag tag, T output) { } } + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + synchronized (RateLimitingDoFn.this) { + context.sideOutputWithTimestamp(tag, output, timestamp); + } + } + @Override public Aggregator createAggregator( String name, Combine.CombineFn combiner) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java index 08d53a7d464b6..722a0227d6ca4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java @@ -174,6 +174,11 @@ public void sideOutput(TupleTag tag, T output) { Arrays.asList(GlobalWindow.Window.INSTANCE)); } + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + sideOutputWindowedValue(tag, output, timestamp, Arrays.asList(GlobalWindow.Window.INSTANCE)); + } + private String generateInternalAggregatorName(String userName) { return "user-" + stepContext.getStepName() + "-" + userName; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnProcessContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnProcessContext.java index d393e6f0b8b66..b6faaa4e8631a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnProcessContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnProcessContext.java @@ -88,12 +88,7 @@ public void output(O output) { @Override public void outputWithTimestamp(O output, Instant timestamp) { - Instant originalTimestamp = windowedValue.getTimestamp(); - - if (originalTimestamp != null) { - Preconditions.checkArgument( - !timestamp.isBefore(originalTimestamp.minus(fn.getAllowedTimestampSkew()))); - } + checkTimestamp(timestamp); context.outputWindowedValue(output, timestamp, windowedValue.getWindows()); } @@ -112,6 +107,12 @@ public void sideOutput(TupleTag tag, T output) { windowedValue.getWindows()); } + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + checkTimestamp(timestamp); + context.sideOutputWindowedValue(tag, output, timestamp, windowedValue.getWindows()); + } + @Override public Aggregator createAggregator( String name, Combine.CombineFn combiner) { @@ -133,4 +134,9 @@ public Instant timestamp() { public Collection windows() { return windowedValue.getWindows(); } + + private void checkTimestamp(Instant timestamp) { + Preconditions.checkArgument( + !timestamp.isBefore(windowedValue.getTimestamp().minus(fn.getAllowedTimestampSkew()))); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java index 7baf0daa951e0..163426b1c4c85 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java @@ -1066,6 +1066,37 @@ public void testParDoOutputWithTimestamp() { p.run(); } + @Test + public void testParDoSideOutputWithTimestamp() { + Pipeline p = TestPipeline.create(); + + PCollection input = + createInts(p, Arrays.asList(3, 42, 6)).setOrdered(true); + + final TupleTag mainTag = new TupleTag(){}; + final TupleTag sideTag = new TupleTag(){}; + + PCollection output = + input + .apply(ParDo.withOutputTags(mainTag, TupleTagList.of(sideTag)).of( + new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.sideOutputWithTimestamp( + sideTag, c.element(), new Instant(c.element().longValue())); + } + })).get(sideTag) + .apply(ParDo.of(new TestShiftTimestampDoFn(Duration.ZERO, Duration.ZERO))) + .apply(ParDo.of(new TestFormatTimestampDoFn())); + + DataflowAssert.that(output).containsInAnyOrder( + "processing: 3, timestamp: 3", + "processing: 42, timestamp: 42", + "processing: 6, timestamp: 6"); + + p.run(); + } + @Test public void testParDoShiftTimestamp() { Pipeline p = TestPipeline.create(); From df1e1afcae2536a1ea9d0b5b4cef51b5ba98d498 Mon Sep 17 00:00:00 2001 From: malo Date: Fri, 19 Dec 2014 13:13:59 -0800 Subject: [PATCH 0046/1541] Set the sdk major_version to 0. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82537631 --- .../cloud/dataflow/sdk/runners/DataflowPipelineRunner.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index a2574774d3315..1a0d873307375 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -196,9 +196,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { // Requirements about the service. Map environmentVersion = new HashMap<>(); - // TODO: Specify the environment major version. - // environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, - // ENVIRONMENT_MAJOR_VERSION); + environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, ENVIRONMENT_MAJOR_VERSION); newJob.getEnvironment().setVersion(environmentVersion); // Default jobType is DATA_PARALLEL which is for java batch. String jobType = "DATA_PARALLEL"; From 3be5a2ab0c81f13c4d086ce3058b65272d070f07 Mon Sep 17 00:00:00 2001 From: kirpichov Date: Fri, 19 Dec 2014 13:48:27 -0800 Subject: [PATCH 0047/1541] Avoids creating a thread in ReadOperation if progress estimation is not enabled or is continuous. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82540119 --- .../sdk/util/common/worker/ReadOperation.java | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java index 1930e0e61aaad..1d4e5ec9bef5b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java @@ -134,20 +134,21 @@ protected void runReadLoop() throws Exception { } // TODO: Consider using the ExecutorService from PipelineOptions instead. - Thread updateRequester = new Thread() { - @Override - public void run() { - while (true) { - isProgressUpdateRequested.set(true); - try { - Thread.sleep(progressUpdatePeriodMs); - } catch (InterruptedException e) { - break; + Thread updateRequester = null; + if (progressUpdatePeriodMs != 0) { + updateRequester = new Thread() { + @Override + public void run() { + while (true) { + isProgressUpdateRequested.set(true); + try { + Thread.sleep(progressUpdatePeriodMs); + } catch (InterruptedException e) { + break; + } } } - } - }; - if (progressUpdatePeriodMs != 0) { + }; updateRequester.start(); } From 69b6df51a8e532dc233bc840a4ddd55acc584d82 Mon Sep 17 00:00:00 2001 From: peihe Date: Fri, 19 Dec 2014 14:04:45 -0800 Subject: [PATCH 0048/1541] Cleanup: tests UngroupedShuffleSource, GroupingShuffleSource APIs with corresponding ShuffleSinks, and avoids tests implementation details about data encoding into shuffle. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82541343 --- .../worker/GroupingShuffleSourceTest.java | 97 ++++++++++++------- .../worker/UngroupedShuffleSourceTest.java | 49 ++++++---- 2 files changed, 93 insertions(+), 53 deletions(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java index df75774d9ccd6..21ecb0af04f2c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java @@ -25,7 +25,6 @@ import com.google.api.services.dataflow.model.Position; import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.coders.InstantCoder; import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; @@ -37,6 +36,7 @@ import com.google.cloud.dataflow.sdk.util.common.Reiterable; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; import com.google.cloud.dataflow.sdk.util.common.worker.Source; import com.google.cloud.dataflow.sdk.util.common.worker.Source.SourceIterator; import com.google.cloud.dataflow.sdk.values.KV; @@ -89,52 +89,81 @@ private enum ValuesToRead { private void runTestReadShuffleSource(List>> input, ValuesToRead valuesToRead) throws Exception { - Coder> elemCoder = - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); + Coder>> sinkElemCoder = + WindowedValue.getFullCoder( + KvCoder.of(BigEndianIntegerCoder.of(), + StringUtf8Coder.of()), + IntervalWindow.getCoder()); + + Coder>>> sourceElemCoder = + WindowedValue.getFullCoder( + KvCoder.of( + BigEndianIntegerCoder.of(), + IterableCoder.of(StringUtf8Coder.of())), + IntervalWindow.getCoder()); + + // Write to shuffle with GROUP_KEYS ShuffleSink. + ShuffleSink> shuffleSink = new ShuffleSink<>( + PipelineOptionsFactory.create(), + null, ShuffleSink.ShuffleKind.GROUP_KEYS, + sinkElemCoder); + + TestShuffleWriter shuffleWriter = new TestShuffleWriter(); + + int kvCount = 0; + List actualSizes = new ArrayList<>(); + try (Sink.SinkWriter>> shuffleSinkWriter = + shuffleSink.writer(shuffleWriter)) { + for (KV> kvs : input) { + Integer key = kvs.getKey(); + for (String value : kvs.getValue()) { + ++kvCount; + actualSizes.add(shuffleSinkWriter.add( + WindowedValue.of(KV.of(key, value), + timestamp, + Lists.newArrayList(window)))); + } + } + } + List records = shuffleWriter.getRecords(); + Assert.assertEquals(kvCount, records.size()); + Assert.assertEquals(shuffleWriter.getSizes(), actualSizes); + + // Read from shuffle with GroupingShuffleSource. BatchModeExecutionContext context = new BatchModeExecutionContext(); - GroupingShuffleSource> shuffleSource = + GroupingShuffleSource shuffleSource = new GroupingShuffleSource<>( PipelineOptionsFactory.create(), null, null, null, - WindowedValue.getFullCoder( - KvCoder.of( - BigEndianIntegerCoder.of(), - IterableCoder.of( - WindowedValue.getFullCoder(StringUtf8Coder.of(), - IntervalWindow.getCoder()))), - IntervalWindow.getCoder()), + sourceElemCoder, context); ExecutorTestUtils.TestSourceObserver observer = new ExecutorTestUtils.TestSourceObserver(shuffleSource); TestShuffleReader shuffleReader = new TestShuffleReader(); List expectedSizes = new ArrayList<>(); - for (KV> kvs : input) { - Integer key = kvs.getKey(); - byte[] keyByte = CoderUtils.encodeToByteArray(BigEndianIntegerCoder.of(), key); - - for (String value : kvs.getValue()) { - byte[] valueByte = CoderUtils.encodeToByteArray( - elemCoder, WindowedValue.of(value, timestamp, Lists.newArrayList(window))); - byte[] skey = CoderUtils.encodeToByteArray(InstantCoder.of(), timestamp); - ShuffleEntry shuffleEntry = new ShuffleEntry(keyByte, skey, valueByte); - shuffleReader.addEntry(shuffleEntry); - expectedSizes.add(shuffleEntry.length()); - } + for (ShuffleEntry record : records) { + expectedSizes.add(record.length()); + shuffleReader.addEntry(record); } - List>>> actual = new ArrayList<>(); - try (SourceIterator>>>> iter = + List>> actual = new ArrayList<>(); + try (SourceIterator>>> iter = shuffleSource.iterator(shuffleReader)) { - Iterable> prevValuesIterable = null; - Iterator> prevValuesIterator = null; + Iterable prevValuesIterable = null; + Iterator prevValuesIterator = null; while (iter.hasNext()) { Assert.assertTrue(iter.hasNext()); Assert.assertTrue(iter.hasNext()); - KV>> elem = iter.next().getValue(); + WindowedValue>> windowedValue = iter.next(); + // Verify value is in an empty windows. + Assert.assertEquals(Long.MIN_VALUE, windowedValue.getTimestamp().getMillis()); + Assert.assertEquals(0, windowedValue.getWindows().size()); + + KV> elem = windowedValue.getValue(); Integer key = elem.getKey(); - List> values = new ArrayList<>(); + List values = new ArrayList<>(); if (valuesToRead.ordinal() > ValuesToRead.SKIP_VALUES.ordinal()) { if (prevValuesIterable != null) { prevValuesIterable.iterator(); // Verifies that this does not throw. @@ -143,8 +172,8 @@ private void runTestReadShuffleSource(List>> input, prevValuesIterator.hasNext(); // Verifies that this does not throw. } - Iterable> valuesIterable = elem.getValue(); - Iterator> valuesIterator = valuesIterable.iterator(); + Iterable valuesIterable = elem.getValue(); + Iterator valuesIterator = valuesIterable.iterator(); if (valuesToRead.ordinal() >= ValuesToRead.READ_ONE_VALUE.ordinal()) { while (valuesIterator.hasNext()) { @@ -187,13 +216,13 @@ private void runTestReadShuffleSource(List>> input, } } - List>>> expected = new ArrayList<>(); + List>> expected = new ArrayList<>(); for (KV> kvs : input) { Integer key = kvs.getKey(); - List> values = new ArrayList<>(); + List values = new ArrayList<>(); if (valuesToRead.ordinal() >= ValuesToRead.READ_ONE_VALUE.ordinal()) { for (String value : kvs.getValue()) { - values.add(WindowedValue.of(value, timestamp, Lists.newArrayList(window))); + values.add(value); if (valuesToRead == ValuesToRead.READ_ONE_VALUE) { break; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java index bf9a15c05b343..75d9803ed7eed 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java @@ -18,13 +18,13 @@ import com.google.cloud.dataflow.sdk.TestUtils; import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; -import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; -import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; import com.google.cloud.dataflow.sdk.util.common.worker.Source; import com.google.common.collect.Lists; @@ -46,35 +46,44 @@ public class UngroupedShuffleSourceTest { private static final Instant timestamp = new Instant(123000); private static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); - private byte[] asShuffleKey(long seqNum) throws Exception { - return CoderUtils.encodeToByteArray(BigEndianLongCoder.of(), seqNum); - } + void runTestReadShuffleSource(List expected) throws Exception { + Coder> elemCoder = + WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), IntervalWindow.getCoder()); - private byte[] asShuffleValue(Integer value) throws Exception { - return CoderUtils.encodeToByteArray( - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), IntervalWindow.getCoder()), - WindowedValue.of(value, timestamp, Lists.newArrayList(window))); - } + // Write to shuffle with UNGROUPED ShuffleSink. + ShuffleSink shuffleSink = new ShuffleSink<>( + PipelineOptionsFactory.create(), + null, ShuffleSink.ShuffleKind.UNGROUPED, + elemCoder); - private void runTestReadShuffleSource(List expected) throws Exception { + TestShuffleWriter shuffleWriter = new TestShuffleWriter(); + + List actualSizes = new ArrayList<>(); + try (Sink.SinkWriter> shuffleSinkWriter = + shuffleSink.writer(shuffleWriter)) { + for (Integer value : expected) { + actualSizes.add(shuffleSinkWriter.add( + WindowedValue.of(value, timestamp, Lists.newArrayList(window)))); + } + } + List records = shuffleWriter.getRecords(); + Assert.assertEquals(expected.size(), records.size()); + Assert.assertEquals(shuffleWriter.getSizes(), actualSizes); + + // Read from shuffle with UngroupedShuffleSource. UngroupedShuffleSource> shuffleSource = new UngroupedShuffleSource<>( PipelineOptionsFactory.create(), null, null, null, - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), IntervalWindow.getCoder())); + elemCoder); ExecutorTestUtils.TestSourceObserver observer = new ExecutorTestUtils.TestSourceObserver(shuffleSource); TestShuffleReader shuffleReader = new TestShuffleReader(); List expectedSizes = new ArrayList<>(); - long seqNum = 0; - for (Integer value : expected) { - byte[] shuffleKey = asShuffleKey(seqNum++); - byte[] shuffleValue = asShuffleValue(value); - shuffleReader.addEntry(shuffleKey, shuffleValue); - - ShuffleEntry record = new ShuffleEntry(shuffleKey, null, shuffleValue); + for (ShuffleEntry record : records) { expectedSizes.add(record.length()); + shuffleReader.addEntry(record); } List actual = new ArrayList<>(); @@ -84,6 +93,8 @@ private void runTestReadShuffleSource(List expected) throws Exception { Assert.assertTrue(iter.hasNext()); Assert.assertTrue(iter.hasNext()); WindowedValue elem = iter.next(); + Assert.assertEquals(timestamp, elem.getTimestamp()); + Assert.assertEquals(Lists.newArrayList(window), elem.getWindows()); actual.add(elem.getValue()); } Assert.assertFalse(iter.hasNext()); From 6aaf644fca4c6f7eb4b40c2adc396050539a3ab5 Mon Sep 17 00:00:00 2001 From: earhart Date: Fri, 19 Dec 2014 15:33:34 -0800 Subject: [PATCH 0049/1541] Quieting a few more warnings. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82547955 --- .../sdk/coders/SerializableCoder.java | 1 + .../runners/DataflowPipelineTranslator.java | 6 ++++- .../sdk/runners/DirectPipelineRunner.java | 1 + .../runners/dataflow/AvroIOTranslator.java | 2 ++ .../runners/dataflow/TextIOTranslator.java | 2 ++ .../runners/worker/AssignWindowsParDoFn.java | 1 + .../sdk/runners/worker/AvroSinkFactory.java | 1 + .../sdk/runners/worker/AvroSourceFactory.java | 1 + .../sdk/runners/worker/CombineValuesFn.java | 12 +++++----- .../sdk/runners/worker/NormalParDoFn.java | 14 +++++++----- .../worker/PartitioningShuffleSource.java | 2 +- .../worker/SourceOperationExecutor.java | 1 + .../sdk/transforms/ApproximateQuantiles.java | 1 + .../cloud/dataflow/sdk/transforms/Create.java | 7 ++++++ .../dataflow/sdk/transforms/GroupByKey.java | 2 +- .../sdk/transforms/RemoveDuplicates.java | 1 + .../sdk/transforms/join/UnionCoder.java | 2 ++ .../dataflow/sdk/util/AbstractWindowSet.java | 1 + .../dataflow/sdk/util/AssignWindowsDoFn.java | 2 ++ .../sdk/util/BatchModeExecutionContext.java | 1 + .../dataflow/sdk/util/BufferingWindowSet.java | 2 +- .../cloud/dataflow/sdk/util/DoFnContext.java | 5 +++-- .../cloud/dataflow/sdk/util/DoFnRunner.java | 1 + .../cloud/dataflow/sdk/util/PTuple.java | 1 + .../cloud/dataflow/sdk/util/Serializer.java | 1 + .../sdk/util/common/PeekingReiterator.java | 2 +- .../worker/GroupingShuffleEntryIterator.java | 2 +- .../util/common/worker/OutputReceiver.java | 10 ++++----- .../sdk/util/common/worker/ReadOperation.java | 3 +++ .../sdk/util/common/worker/WorkExecutor.java | 1 + .../util/common/worker/WriteOperation.java | 15 ++++++++----- .../dataflow/sdk/values/CodedTupleTag.java | 2 +- .../dataflow/sdk/values/CodedTupleTagMap.java | 1 + .../dataflow/sdk/values/PCollection.java | 6 ++--- .../dataflow/sdk/values/PCollectionTuple.java | 2 +- .../dataflow/sdk/values/TimestampedValue.java | 1 + .../sdk/coders/ByteArrayCoderTest.java | 9 +++++--- .../cloud/dataflow/sdk/io/AvroIOTest.java | 1 + .../cloud/dataflow/sdk/io/TextIOTest.java | 5 +++-- .../options/PipelineOptionsFactoryTest.java | 1 + .../BlockingDataflowPipelineRunnerTest.java | 4 ++-- .../DataflowPipelineTranslatorTest.java | 1 + .../sdk/runners/PipelineRunnerTest.java | 4 ++-- .../runners/worker/AvroSinkFactoryTest.java | 2 +- .../runners/worker/AvroSourceFactoryTest.java | 1 + .../runners/worker/CombineValuesFnTest.java | 1 + .../DataflowWorkProgressUpdaterTest.java | 1 + .../worker/InMemorySourceFactoryTest.java | 2 +- .../runners/worker/ParDoFnFactoryTest.java | 2 +- .../worker/ShuffleSinkFactoryTest.java | 1 + .../worker/ShuffleSourceFactoryTest.java | 1 + .../runners/worker/TextSinkFactoryTest.java | 2 +- .../runners/worker/TextSourceFactoryTest.java | 2 +- .../sdk/runners/worker/TextSourceTest.java | 22 ++++++++++++------- .../transforms/ApproximateQuantilesTest.java | 2 +- .../dataflow/sdk/transforms/CombineTest.java | 9 ++++++-- .../dataflow/sdk/transforms/CountTest.java | 5 +++-- .../dataflow/sdk/transforms/CreateTest.java | 2 +- .../sdk/transforms/GroupByKeyTest.java | 2 +- .../dataflow/sdk/transforms/KeysTest.java | 2 ++ .../dataflow/sdk/transforms/KvSwapTest.java | 1 + .../dataflow/sdk/transforms/ParDoTest.java | 2 +- .../dataflow/sdk/transforms/SampleTest.java | 3 +++ .../sdk/transforms/SimpleStatsFnsTest.java | 3 +++ .../dataflow/sdk/transforms/TopTest.java | 4 ++-- .../dataflow/sdk/transforms/ValuesTest.java | 2 ++ .../sdk/transforms/join/UnionCoderTest.java | 2 +- .../transforms/windowing/WindowingTest.java | 2 +- .../dataflow/sdk/util/AggregatorImplTest.java | 1 + .../sdk/util/GroupAlsoByWindowsDoFnTest.java | 1 + .../sdk/util/InstanceBuilderTest.java | 1 + .../StreamingGroupAlsoByWindowsDoFnTest.java | 1 + .../dataflow/sdk/util/common/CounterTest.java | 1 + .../sdk/util/common/CounterTestUtils.java | 6 ++--- .../util/common/worker/ExecutorTestUtils.java | 1 + .../common/worker/FlattenOperationTest.java | 1 + .../common/worker/MapTaskExecutorTest.java | 1 + .../common/worker/OutputReceiverTest.java | 5 +++-- .../common/worker/ParDoOperationTest.java | 1 + .../PartialGroupByKeyOperationTest.java | 1 + .../util/common/worker/ReadOperationTest.java | 1 + .../util/common/worker/StateSamplerTest.java | 5 +++++ .../common/worker/WriteOperationTest.java | 1 + 83 files changed, 172 insertions(+), 75 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java index 8e58b4d96e291..303370aac0e51 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java @@ -57,6 +57,7 @@ public static SerializableCoder of(Class type) { } @JsonCreator + @SuppressWarnings("unchecked") public static SerializableCoder of(@JsonProperty("type") String classType) throws ClassNotFoundException { Class clazz = Class.forName(classType); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index 2fe5d1d8d147e..aff881ddfe999 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -100,6 +100,7 @@ * DataflowPipelineTranslator knows how to translate Pipeline objects * into Dataflow API Jobs. */ +@SuppressWarnings({"rawtypes", "unchecked"}) public class DataflowPipelineTranslator { // Must be kept in sync with their internal counterparts. public static final String HARNESS_WORKER_POOL = "harness"; @@ -495,7 +496,10 @@ public void addStep(PTransform transform, Step original) { @Nullable List> outputInfoList = null; try { // TODO: This should be done via a Structs accessor. - outputInfoList = (List>) properties.get(PropertyNames.OUTPUT_INFO); + @SuppressWarnings("unchecked") + @Nullable List> list = + (List>) properties.get(PropertyNames.OUTPUT_INFO); + outputInfoList = list; } catch (Exception e) { throw new RuntimeException("Inconsistent dataflow pipeline translation", e); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java index 2b0ca03beaab0..c71eaf8434d3c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java @@ -67,6 +67,7 @@ * *

Throws an exception from {@link #run} if execution fails. */ +@SuppressWarnings({"rawtypes", "unchecked"}) public class DirectPipelineRunner extends PipelineRunner { private static final Logger LOG = LoggerFactory.getLogger(DirectPipelineRunner.class); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java index d7e36c54fc050..f191b25ba86ee 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java @@ -34,6 +34,7 @@ public class AvroIOTranslator { /** * Implements AvroIO Read translation for the Dataflow backend. */ + @SuppressWarnings("rawtypes") public static class ReadTranslator implements TransformTranslator { @Override @@ -63,6 +64,7 @@ private void translateReadHelper( /** * Implements AvroIO Write translation for the Dataflow backend. */ + @SuppressWarnings("rawtypes") public static class WriteTranslator implements TransformTranslator { @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java index 0a2f1f8a32571..ee70949d5954c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java @@ -34,6 +34,7 @@ public class TextIOTranslator { /** * Implements TextIO Read translation for the Dataflow backend. */ + @SuppressWarnings({"rawtypes", "unchecked"}) public static class ReadTranslator implements TransformTranslator { @Override public void translate( @@ -75,6 +76,7 @@ private void translateReadHelper( /** * Implements TextIO Write translation for the Dataflow backend. */ + @SuppressWarnings({"rawtypes", "unchecked"}) public static class WriteTranslator implements TransformTranslator { @Override public void translate( diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java index f1ae7f11b9374..00558a306ead6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java @@ -41,6 +41,7 @@ * A wrapper around an AssignWindowsDoFn. This class is the same as * NormalParDoFn, except that it gets deserialized differently. */ +@SuppressWarnings({"rawtypes", "unchecked"}) class AssignWindowsParDoFn extends NormalParDoFn { public static AssignWindowsParDoFn create( PipelineOptions options, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java index 9a20d17aee220..f1c96dcf478c1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java @@ -30,6 +30,7 @@ /** * Creates an AvroSink from a CloudObject spec. */ +@SuppressWarnings("rawtypes") public final class AvroSinkFactory { // Do not instantiate. private AvroSinkFactory() {} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactory.java index 329d8b66e2ee1..740f94965dd80 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactory.java @@ -31,6 +31,7 @@ /** * Creates an AvroSource from a CloudObject spec. */ +@SuppressWarnings("rawtypes") public class AvroSourceFactory { // Do not instantiate. private AvroSourceFactory() {} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java index 761aaa950dba4..63c4089f616a2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java @@ -42,7 +42,7 @@ /** * A wrapper around a decoded user value combining function. */ -@SuppressWarnings("serial") +@SuppressWarnings({"rawtypes", "serial", "unchecked"}) public class CombineValuesFn extends NormalParDoFn { /** * The optimizer may split run the user combiner in 3 separate @@ -141,8 +141,8 @@ private CombineValuesDoFn( @Override public void processElement(ProcessContext c) { - KV> kv = (KV>) c.element(); - K key = (K) kv.getKey(); + KV> kv = c.element(); + K key = kv.getKey(); c.output(KV.of(key, this.combineFn.apply(key, kv.getValue()))); } @@ -162,7 +162,7 @@ private AddInputsDoFn( @Override public void processElement(ProcessContext c) { - KV> kv = (KV>) c.element(); + KV> kv = c.element(); K key = kv.getKey(); VA accum = this.combineFn.createAccumulator(key); for (VI input : kv.getValue()) { @@ -187,7 +187,7 @@ private MergeAccumulatorsDoFn( @Override public void processElement(ProcessContext c) { - KV> kv = (KV>) c.element(); + KV> kv = c.element(); K key = kv.getKey(); VA accum = this.combineFn.mergeAccumulators(key, kv.getValue()); @@ -209,7 +209,7 @@ private ExtractOutputDoFn( @Override public void processElement(ProcessContext c) { - KV kv = (KV) c.element(); + KV kv = c.element(); K key = kv.getKey(); VO output = this.combineFn.extractOutput(key, kv.getValue()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java index c6e5f9f163e35..79d729b974d87 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java @@ -48,6 +48,7 @@ /** * A wrapper around a decoded user DoFn. */ +@SuppressWarnings({"rawtypes", "unchecked"}) public class NormalParDoFn extends ParDoFn { public static NormalParDoFn create( PipelineOptions options, @@ -67,14 +68,14 @@ public static NormalParDoFn create( if (!(deserializedFn instanceof DoFn)) { throw new Exception("unexpected kind of DoFn: " + deserializedFn.getClass().getName()); } - DoFn fn = (DoFn) deserializedFn; + DoFn fn = (DoFn) deserializedFn; PTuple sideInputValues = PTuple.empty(); if (sideInputInfos != null) { for (SideInputInfo sideInputInfo : sideInputInfos) { Object sideInputValue = SideInputUtils.readSideInput( options, sideInputInfo, executionContext); - TupleTag tag = new TupleTag(sideInputInfo.getTag()); + TupleTag tag = new TupleTag<>(sideInputInfo.getTag()); sideInputValues = sideInputValues.and(tag, sideInputValue); } } @@ -119,16 +120,16 @@ public NormalParDoFn(PipelineOptions options, ExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator) { this.options = options; - this.fn = fn; + this.fn = (DoFn) fn; this.sideInputValues = sideInputValues; if (outputTags.size() < 1) { throw new AssertionError("expected at least one output"); } - this.mainOutputTag = new TupleTag(outputTags.get(0)); + this.mainOutputTag = new TupleTag<>(outputTags.get(0)); this.sideOutputTags = new ArrayList<>(); if (outputTags.size() > 1) { for (String tag : outputTags.subList(1, outputTags.size())) { - this.sideOutputTags.add(new TupleTag(tag)); + this.sideOutputTags.add(new TupleTag(tag)); } } this.stepName = stepName; @@ -202,8 +203,9 @@ public void output(Receiver receiver, WindowedValue output) { } @Override + @SuppressWarnings("unchecked") public void processElement(Object elem) throws Exception { - fnRunner.processElement((WindowedValue) elem); + fnRunner.processElement((WindowedValue) elem); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSource.java index 5394a26cc47fc..e1401b6cd83ad 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSource.java @@ -75,7 +75,7 @@ private void initCoder(Coder>> coder) throws Exception { "unexpected kind of coder for elements read from " + "a key-partitioning shuffle: " + elemCoder); } - KvCoder kvCoder = (KvCoder) elemCoder; + KvCoder kvCoder = (KvCoder) elemCoder; this.keyCoder = kvCoder.getKeyCoder(); windowedValueCoder = windowedElemCoder.withValueCoder(kvCoder.getValueCoder()); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java index 2db18b2724740..34bd27966bfc7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java @@ -32,6 +32,7 @@ /** * An executor for a source operation, defined by a {@code SourceOperationRequest}. */ +@SuppressWarnings("resource") public class SourceOperationExecutor extends WorkExecutor { private static final Logger LOG = LoggerFactory.getLogger(MapTaskExecutor.class); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java index a2f0094cbb463..5702583ebe65c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java @@ -206,6 +206,7 @@ PTransform, PCollection>> globally(int numQuantiles) { * * @param the type of the values being combined */ + @SuppressWarnings("serial") public static class ApproximateQuantilesCombineFn & Serializable> extends AccumulatingCombineFn diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java index 0ff9deb039bc9..d28e6a28193f4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java @@ -77,6 +77,7 @@ * * @param the type of the elements of the resulting {@code PCollection} */ +@SuppressWarnings("serial") public class Create extends PTransform> { /** @@ -111,6 +112,7 @@ public static Create of(Iterable elems) { * {@link Coder} specified explicitly, via a call to * {@link PCollection#setCoder}. */ + @SafeVarargs public static Create of(T... elems) { return of(Arrays.asList(elems)); } @@ -287,6 +289,11 @@ public void processElement(ProcessContext c) { ///////////////////////////////////////////////////////////////////////////// static { + registerDefaultTransformEvaluator(); + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + private static void registerDefaultTransformEvaluator() { DirectPipelineRunner.registerDefaultTransformEvaluator( Create.class, new DirectPipelineRunner.TransformEvaluator() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index e8acdbad5be39..c6d0acf4094d4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -375,7 +375,7 @@ boolean sortsValues() { registerWithDirectPipelineRunner(); } - @SuppressWarnings("rawtypes") + @SuppressWarnings({"rawtypes", "unchecked"}) private static void registerWithDirectPipelineRunner() { DirectPipelineRunner.registerDefaultTransformEvaluator( GroupByKeyOnly.class, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java index 0e4f21f75b781..9540eda6da7d1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java @@ -52,6 +52,7 @@ * @param the type of the elements of the input and output * {@code PCollection}s */ +@SuppressWarnings("serial") public class RemoveDuplicates extends PTransform, PCollection> { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java index f0987dfaffee5..f3d128c67b9e1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java @@ -110,6 +110,7 @@ public List> getComponents() { @Override public boolean isRegisterByteSizeObserverCheap(RawUnionValue union, Context context) { int index = getIndexForEncoding(union); + @SuppressWarnings("unchecked") Coder coder = (Coder) elementCoders.get(index); return coder.isRegisterByteSizeObserverCheap(union.getValue(), context); } @@ -125,6 +126,7 @@ public void registerByteSizeObserver( // Write out the union tag. observer.update(VarInt.getLength(index)); // Write out the actual value. + @SuppressWarnings("unchecked") Coder coder = (Coder) elementCoders.get(index); coder.registerByteSizeObserver(union.getValue(), observer, context); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AbstractWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AbstractWindowSet.java index dda2488dac346..fa54eee605b9c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AbstractWindowSet.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AbstractWindowSet.java @@ -59,6 +59,7 @@ static class WindowMergeContext extends WindowingFn.MergeContext { private final AbstractWindowSet windowSet; + @SuppressWarnings("unchecked") public WindowMergeContext( AbstractWindowSet windowSet, WindowingFn windowingFn) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AssignWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AssignWindowsDoFn.java index 7649a8c637248..a0b91b53037d9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AssignWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AssignWindowsDoFn.java @@ -30,6 +30,7 @@ * @param Type of elements being windowed * @param Window type */ +@SuppressWarnings("serial") public class AssignWindowsDoFn extends DoFn { private WindowingFn fn; @@ -38,6 +39,7 @@ public AssignWindowsDoFn(WindowingFn fn) { } @Override + @SuppressWarnings("unchecked") public void processElement(ProcessContext c) throws Exception { final DoFnProcessContext context = (DoFnProcessContext) c; Collection windows = diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java index 2d42407c94377..fb5277189f2ba 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java @@ -142,6 +142,7 @@ public void deleteTagList(CodedTupleTag tag) { } @Override + @SuppressWarnings("unchecked") public Iterable readTagList(CodedTupleTag tag) { Map, List> perKeyTagLists = tagLists.get(getKey()); if (perKeyTagLists == null || perKeyTagLists.get(tag) == null) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java index 4801d6d64c3c3..957434c154421 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java @@ -186,7 +186,7 @@ private static Map> emptyIfNull(Map> input) { private Map> deepCopy(Map> mergeTree) { Map> newMergeTree = new HashMap<>(); for (Map.Entry> entry : mergeTree.entrySet()) { - newMergeTree.put(entry.getKey(), new HashSet(entry.getValue())); + newMergeTree.put(entry.getKey(), new HashSet(entry.getValue())); } return newMergeTree; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java index 722a0227d6ca4..7eca7d5f510bf 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java @@ -73,14 +73,14 @@ public DoFnContext(PipelineOptions options, this.mainOutputTag = mainOutputTag; this.outputMap = new HashMap<>(); outputMap.put(mainOutputTag, outputManager.initialize(mainOutputTag)); - for (TupleTag sideOutputTag : sideOutputTags) { + for (TupleTag sideOutputTag : sideOutputTags) { outputMap.put(sideOutputTag, outputManager.initialize(sideOutputTag)); } this.stepContext = stepContext; this.addCounterMutator = addCounterMutator; } - public R getReceiver(TupleTag tag) { + public R getReceiver(TupleTag tag) { R receiver = outputMap.get(tag); if (receiver == null) { throw new IllegalArgumentException( @@ -97,6 +97,7 @@ public PipelineOptions getPipelineOptions() { } @Override + @SuppressWarnings("unchecked") public T sideInput(PCollectionView view) { TupleTag tag = view.getTagInternal(); if (!sideInputs.has(tag)) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java index 3a6d4af1c1a92..d4d788d89af7f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java @@ -79,6 +79,7 @@ public static DoFnRunner create( mainOutputTag, sideOutputTags, stepContext, addCounterMutator); } + @SuppressWarnings({"rawtypes", "unchecked"}) public static DoFnRunner createWithListOutputs( PipelineOptions options, DoFn fn, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PTuple.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PTuple.java index 98fe4606807a5..879693584ae56 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PTuple.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PTuple.java @@ -132,6 +132,7 @@ public V get(TupleTag tag) { private final Map, ?> valueMap; + @SuppressWarnings("rawtypes") private PTuple() { this(new LinkedHashMap()); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Serializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Serializer.java index 42071ec467ee3..68ada31d981ce 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Serializer.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Serializer.java @@ -119,6 +119,7 @@ public static T deserialize(Map serialized, Class clazz) * URI value from the {@link CloudKnownType} enum. Upon finding this type * information, it converts it into the correspondingly typed Java value. */ + @SuppressWarnings("unchecked") private static Object deserializeCloudKnownTypes(Object src) { if (src instanceof Map) { Map srcMap = (Map) src; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/PeekingReiterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/PeekingReiterator.java index d139380c65c12..8789c4e84c8df 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/PeekingReiterator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/PeekingReiterator.java @@ -69,7 +69,7 @@ public void remove() { @Override public PeekingReiterator copy() { - return new PeekingReiterator(this); + return new PeekingReiterator<>(this); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/GroupingShuffleEntryIterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/GroupingShuffleEntryIterator.java index 19428201f0395..4dc9a1e1a8f16 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/GroupingShuffleEntryIterator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/GroupingShuffleEntryIterator.java @@ -61,7 +61,7 @@ public abstract class GroupingShuffleEntryIterator public GroupingShuffleEntryIterator( Reiterator shuffleIterator) { this.shuffleIterator = - new PeekingReiterator( + new PeekingReiterator<>( new ProgressTrackingReiterator<>( shuffleIterator, new ProgressTrackerGroup() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiver.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiver.java index a13b74afbf8b3..d758bbd298d9f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiver.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiver.java @@ -37,7 +37,7 @@ public class OutputReceiver implements Receiver { private final String outputName; // Might be null, e.g., undeclared outputs will not have an // elementByteSizeObservable. - private final ElementByteSizeObservable elementByteSizeObservable; + private final ElementByteSizeObservable elementByteSizeObservable; private final Counter elementCount; private Counter byteCount = null; private Counter meanByteCount = null; @@ -52,16 +52,16 @@ public class OutputReceiver implements Receiver { public OutputReceiver(String outputName, String counterPrefix, CounterSet.AddCounterMutator addCounterMutator) { - this(outputName, (ElementByteSizeObservable) null, - counterPrefix, addCounterMutator); + this(outputName, null, counterPrefix, addCounterMutator); } + @SuppressWarnings("unchecked") public OutputReceiver(String outputName, - ElementByteSizeObservable elementByteSizeObservable, + ElementByteSizeObservable elementByteSizeObservable, String counterPrefix, CounterSet.AddCounterMutator addCounterMutator) { this.outputName = outputName; - this.elementByteSizeObservable = elementByteSizeObservable; + this.elementByteSizeObservable = (ElementByteSizeObservable) elementByteSizeObservable; elementCount = addCounterMutator.addCounter( Counter.longs(elementsCounterName(counterPrefix, outputName), SUM)); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java index 1d4e5ec9bef5b..f4caef51d0d1e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java @@ -114,6 +114,7 @@ public Source getSource() { @Override public void start() throws Exception { try (StateSampler.ScopedState start = stateSampler.scopedState(startState)) { + assert start != null; super.start(); runReadLoop(); } @@ -129,6 +130,7 @@ protected void runReadLoop() throws Exception { source.addObserver(new SourceObserver()); try (StateSampler.ScopedState process = stateSampler.scopedState(processState)) { + assert process != null; synchronized (sourceIteratorLock) { sourceIterator = source.iterator(); } @@ -162,6 +164,7 @@ public void run() { // Stop position update request comes concurrently. // Accesses to iterator need to be synchronized. try (StateSampler.ScopedState read = stateSampler.scopedState(readState)) { + assert read != null; synchronized (sourceIteratorLock) { if (!sourceIterator.hasNext()) { break; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java index 63270b682ebce..c4a603f70caad 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java @@ -30,6 +30,7 @@ /** * Abstract executor for WorkItem tasks. */ +@SuppressWarnings("resource") public abstract class WorkExecutor implements AutoCloseable { /** The output counters for this task. */ private final CounterSet outputCounters; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperation.java index 6f8b2e586548f..3965e0d0a19c5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperation.java @@ -28,7 +28,7 @@ public class WriteOperation extends ReceivingOperation { /** * The Sink this operation writes to. */ - public final Sink sink; + public final Sink sink; /** * The total byte counter for all data written by this operation. @@ -38,10 +38,10 @@ public class WriteOperation extends ReceivingOperation { /** * The Sink's writer this operation writes to, created by start(). */ - Sink.SinkWriter writer; + Sink.SinkWriter writer; public WriteOperation(String operationName, - Sink sink, + Sink sink, OutputReceiver[] receivers, String counterPrefix, CounterSet.AddCounterMutator addCounterMutator, @@ -54,7 +54,7 @@ public WriteOperation(String operationName, } /** Invoked by tests. */ - public WriteOperation(Sink sink, + public WriteOperation(Sink sink, String counterPrefix, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) { @@ -67,7 +67,7 @@ protected String bytesCounterName(String counterPrefix, return operationName + "-ByteCount"; } - public Sink getSink() { + public Sink getSink() { return sink; } @@ -75,8 +75,9 @@ public Sink getSink() { public void start() throws Exception { try (StateSampler.ScopedState start = stateSampler.scopedState(startState)) { + assert start != null; super.start(); - writer = sink.writer(); + writer = (Sink.SinkWriter) sink.writer(); } } @@ -84,6 +85,7 @@ public void start() throws Exception { public void process(Object outputElem) throws Exception { try (StateSampler.ScopedState process = stateSampler.scopedState(processState)) { + assert process != null; checkStarted(); byteCount.addValue(writer.add(outputElem)); } @@ -93,6 +95,7 @@ public void process(Object outputElem) throws Exception { public void finish() throws Exception { try (StateSampler.ScopedState finish = stateSampler.scopedState(finishState)) { + assert finish != null; checkStarted(); writer.close(); super.finish(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTag.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTag.java index 770d460249339..ffbf9c577ce53 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTag.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTag.java @@ -46,7 +46,7 @@ public class CodedTupleTag extends TupleTag { * @param the type of the values associated with the tag */ public static CodedTupleTag of(String id, Coder coder) { - return new CodedTupleTag(id, coder); + return new CodedTupleTag<>(id, coder); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTagMap.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTagMap.java index 6f96c694ea2e6..a30d3feeeb612 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTagMap.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/CodedTupleTagMap.java @@ -45,6 +45,7 @@ public static CodedTupleTagMap of(Map, Object> map) { * {@code CodedTupleTagMap}, or {@code null} if the tag has no * asssociated value. */ + @SuppressWarnings("unchecked") public T get(CodedTupleTag tag) { return (T) map.get(tag); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java index 34ad2163a8a1e..fa09bd09106af 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java @@ -191,7 +191,7 @@ public Output apply( * *

By default, no merging is performed. */ - private WindowingFn windowingFn; + private WindowingFn windowingFn; private PCollection() {} @@ -212,7 +212,7 @@ public PCollection setTypeTokenInternal(TypeToken typeToken) { * *

For use by primitive transformations only. */ - public PCollection setWindowingFnInternal(WindowingFn windowingFn) { + public PCollection setWindowingFnInternal(WindowingFn windowingFn) { this.windowingFn = windowingFn; return this; } @@ -234,7 +234,7 @@ public PCollection setPipelineInternal(Pipeline pipeline) { *

For use by primitive transformations only. */ public static PCollection createPrimitiveOutputInternal( - WindowingFn windowingFn) { + WindowingFn windowingFn) { return new PCollection().setWindowingFnInternal(windowingFn); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java index fecc175f4d3cc..349ec2cc750cc 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java @@ -183,7 +183,7 @@ public Output apply( *

For use by primitive transformations only. */ public static PCollectionTuple ofPrimitiveOutputsInternal( - TupleTagList outputTags, WindowingFn windowingFn) { + TupleTagList outputTags, WindowingFn windowingFn) { Map, PCollection> pcollectionMap = new LinkedHashMap<>(); for (TupleTag outputTag : outputTags.tupleTags) { if (pcollectionMap.containsKey(outputTag)) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java index 9d91a18cb3cf0..ffa0202e15b2c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java @@ -65,6 +65,7 @@ public Instant getTimestamp() { /** * Coder for {@code TimestampedValue}. */ + @SuppressWarnings("serial") public static class TimestampedValueCoder extends StandardCoder> { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoderTest.java index b6d2b3c657d04..8f500a1655994 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoderTest.java @@ -32,7 +32,8 @@ /** Unit tests for {@link ByteArrayCoder}. */ @RunWith(JUnit4.class) public class ByteArrayCoderTest { - @Test public void testOuterContext() throws CoderException, IOException { + @Test + public void testOuterContext() throws CoderException, IOException { byte[] buffer = {0xa, 0xb, 0xc}; ByteArrayOutputStream os = new ByteArrayOutputStream(); @@ -44,7 +45,8 @@ public class ByteArrayCoderTest { assertThat(decoded, equalTo(buffer)); } - @Test public void testNestedContext() throws CoderException, IOException { + @Test + public void testNestedContext() throws CoderException, IOException { byte[][] buffers = {{0xa, 0xb, 0xc}, {}, {}, {0xd, 0xe}, {}}; ByteArrayOutputStream os = new ByteArrayOutputStream(); @@ -60,7 +62,8 @@ public class ByteArrayCoderTest { } } - @Test public void testRegisterByteSizeObserver() throws Exception { + @Test + public void testRegisterByteSizeObserver() throws Exception { CounterTestUtils.testByteCount(ByteArrayCoder.of(), Coder.Context.OUTER, new byte[][]{{ 0xa, 0xb, 0xc }}); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java index ad6f16567e92c..1d45f4ab9386c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java @@ -249,6 +249,7 @@ void runTestWrite(AvroIO.Write.Bound write, String expectedName) User[] users = generateAvroObjects(); DirectPipeline p = DirectPipeline.createForTest(); + @SuppressWarnings("unchecked") PCollection input = p.apply(Create.of(Arrays.asList((T[]) users))) .setCoder((Coder) AvroCoder.of(User.class)); PDone output = input.apply(write.withoutSharding()); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java index 31bb18cd955f5..46bdd5944a969 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java @@ -69,6 +69,7 @@ * Tests for TextIO Read and Write transforms. */ @RunWith(JUnit4.class) +@SuppressWarnings("unchecked") public class TextIOTest { @Rule @@ -152,7 +153,7 @@ void runTestRead(T[] expected, Coder coder) throws Exception { if (coder.equals(StringUtf8Coder.of())) { TextIO.Read.Bound readStrings = TextIO.Read.from(filename); // T==String - read = (TextIO.Read.Bound) readStrings; + read = (TextIO.Read.Bound) readStrings; } else { read = TextIO.Read.from(filename).withCoder(coder); } @@ -222,7 +223,7 @@ void runTestWrite(T[] elems, Coder coder) throws Exception { TextIO.Write.Bound writeStrings = TextIO.Write.to(filename).withoutSharding(); // T==String - write = (TextIO.Write.Bound) writeStrings; + write = (TextIO.Write.Bound) writeStrings; } else { write = TextIO.Write.to(filename).withCoder(coder).withoutSharding(); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java index 872f31dae3554..d5fb27d9203c1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java @@ -380,6 +380,7 @@ public static interface Arrays extends PipelineOptions { } @Test + @SuppressWarnings("rawtypes") public void testArrays() { String[] args = new String[] { "--boolean=true", diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunnerTest.java index 398326e8a385e..8ab8de6dbfe12 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunnerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunnerTest.java @@ -49,10 +49,10 @@ public class BlockingDataflowPipelineRunnerTest { // This class mocks a call to DataflowPipelineJob.waitToFinish(): // it blocks the thread to simulate waiting, // and releases the blocking once signaled - static class MockWaitToFinish implements Answer { + static class MockWaitToFinish implements Answer { NotificationHelper jobCompleted = new NotificationHelper(); - public Object answer(InvocationOnMock invocation) throws InterruptedException { + public JobState answer(InvocationOnMock invocation) throws InterruptedException { System.out.println("MockWaitToFinish.answer(): Wait for signaling job completion."); assertTrue("Test did not receive mock job completion signal", jobCompleted.waitTillSet(10000)); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java index 12649de8acfca..3eb90edbf78e6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java @@ -239,6 +239,7 @@ public void testPredefinedAddStep() throws Exception { */ private static OutputReference getOutputPortReference(Step step) throws Exception { // TODO: This should be done via a Structs accessor. + @SuppressWarnings("unchecked") List> output = (List>) step.getProperties().get(PropertyNames.OUTPUT_INFO); String outputTagId = getString(Iterables.getOnlyElement(output), PropertyNames.OUTPUT_NAME); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerTest.java index 520e03e28b9dd..238e1844ca16a 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerTest.java @@ -58,7 +58,7 @@ public void testLongName() throws IOException { options.setGcsUtil(mockGcsUtil); options.setRunner(DirectPipelineRunner.class); options.setGcpCredential(new TestCredential()); - PipelineRunner runner = PipelineRunner.fromOptions(options); + PipelineRunner runner = PipelineRunner.fromOptions(options); assertTrue(runner instanceof DirectPipelineRunner); } @@ -71,7 +71,7 @@ public void testShortName() throws IOException { options.setGcsUtil(mockGcsUtil); options.setRunner(DirectPipelineRunner.class); options.setGcpCredential(new TestCredential()); - PipelineRunner runner = PipelineRunner.fromOptions(options); + PipelineRunner runner = PipelineRunner.fromOptions(options); assertTrue(runner instanceof DirectPipelineRunner); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactoryTest.java index 79653feabc4ce..c55be77cbceba 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactoryTest.java @@ -76,7 +76,7 @@ public void testCreateAvroSink() throws Exception { Sink sink = runTestCreateAvroSink(pathToAvroFile, coder.asCloudObject()); Assert.assertThat(sink, new IsInstanceOf(AvroSink.class)); - AvroSink avroSink = (AvroSink) sink; + AvroSink avroSink = (AvroSink) sink; Assert.assertEquals(pathToAvroFile, avroSink.filenamePrefix); Assert.assertEquals(coder.getValueCoder(), avroSink.avroCoder); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactoryTest.java index 3c81950fd29d6..1db06de456057 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactoryTest.java @@ -40,6 +40,7 @@ * Tests for AvroSourceFactory. */ @RunWith(JUnit4.class) +@SuppressWarnings("rawtypes") public class AvroSourceFactoryTest { private final String pathToAvroFile = "/path/to/file.avro"; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java index 4833ff19a3109..abddf51ad5bc9 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java @@ -184,6 +184,7 @@ public void process(Object outputElem) { } } + @SuppressWarnings("rawtypes") private static ParDoFn createCombineValuesFn( String phase, Combine.KeyedCombineFn combineFn) throws Exception { // This partially mirrors the work that diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java index 2167a504183fc..952a51bbce9c6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java @@ -77,6 +77,7 @@ /** Unit tests for {@link DataflowWorkProgressUpdater}. */ @RunWith(JUnit4.class) +@SuppressWarnings("resource") public class DataflowWorkProgressUpdaterTest { static class TestMapTaskExecutor extends MapTaskExecutor { ApproximateProgress progress = null; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactoryTest.java index 64cf4f5520217..82d2c82e99761 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactoryTest.java @@ -82,7 +82,7 @@ void runTestCreateInMemorySource(List elements, Source source = SourceFactory.create(PipelineOptionsFactory.create(), cloudSource, new BatchModeExecutionContext()); Assert.assertThat(source, new IsInstanceOf(InMemorySource.class)); - InMemorySource inMemorySource = (InMemorySource) source; + InMemorySource inMemorySource = (InMemorySource) source; Assert.assertEquals(encodedElements(elements, coder), inMemorySource.encodedElements); Assert.assertEquals(expectedStart, inMemorySource.startIndex); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java index 63d38ebdd4109..6ceb61f267708 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java @@ -43,7 +43,7 @@ * Tests for ParDoFnFactory. */ @RunWith(JUnit4.class) -@SuppressWarnings("serial") +@SuppressWarnings({"rawtypes", "serial", "unchecked"}) public class ParDoFnFactoryTest { static class TestDoFn extends DoFn { final String stringState; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkFactoryTest.java index 4b8901af34b57..0bf4916359cfa 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkFactoryTest.java @@ -42,6 +42,7 @@ * Tests for ShuffleSinkFactory. */ @RunWith(JUnit4.class) +@SuppressWarnings("rawtypes") public class ShuffleSinkFactoryTest { ShuffleSink runTestCreateShuffleSinkHelper(byte[] shuffleWriterConfig, String shuffleKind, diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSourceFactoryTest.java index 75fc7479687e7..1803b06d7fd1e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSourceFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSourceFactoryTest.java @@ -45,6 +45,7 @@ * and PartitioningShuffleSourceFactory. */ @RunWith(JUnit4.class) +@SuppressWarnings({"rawtypes", "unchecked"}) public class ShuffleSourceFactoryTest { T runTestCreateShuffleSource(byte[] shuffleReaderConfig, diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkFactoryTest.java index 9f9e63090a6e5..1a2b843242c82 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSinkFactoryTest.java @@ -70,7 +70,7 @@ void runTestCreateTextSink(String filename, cloudSink, new BatchModeExecutionContext()); Assert.assertThat(sink, new IsInstanceOf(TextSink.class)); - TextSink textSink = (TextSink) sink; + TextSink textSink = (TextSink) sink; Assert.assertEquals(filename, textSink.namePrefix); Assert.assertEquals( appendTrailingNewlines == null ? true : appendTrailingNewlines, diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactoryTest.java index 2fa50b567e781..05d8a721fefb3 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactoryTest.java @@ -70,7 +70,7 @@ void runTestCreateTextSource(String filename, cloudSource, new BatchModeExecutionContext()); Assert.assertThat(source, new IsInstanceOf(TextSource.class)); - TextSource textSource = (TextSource) source; + TextSource textSource = (TextSource) source; Assert.assertEquals(filename, textSource.filename); Assert.assertEquals( stripTrailingNewlines == null ? true : stripTrailingNewlines, diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceTest.java index 8aee7aaf00529..87f8379479ad8 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceTest.java @@ -25,7 +25,6 @@ import com.google.cloud.dataflow.sdk.TestUtils; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.TextualIntegerCoder; -import com.google.cloud.dataflow.sdk.runners.worker.TextSource.TextFileIterator; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; import com.google.cloud.dataflow.sdk.util.common.worker.Source; @@ -421,7 +420,8 @@ public void testUpdateStopPosition() throws Exception { ExecutorTestUtils.TestSourceObserver observer = new ExecutorTestUtils.TestSourceObserver(textSource); - try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + try (TextSource.TextFileIterator iterator = + (TextSource.TextFileIterator) textSource.iterator()) { Assert.assertNull(iterator.updateStopPosition( cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); } @@ -437,7 +437,8 @@ public void testUpdateStopPosition() throws Exception { ExecutorTestUtils.TestSourceObserver observer = new ExecutorTestUtils.TestSourceObserver(textSource); - try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + try (TextSource.TextFileIterator iterator = + (TextSource.TextFileIterator) textSource.iterator()) { Assert.assertNull(iterator.getEndOffset()); Assert.assertEquals( stop, @@ -463,7 +464,8 @@ public void testUpdateStopPosition() throws Exception { ExecutorTestUtils.TestSourceObserver observer = new ExecutorTestUtils.TestSourceObserver(textSource); - try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + try (TextSource.TextFileIterator iterator = + (TextSource.TextFileIterator) textSource.iterator()) { Assert.assertEquals(fileContent[0], iterator.next()); Assert.assertEquals(fileContent[1], iterator.next()); Assert.assertThat(sourceProgressToCloudProgress(iterator.getProgress()) @@ -488,7 +490,8 @@ public void testUpdateStopPosition() throws Exception { ExecutorTestUtils.TestSourceObserver observer = new ExecutorTestUtils.TestSourceObserver(textSource); - try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + try (TextSource.TextFileIterator iterator = + (TextSource.TextFileIterator) textSource.iterator()) { Assert.assertEquals(fileContent[0], iterator.next()); Assert.assertNull(iterator.updateStopPosition( cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); @@ -535,7 +538,8 @@ private void stopPositionTestInternal(Long startOffset, tmpFile.getPath(), false, startOffset, endOffset, StringUtf8Coder.of()); - try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + try (TextSource.TextFileIterator iterator = + (TextSource.TextFileIterator) textSource.iterator()) { while (iterator.hasNext()) { accumulatedRead.append((String) iterator.next()); } @@ -548,7 +552,8 @@ private void stopPositionTestInternal(Long startOffset, StringUtf8Coder.of()); accumulatedRead = new StringBuilder(); - try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + try (TextSource.TextFileIterator iterator = + (TextSource.TextFileIterator) textSource.iterator()) { while (iterator.hasNext()) { accumulatedRead.append((String) iterator.next()); } @@ -561,7 +566,8 @@ private void stopPositionTestInternal(Long startOffset, StringUtf8Coder.of()); accumulatedRead = new StringBuilder(); - try (TextFileIterator iterator = (TextFileIterator) textSource.iterator()) { + try (TextSource.TextFileIterator iterator = + (TextSource.TextFileIterator) textSource.iterator()) { while (iterator.hasNext()) { accumulatedRead.append((String) iterator.next()); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java index 808ef4a23b6a8..045c1df4d913e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java @@ -47,7 +47,7 @@ * Tests for {@link ApproximateQuantiles}. */ @RunWith(JUnit4.class) -@SuppressWarnings("serial") +@SuppressWarnings({"serial", "unchecked"}) public class ApproximateQuantilesTest { static final List> TABLE = Arrays.asList( diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java index 4307619d3f93d..403190b6115aa 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java @@ -72,7 +72,7 @@ @SuppressWarnings("serial") public class CombineTest { - @SuppressWarnings("unchecked") + @SuppressWarnings({"rawtypes", "unchecked"}) static final KV[] TABLE = new KV[] { KV.of("a", 1), KV.of("a", 1), @@ -81,7 +81,7 @@ public class CombineTest { KV.of("b", 13), }; - @SuppressWarnings("unchecked") + @SuppressWarnings({"rawtypes", "unchecked"}) static final KV[] EMPTY_TABLE = new KV[] { }; @@ -117,6 +117,7 @@ private void runTestSimpleCombine(KV[] table, @Test @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + @SuppressWarnings({"rawtypes", "unchecked"}) public void testSimpleCombine() { runTestSimpleCombine(TABLE, 20, new KV[] { KV.of("a", 6), KV.of("b", 14) }); @@ -124,10 +125,12 @@ public void testSimpleCombine() { @Test @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + @SuppressWarnings({"rawtypes", "unchecked"}) public void testSimpleCombineEmpty() { runTestSimpleCombine(EMPTY_TABLE, 0, new KV[] { }); } + @SuppressWarnings("unchecked") private void runTestBasicCombine(KV[] table, Set globalUnique, KV>[] perKeyUnique) { @@ -151,6 +154,7 @@ private void runTestBasicCombine(KV[] table, @Test @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + @SuppressWarnings({"rawtypes", "unchecked"}) public void testBasicCombine() { runTestBasicCombine(TABLE, ImmutableSet.of(1, 13, 4), new KV[] { KV.of("a", (Set) ImmutableSet.of(1, 4)), @@ -159,6 +163,7 @@ public void testBasicCombine() { @Test @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + @SuppressWarnings("rawtypes") public void testBasicCombineEmpty() { runTestBasicCombine(EMPTY_TABLE, ImmutableSet.of(), new KV[] { }); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CountTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CountTest.java index 05375bd7c5364..256b06198bc30 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CountTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CountTest.java @@ -46,6 +46,7 @@ public class CountTest { @Test @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + @SuppressWarnings("unchecked") public void testCountPerElementBasic() { Pipeline p = TestPipeline.create(); @@ -67,6 +68,7 @@ public void testCountPerElementBasic() { @Test @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + @SuppressWarnings("unchecked") public void testCountPerElementEmpty() { Pipeline p = TestPipeline.create(); @@ -75,8 +77,7 @@ public void testCountPerElementEmpty() { PCollection> output = input.apply(Count.perElement()); - DataflowAssert.that(output) - .containsInAnyOrder(); + DataflowAssert.that(output).containsInAnyOrder(); p.run(); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CreateTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CreateTest.java index 2d2615c8a5eaf..3875ac465b331 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CreateTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CreateTest.java @@ -47,7 +47,7 @@ * Tests for Create. */ @RunWith(JUnit4.class) -@SuppressWarnings("serial") +@SuppressWarnings({"serial", "unchecked"}) public class CreateTest { @Rule public final ExpectedException thrown = ExpectedException.none(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java index f20e740c84c3f..0ea40ae263ce9 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java @@ -53,7 +53,7 @@ * Tests for GroupByKey. */ @RunWith(JUnit4.class) -@SuppressWarnings("serial") +@SuppressWarnings({"rawtypes", "serial", "unchecked"}) public class GroupByKeyTest { @Rule diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KeysTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KeysTest.java index 1d6e233adef85..6763793d490f1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KeysTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KeysTest.java @@ -37,6 +37,7 @@ */ @RunWith(JUnit4.class) public class KeysTest { + @SuppressWarnings({"rawtypes", "unchecked"}) static final KV[] TABLE = new KV[] { KV.of("one", 1), KV.of("two", 2), @@ -45,6 +46,7 @@ public class KeysTest { KV.of("dup", 5) }; + @SuppressWarnings({"rawtypes", "unchecked"}) static final KV[] EMPTY_TABLE = new KV[] { }; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KvSwapTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KvSwapTest.java index 15c2ff2ff7366..d192b39fcd6ca 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KvSwapTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/KvSwapTest.java @@ -36,6 +36,7 @@ * Tests for KvSwap transform. */ @RunWith(JUnit4.class) +@SuppressWarnings({"rawtypes", "unchecked"}) public class KvSwapTest { static final KV[] TABLE = new KV[] { KV.of("one", 1), diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java index 163426b1c4c85..c5e699216707b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java @@ -868,7 +868,7 @@ public void testMultiOutputChaining() { @Test public void testJsonEscaping() { // Declare an arbitrary function and make sure we can serialize it - DoFn doFn = new DoFn() { + DoFn doFn = new DoFn() { @Override public void processElement(ProcessContext c) { c.output(c.element() + 1); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java index 95f88489dbd54..9eaeb9a0e3fac 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java @@ -49,6 +49,7 @@ public class SampleTest { * Verifies that the result of a Sample operation contains the expected number of elements, * and that those elements are a subset of the items in expected. */ + @SuppressWarnings("rawtypes") public static class VerifyCorrectSample implements SerializableFunction, Void> { private T[] expectedValues; @@ -58,12 +59,14 @@ public static class VerifyCorrectSample * expectedSize is the number of elements that the Sample should contain. expected is the set * of elements that the sample may contain. */ + @SafeVarargs VerifyCorrectSample(int expectedSize, T... expected) { this.expectedValues = expected; this.expectedSize = expectedSize; } @Override + @SuppressWarnings("unchecked") public Void apply(Iterable in) { List actual = new ArrayList<>(); for (T elem : in) { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SimpleStatsFnsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SimpleStatsFnsTest.java index 909dcba9981fd..131aaad3142e7 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SimpleStatsFnsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SimpleStatsFnsTest.java @@ -47,6 +47,8 @@ private static class TestCase> { final N sum; final Double mean; + @SafeVarargs + @SuppressWarnings("all") public TestCase(N min, N max, N sum, N... values) { this.data = Arrays.asList(values); this.min = min; @@ -115,6 +117,7 @@ public void testLongStats() { } @Test + @SuppressWarnings("unchecked") public void testMeanCountSumSerializable() { Pipeline p = TestPipeline.create(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java index fced952d90670..92dc8a90c1dff 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java @@ -59,7 +59,7 @@ public class TopTest { static final String[] EMPTY_COLLECTION = new String[] { }; - @SuppressWarnings("unchecked") + @SuppressWarnings({"rawtypes", "unchecked"}) static final KV[] TABLE = new KV[] { KV.of("a", 1), KV.of("a", 2), @@ -70,7 +70,7 @@ public class TopTest { KV.of("b", 100), }; - @SuppressWarnings("unchecked") + @SuppressWarnings({"rawtypes", "unchecked"}) static final KV[] EMPTY_TABLE = new KV[] { }; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ValuesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ValuesTest.java index 497d8fc8406e0..0dde1f92cd48d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ValuesTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ValuesTest.java @@ -37,6 +37,7 @@ */ @RunWith(JUnit4.class) public class ValuesTest { + @SuppressWarnings({"rawtypes", "unchecked"}) static final KV[] TABLE = new KV[] { KV.of("one", 1), KV.of("two", 2), @@ -45,6 +46,7 @@ public class ValuesTest { KV.of("dup", 4) }; + @SuppressWarnings({"rawtypes", "unchecked"}) static final KV[] EMPTY_TABLE = new KV[] { }; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoderTest.java index 24e6dde65c4fe..8144851f5cbb6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoderTest.java @@ -42,7 +42,7 @@ public void testSerializationDeserialization() { UnionCoder.of(Arrays.>asList(StringUtf8Coder.of(), DoubleCoder.of())); CloudObject encoding = newCoder.asCloudObject(); - Coder decodedCoder = Serializer.deserialize(encoding, Coder.class); + Coder decodedCoder = Serializer.deserialize(encoding, Coder.class); assertEquals(newCoder, decodedCoder); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java index e9041c860cee7..9e05a7eb26c4b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java @@ -50,7 +50,7 @@ /** Unit tests for bucketing. */ @RunWith(JUnit4.class) -@SuppressWarnings("serial") +@SuppressWarnings({"serial", "unchecked"}) public class WindowingTest implements Serializable { @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java index 8f8dfbb9bafca..9a375678d3e52 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java @@ -53,6 +53,7 @@ public class AggregatorImplTest { private static final String AGGREGATOR_NAME = "aggregator_name"; + @SuppressWarnings("rawtypes") private void testAggregator(List items, SerializableFunction, V> combiner, Counter expectedCounter) { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java index d01fe1008fa39..37c0f8edbff4a 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java @@ -45,6 +45,7 @@ /** Unit tests for {@link GroupAlsoByWindowsDoFn}. */ @RunWith(JUnit4.class) +@SuppressWarnings({"rawtypes", "unchecked"}) public class GroupAlsoByWindowsDoFnTest { ExecutionContext execContext; CounterSet counters; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/InstanceBuilderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/InstanceBuilderTest.java index 18777b2aa3942..2c0eb94123373 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/InstanceBuilderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/InstanceBuilderTest.java @@ -30,6 +30,7 @@ * Tests of InstanceBuilder. */ @RunWith(JUnit4.class) +@SuppressWarnings("rawtypes") public class InstanceBuilderTest { @Rule diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java index 94c44c707d0f3..f5895037e4fcb 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java @@ -47,6 +47,7 @@ /** Unit tests for {@link StreamingGroupAlsoByWindowsDoFn}. */ @RunWith(JUnit4.class) +@SuppressWarnings({"rawtypes", "unchecked"}) public class StreamingGroupAlsoByWindowsDoFnTest { ExecutionContext execContext; CounterSet counters; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTest.java index ff40e0d06f182..249df71f71e18 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTest.java @@ -95,6 +95,7 @@ public void testNameKindAndCloudCounterRepresentation() { cc = flush(c3); assertEquals("c3", cc.getName().getName()); assertEquals("SET", cc.getKind()); + @SuppressWarnings("unchecked") Set s = (Set) cc.getSet(); assertEquals(2, s.size()); assertTrue(s.containsAll(Arrays.asList( diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTestUtils.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTestUtils.java index 9c428476e28f0..6f7157c8b0e96 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTestUtils.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/CounterTestUtils.java @@ -104,13 +104,13 @@ public static Set getDeltaSet(Counter counter) { * they are mutually consistent. This is useful for testing coder * implementations. */ - public static void testByteCount(Coder coder, Coder.Context context, Object[] elements) + public static void testByteCount(Coder coder, Coder.Context context, T[] elements) throws Exception { Counter meanByteCount = Counter.longs("meanByteCount", MEAN); ElementByteSizeObserver observer = new ElementByteSizeObserver(meanByteCount); ByteArrayOutputStream os = new ByteArrayOutputStream(); - for (Object elem : elements) { + for (T elem : elements) { coder.registerByteSizeObserver(elem, observer, context); coder.encode(elem, os, context); observer.advance(); @@ -118,6 +118,6 @@ public static void testByteCount(Coder coder, Coder.Context context, Object[] el long expectedLength = os.toByteArray().length; Assert.assertEquals(expectedLength, (long) getTotalAggregate(meanByteCount)); - Assert.assertEquals(elements.length, (long) getTotalCount(meanByteCount)); + Assert.assertEquals(elements.length, getTotalCount(meanByteCount)); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ExecutorTestUtils.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ExecutorTestUtils.java index 0c678abe75d55..07b3bdde5dc26 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ExecutorTestUtils.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ExecutorTestUtils.java @@ -34,6 +34,7 @@ /** * Utilities for tests. */ +@SuppressWarnings({"rawtypes", "unchecked"}) public class ExecutorTestUtils { // Do not instantiate. private ExecutorTestUtils() { } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperationTest.java index d0f8e747de7eb..84fe39a6d9212 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperationTest.java @@ -32,6 +32,7 @@ * Tests for FlattenOperation. */ @RunWith(JUnit4.class) +@SuppressWarnings("unchecked") public class FlattenOperationTest { @Test public void testRunFlattenOperation() throws Exception { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java index 27017962ccc5f..fa46e2f235b71 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java @@ -149,6 +149,7 @@ public void testExecuteMapTaskExecutor() throws Exception { } @Test + @SuppressWarnings("unchecked") public void testGetOutputCounters() throws Exception { CounterSet counters = new CounterSet(); String counterPrefix = "test-"; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiverTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiverTest.java index 08955ac564d7c..044674e7e9318 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiverTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/OutputReceiverTest.java @@ -44,6 +44,7 @@ public TestOutputReceiver() { this(new CounterSet()); } + @SuppressWarnings("rawtypes") public TestOutputReceiver(CounterSet counters) { super("output_name", new ElementByteSizeObservableCoder(StringUtf8Coder.of()), @@ -72,7 +73,7 @@ public void testEmptyOutputReceiver() throws Exception { (long) CounterTestUtils.getTotalAggregate(fanOut.getMeanByteCount())); Assert.assertEquals( 2, - (long) CounterTestUtils.getTotalCount(fanOut.getMeanByteCount())); + CounterTestUtils.getTotalCount(fanOut.getMeanByteCount())); } @Test @@ -100,7 +101,7 @@ public void testMultipleOutputReceiver() throws Exception { (long) CounterTestUtils.getTotalAggregate(fanOut.getMeanByteCount())); Assert.assertEquals( 2, - (long) CounterTestUtils.getTotalCount(fanOut.getMeanByteCount())); + CounterTestUtils.getTotalCount(fanOut.getMeanByteCount())); Assert.assertThat(receiver1.outputElems, CoreMatchers.hasItems("hi", "bob")); Assert.assertThat(receiver2.outputElems, diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperationTest.java index b08266cbb4d82..1c95766e73c01 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperationTest.java @@ -32,6 +32,7 @@ * Tests for ParDoOperation. */ @RunWith(JUnit4.class) +@SuppressWarnings("unchecked") public class ParDoOperationTest { static class TestParDoFn extends ParDoFn { final OutputReceiver outputReceiver; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperationTest.java index 620ac0c89894f..38aa5a35b3057 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperationTest.java @@ -63,6 +63,7 @@ * Tests for PartialGroupByKeyOperation. */ @RunWith(JUnit4.class) +@SuppressWarnings({"rawtypes", "unchecked"}) public class PartialGroupByKeyOperationTest { @Test public void testRunPartialGroupByKeyOperation() throws Exception { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java index d1a69e7a10cf0..c324a0bfe0010 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java @@ -139,6 +139,7 @@ public void process(Object outputElem) throws Exception { } @Test + @SuppressWarnings("unchecked") public void testRunReadOperation() throws Exception { TestSource source = new TestSource(); source.addInput("hi", "there", "", "bob"); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java index d350db1798bfe..291efe14a5546 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java @@ -50,11 +50,13 @@ public void basicTest() throws InterruptedException { try (StateSampler.ScopedState s1 = stateSampler.scopedState(state1)) { + assert s1 != null; Thread.sleep(2 * periodMs); } try (StateSampler.ScopedState s2 = stateSampler.scopedState(state2)) { + assert s2 != null; Thread.sleep(3 * periodMs); } @@ -85,14 +87,17 @@ public void nestingTest() throws InterruptedException { try (StateSampler.ScopedState s1 = stateSampler.scopedState(state1)) { + assert s1 != null; Thread.sleep(2 * periodMs); try (StateSampler.ScopedState s2 = stateSampler.scopedState(state2)) { + assert s2 != null; Thread.sleep(2 * periodMs); try (StateSampler.ScopedState s3 = stateSampler.scopedState(state3)) { + assert s3 != null; Thread.sleep(2 * periodMs); } Thread.sleep(periodMs); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperationTest.java index 6b51bc603531f..4100880427f2b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperationTest.java @@ -33,6 +33,7 @@ @RunWith(JUnit4.class) public class WriteOperationTest { @Test + @SuppressWarnings("unchecked") public void testRunWriteOperation() throws Exception { ExecutorTestUtils.TestSink sink = new ExecutorTestUtils.TestSink(); CounterSet counterSet = new CounterSet(); From 78297492844d7dffb0d7813ec856d231c826d8e0 Mon Sep 17 00:00:00 2001 From: peihe Date: Fri, 19 Dec 2014 16:22:16 -0800 Subject: [PATCH 0050/1541] Fix equals in WindowedValue.java. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82551379 --- .../com/google/cloud/dataflow/sdk/util/WindowedValue.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java index d47e810fa9909..67daaf0de246a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java @@ -141,7 +141,9 @@ public static WindowedValueCoder getValueOnlyCoder(Coder valueCoder) { public boolean equals(Object o) { if (o instanceof WindowedValue) { WindowedValue that = (WindowedValue) o; - if (that.timestamp.isEqual(timestamp) && that.windows.size() == windows.size()) { + if (Objects.equals(that.value, this.value) + && that.timestamp.isEqual(timestamp) + && that.windows.size() == windows.size()) { for (Iterator thatIterator = that.windows.iterator(), thisIterator = windows.iterator(); thatIterator.hasNext() && thisIterator.hasNext(); /* do nothing */) { From 44c39792718cec7a4c07008dc51f0355af336cc7 Mon Sep 17 00:00:00 2001 From: lcwik Date: Fri, 19 Dec 2014 16:38:23 -0800 Subject: [PATCH 0051/1541] Remove java.util.logging usage, SDK code is meant to use SLF4J as logging API. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82552388 --- .../cloud/dataflow/sdk/runners/worker/BigQuerySource.java | 4 ---- .../com/google/cloud/dataflow/sdk/util/common/Counter.java | 3 --- 2 files changed, 7 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySource.java index b43c942b3ed98..fa1afddf87a7f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySource.java @@ -28,7 +28,6 @@ import java.io.IOException; import java.util.NoSuchElementException; -import java.util.logging.Logger; /** * A source that reads a BigQuery table and yields TableRow objects. @@ -39,9 +38,6 @@ * read by each worker (i.e. the source is used as a side input). */ public class BigQuerySource extends Source { - private static final Logger LOG = - Logger.getLogger(BigQuerySource.class.getName()); - final TableReference tableRef; final BigQueryOptions bigQueryOptions; final Bigquery bigQueryClient; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Counter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Counter.java index 4516edd5dbdfe..1df87a244cb61 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Counter.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/Counter.java @@ -26,7 +26,6 @@ import java.util.HashSet; import java.util.Objects; import java.util.Set; -import java.util.logging.Logger; /** * A Counter enables the aggregation of a stream of values over time. The @@ -40,8 +39,6 @@ * @param the type of values aggregated by this counter */ public abstract class Counter { - private static final Logger LOG = Logger.getLogger(Counter.class.getName()); - /** * Possible kinds of counter aggregation. */ From 12addd04cb041058f77bb9a668fa7b6e6d258f5e Mon Sep 17 00:00:00 2001 From: peihe Date: Fri, 19 Dec 2014 16:40:36 -0800 Subject: [PATCH 0052/1541] Fix the Count example in java doc. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82552517 --- .../java/com/google/cloud/dataflow/sdk/transforms/Count.java | 2 +- .../google/cloud/dataflow/sdk/transforms/windowing/Window.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java index 49251dfeca210..0550b2fff5f40 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java @@ -46,7 +46,7 @@ *
 {@code
  * PCollection words = ...;
  * PCollection> wordCounts =
- *     words.apply(Count.create());
+ *     words.apply(Count.perElement());
  * } 
*/ public class Count { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java index 424fdd7c02120..71e8b0be3e36d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java @@ -70,7 +70,7 @@ * PCollection windowed_items = item.apply( * Window.into(FixedWindows.of(1, TimeUnit.MINUTES))); * PCollection> windowed_counts = windowed_items.apply( - * Count.create()); + * Count.perElement()); * } * *

Let (data, timestamp) denote a data element along with its timestamp. From 7c27fb60aaa97061dacc4d33ad8db65066a73db5 Mon Sep 17 00:00:00 2001 From: kirpichov Date: Fri, 19 Dec 2014 21:23:11 -0800 Subject: [PATCH 0053/1541] Renames Source -> Reader (ergo in associated classes such as SourceIterator, SourceFactory, tests, local variables etc.). This separates the concepts of "Source" (description of an input) and Reader (a way to read it) and avoids conflicts with the Source protobuf. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82563808 --- .../google/cloud/dataflow/sdk/io/AvroIO.java | 112 +++--- .../cloud/dataflow/sdk/io/BigQueryIO.java | 133 +++---- .../google/cloud/dataflow/sdk/io/TextIO.java | 97 +++-- ...vroByteSource.java => AvroByteReader.java} | 26 +- .../{AvroSource.java => AvroReader.java} | 52 ++- ...rceFactory.java => AvroReaderFactory.java} | 25 +- ...igQuerySource.java => BigQueryReader.java} | 26 +- ...actory.java => BigQueryReaderFactory.java} | 11 +- .../worker/DataflowWorkProgressUpdater.java | 24 +- .../sdk/runners/worker/DataflowWorker.java | 70 ++-- ...eBasedSource.java => FileBasedReader.java} | 74 ++-- ...Source.java => GroupingShuffleReader.java} | 119 +++--- ...java => GroupingShuffleReaderFactory.java} | 30 +- ...nMemorySource.java => InMemoryReader.java} | 46 +-- ...actory.java => InMemoryReaderFactory.java} | 24 +- ...ator.java => LazyMultiReaderIterator.java} | 20 +- .../worker/MapTaskExecutorFactory.java | 217 ++++------- ...ce.java => PartitioningShuffleReader.java} | 40 +- ... => PartitioningShuffleReaderFactory.java} | 23 +- ...{SourceFactory.java => ReaderFactory.java} | 62 ++-- .../sdk/runners/worker/ShuffleSink.java | 45 +-- .../sdk/runners/worker/SideInputUtils.java | 51 +-- .../worker/SourceTranslationUtils.java | 112 +++--- .../{TextSource.java => TextReader.java} | 114 +++--- ...rceFactory.java => TextReaderFactory.java} | 35 +- ...ource.java => UngroupedShuffleReader.java} | 27 +- ...ava => UngroupedShuffleReaderFactory.java} | 27 +- .../dataflow/sdk/util/CloudSourceUtils.java | 36 +- .../cloud/dataflow/sdk/util/ReaderUtils.java | 44 +++ .../common/worker/CustomSourceFormat.java | 16 +- .../util/common/worker/MapTaskExecutor.java | 23 +- .../sdk/util/common/worker/ReadOperation.java | 58 +-- .../worker/{Source.java => Reader.java} | 45 ++- .../sdk/util/common/worker/WorkExecutor.java | 9 +- .../common/worker/WorkProgressUpdater.java | 45 ++- .../dataflow/sdk/values/PCollection.java | 8 +- ...ourceTest.java => AvroByteReaderTest.java} | 55 ++- .../sdk/runners/worker/AvroByteSinkTest.java | 10 +- .../runners/worker/AvroReaderFactoryTest.java | 107 ++++++ ...vroSourceTest.java => AvroReaderTest.java} | 60 ++- .../sdk/runners/worker/AvroSinkTest.java | 10 +- .../runners/worker/AvroSourceFactoryTest.java | 116 ------ ...st.java => BigQueryReaderFactoryTest.java} | 46 +-- ...ourceTest.java => BigQueryReaderTest.java} | 80 ++-- .../DataflowWorkProgressUpdaterTest.java | 135 +++---- ...st.java => GroupingShuffleReaderTest.java} | 243 ++++++------ ...st.java => InMemoryReaderFactoryTest.java} | 67 ++-- .../runners/worker/InMemoryReaderTest.java | 190 ++++++++++ .../runners/worker/InMemorySourceTest.java | 236 ------------ .../worker/MapTaskExecutorFactoryTest.java | 308 ++++++---------- ...ava => PartitioningShuffleReaderTest.java} | 43 +-- ...actoryTest.java => ReaderFactoryTest.java} | 72 ++-- .../worker/ShuffleReaderFactoryTest.java | 185 ++++++++++ .../worker/ShuffleSourceFactoryTest.java | 231 ------------ .../runners/worker/SideInputUtilsTest.java | 63 ++-- ...ryTest.java => TextReaderFactoryTest.java} | 53 ++- ...extSourceTest.java => TextReaderTest.java} | 346 ++++++++---------- ...t.java => UngroupedShuffleReaderTest.java} | 30 +- .../dataflow/sdk/util/IOFactoryTest.java | 13 +- .../util/common/worker/ExecutorTestUtils.java | 98 ++--- .../common/worker/MapTaskExecutorTest.java | 181 ++++----- .../util/common/worker/ReadOperationTest.java | 118 +++--- 62 files changed, 2133 insertions(+), 2889 deletions(-) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{AvroByteSource.java => AvroByteReader.java} (76%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{AvroSource.java => AvroReader.java} (76%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{AvroSourceFactory.java => AvroReaderFactory.java} (70%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{BigQuerySource.java => BigQueryReader.java} (81%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{BigQuerySourceFactory.java => BigQueryReaderFactory.java} (86%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{FileBasedSource.java => FileBasedReader.java} (77%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{GroupingShuffleSource.java => GroupingShuffleReader.java} (75%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{GroupingShuffleSourceFactory.java => GroupingShuffleReaderFactory.java} (68%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{InMemorySource.java => InMemoryReader.java} (76%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{InMemorySourceFactory.java => InMemoryReaderFactory.java} (64%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{LazyMultiSourceIterator.java => LazyMultiReaderIterator.java} (78%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{PartitioningShuffleSource.java => PartitioningShuffleReader.java} (75%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{PartitioningShuffleSourceFactory.java => PartitioningShuffleReaderFactory.java} (65%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{SourceFactory.java => ReaderFactory.java} (59%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{TextSource.java => TextReader.java} (73%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{TextSourceFactory.java => TextReaderFactory.java} (64%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{UngroupedShuffleSource.java => UngroupedShuffleReader.java} (75%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{UngroupedShuffleSourceFactory.java => UngroupedShuffleReaderFactory.java} (70%) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReaderUtils.java rename sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/{Source.java => Reader.java} (78%) rename sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/{AvroByteSourceTest.java => AvroByteReaderTest.java} (74%) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactoryTest.java rename sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/{AvroSourceTest.java => AvroReaderTest.java} (73%) delete mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactoryTest.java rename sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/{BigQuerySourceFactoryTest.java => BigQueryReaderFactoryTest.java} (53%) rename sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/{BigQuerySourceTest.java => BigQueryReaderTest.java} (69%) rename sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/{GroupingShuffleSourceTest.java => GroupingShuffleReaderTest.java} (68%) rename sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/{InMemorySourceFactoryTest.java => InMemoryReaderFactoryTest.java} (50%) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java delete mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceTest.java rename sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/{PartitioningShuffleSourceTest.java => PartitioningShuffleReaderTest.java} (78%) rename sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/{SourceFactoryTest.java => ReaderFactoryTest.java} (52%) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleReaderFactoryTest.java delete mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSourceFactoryTest.java rename sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/{TextSourceFactoryTest.java => TextReaderFactoryTest.java} (56%) rename sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/{TextSourceTest.java => TextReaderTest.java} (57%) rename sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/{UngroupedShuffleSourceTest.java => UngroupedShuffleReaderTest.java} (81%) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java index 7a9e6ea3d394c..ed8be3a761567 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java @@ -17,17 +17,17 @@ package com.google.cloud.dataflow.sdk.io; import static com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.ValueWithMetadata; -import static com.google.cloud.dataflow.sdk.util.CloudSourceUtils.readElemsFromSource; import com.google.api.client.util.Preconditions; import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.worker.AvroReader; import com.google.cloud.dataflow.sdk.runners.worker.AvroSink; -import com.google.cloud.dataflow.sdk.runners.worker.AvroSource; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.util.ReaderUtils; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.common.worker.Sink; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -116,14 +116,12 @@ * } */ public class AvroIO { - /** * A root PTransform that reads from an Avro file (or multiple Avro * files matching a pattern) and returns a PCollection containing * the decoding of each record. */ public static class Read { - /** * Returns an AvroIO.Read PTransform with the given step name. */ @@ -181,16 +179,17 @@ public static Bound withSchema(String schema) { * @param the type of each of the elements of the resulting * PCollection */ - public static class Bound - extends PTransform> { + public static class Bound extends PTransform> { private static final long serialVersionUID = 0; /** The filepattern to read from. */ - @Nullable final String filepattern; + @Nullable + final String filepattern; /** The class type of the records. */ final Class type; /** The schema of the input file. */ - @Nullable final Schema schema; + @Nullable + final Schema schema; Bound(Class type) { this(null, null, type, null); @@ -258,8 +257,7 @@ public PCollection apply(PInput input) { "need to set the filepattern of an AvroIO.Read transform"); } if (schema == null) { - throw new IllegalStateException( - "need to set the schema of an AvroIO.Read transform"); + throw new IllegalStateException("need to set the schema of an AvroIO.Read transform"); } // Force the output's Coder to be what the read is using, and @@ -275,7 +273,9 @@ protected Coder getDefaultOutputCoder() { } @Override - protected String getKindString() { return "AvroIO.Read"; } + protected String getKindString() { + return "AvroIO.Read"; + } public String getFilepattern() { return filepattern; @@ -287,12 +287,10 @@ public Schema getSchema() { static { DirectPipelineRunner.registerDefaultTransformEvaluator( - Bound.class, - new DirectPipelineRunner.TransformEvaluator() { + Bound.class, new DirectPipelineRunner.TransformEvaluator() { @Override public void evaluate( - Bound transform, - DirectPipelineRunner.EvaluationContext context) { + Bound transform, DirectPipelineRunner.EvaluationContext context) { evaluateReadHelper(transform, context); } }); @@ -307,7 +305,6 @@ public void evaluate( * multiple Avro files matching a sharding pattern). */ public static class Write { - /** * Returns an AvroIO.Write PTransform with the given step name. */ @@ -407,12 +404,12 @@ public static Bound withSchema(String schema) { * * @param the type of each of the elements of the input PCollection */ - public static class Bound - extends PTransform, PDone> { + public static class Bound extends PTransform, PDone> { private static final long serialVersionUID = 0; /** The filename to write to. */ - @Nullable final String filenamePrefix; + @Nullable + final String filenamePrefix; /** Suffix to use for each filename. */ final String filenameSuffix; /** Requested number of shards. 0 for automatic. */ @@ -422,15 +419,15 @@ public static class Bound /** The class type of the records. */ final Class type; /** The schema of the output file. */ - @Nullable final Schema schema; + @Nullable + final Schema schema; Bound(Class type) { this(null, null, "", 0, ShardNameTemplate.INDEX_OF_MAX, type, null); } - Bound(String name, String filenamePrefix, String filenameSuffix, - int numShards, String shardTemplate, - Class type, Schema schema) { + Bound(String name, String filenamePrefix, String filenameSuffix, int numShards, + String shardTemplate, Class type, Schema schema) { super(name); this.filenamePrefix = filenamePrefix; this.filenameSuffix = filenameSuffix; @@ -445,8 +442,8 @@ public static class Bound * with the given step name. Does not modify this object. */ public Bound named(String name) { - return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, - type, schema); + return new Bound<>( + name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema); } /** @@ -459,8 +456,8 @@ public Bound named(String name) { */ public Bound to(String filenamePrefix) { validateOutputComponent(filenamePrefix); - return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, - type, schema); + return new Bound<>( + name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema); } /** @@ -473,8 +470,8 @@ public Bound to(String filenamePrefix) { */ public Bound withSuffix(String filenameSuffix) { validateOutputComponent(filenameSuffix); - return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, - type, schema); + return new Bound<>( + name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema); } /** @@ -493,8 +490,8 @@ public Bound withSuffix(String filenameSuffix) { */ public Bound withNumShards(int numShards) { Preconditions.checkArgument(numShards >= 0); - return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, - type, schema); + return new Bound<>( + name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema); } /** @@ -506,8 +503,8 @@ public Bound withNumShards(int numShards) { * @see ShardNameTemplate */ public Bound withShardNameTemplate(String shardTemplate) { - return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, - type, schema); + return new Bound<>( + name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema); } /** @@ -531,9 +528,8 @@ public Bound withoutSharding() { * @param the type of the elements of the input PCollection */ public Bound withSchema(Class type) { - return new Bound<>(name, filenamePrefix, filenameSuffix, - numShards, shardTemplate, - type, ReflectData.get().getSchema(type)); + return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, + ReflectData.get().getSchema(type)); } /** @@ -542,9 +538,8 @@ public Bound withSchema(Class type) { * schema. Does not modify this object. */ public Bound withSchema(Schema schema) { - return new Bound<>(name, filenamePrefix, filenameSuffix, - numShards, shardTemplate, - GenericRecord.class, schema); + return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, + GenericRecord.class, schema); } /** @@ -563,8 +558,7 @@ public PDone apply(PCollection input) { "need to set the filename prefix of an AvroIO.Write transform"); } if (schema == null) { - throw new IllegalStateException( - "need to set the schema of an AvroIO.Write transform"); + throw new IllegalStateException("need to set the schema of an AvroIO.Write transform"); } return new PDone(); @@ -583,7 +577,9 @@ protected Coder getDefaultOutputCoder() { } @Override - protected String getKindString() { return "AvroIO.Write"; } + protected String getKindString() { + return "AvroIO.Write"; + } public String getFilenamePrefix() { return filenamePrefix; @@ -611,12 +607,10 @@ public Schema getSchema() { static { DirectPipelineRunner.registerDefaultTransformEvaluator( - Bound.class, - new DirectPipelineRunner.TransformEvaluator() { + Bound.class, new DirectPipelineRunner.TransformEvaluator() { @Override public void evaluate( - Bound transform, - DirectPipelineRunner.EvaluationContext context) { + Bound transform, DirectPipelineRunner.EvaluationContext context) { evaluateWriteHelper(transform, context); } }); @@ -626,25 +620,22 @@ public void evaluate( // Pattern which matches old-style shard output patterns, which are now // disallowed. - private static final Pattern SHARD_OUTPUT_PATTERN = - Pattern.compile("@([0-9]+|\\*)"); + private static final Pattern SHARD_OUTPUT_PATTERN = Pattern.compile("@([0-9]+|\\*)"); private static void validateOutputComponent(String partialFilePattern) { Preconditions.checkArgument( !SHARD_OUTPUT_PATTERN.matcher(partialFilePattern).find(), "Output name components are not allowed to contain @* or @N patterns: " - + partialFilePattern); + + partialFilePattern); } ///////////////////////////////////////////////////////////////////////////// private static void evaluateReadHelper( - Read.Bound transform, - DirectPipelineRunner.EvaluationContext context) { - AvroSource source = new AvroSource<>( - transform.filepattern, null, null, WindowedValue.getValueOnlyCoder( - transform.getDefaultOutputCoder())); - List> elems = readElemsFromSource(source); + Read.Bound transform, DirectPipelineRunner.EvaluationContext context) { + AvroReader reader = new AvroReader<>(transform.filepattern, null, null, + WindowedValue.getValueOnlyCoder(transform.getDefaultOutputCoder())); + List> elems = ReaderUtils.readElemsFromReader(reader); List> output = new ArrayList<>(); for (WindowedValue elem : elems) { output.add(ValueWithMetadata.of(elem)); @@ -653,8 +644,7 @@ private static void evaluateReadHelper( } private static void evaluateWriteHelper( - Write.Bound transform, - DirectPipelineRunner.EvaluationContext context) { + Write.Bound transform, DirectPipelineRunner.EvaluationContext context) { List> elems = context.getPCollectionWindowedValues(transform.getInput()); int numShards = transform.numShards; if (numShards < 1) { @@ -662,17 +652,15 @@ private static void evaluateWriteHelper( numShards = 1; } AvroSink writer = new AvroSink<>(transform.filenamePrefix, transform.shardTemplate, - transform.filenameSuffix, numShards, - WindowedValue.getValueOnlyCoder( - AvroCoder.of(transform.type, transform.schema))); + transform.filenameSuffix, numShards, + WindowedValue.getValueOnlyCoder(AvroCoder.of(transform.type, transform.schema))); try (Sink.SinkWriter> sink = writer.writer()) { for (WindowedValue elem : elems) { sink.add(elem); } } catch (IOException exn) { throw new RuntimeException( - "unable to write to output file \"" + transform.filenamePrefix + "\"", - exn); + "unable to write to output file \"" + transform.filenamePrefix + "\"", exn); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java index 37e2963035e64..d7ed8c2116a1f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java @@ -27,13 +27,13 @@ import com.google.cloud.dataflow.sdk.options.BigQueryOptions; import com.google.cloud.dataflow.sdk.options.GcpOptions; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; -import com.google.cloud.dataflow.sdk.runners.worker.BigQuerySource; +import com.google.cloud.dataflow.sdk.runners.worker.BigQueryReader; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.util.BigQueryTableInserter; -import com.google.cloud.dataflow.sdk.util.CloudSourceUtils; +import com.google.cloud.dataflow.sdk.util.ReaderUtils; import com.google.cloud.dataflow.sdk.util.Transport; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -125,8 +125,7 @@ public class BigQueryIO { * This regex isn't exact - this allows for patterns that would be rejected by * the service, but this is sufficient for basic parsing of table references. */ - private static final String PROJECT_ID_REGEXP = - "[a-z][-a-z0-9:.]{4,61}[a-z0-9]"; + private static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]{4,61}[a-z0-9]"; /** * Regular expression which matches Dataset IDs. @@ -142,12 +141,11 @@ public class BigQueryIO { * Matches table specifications in the form * "[project_id]:[dataset_id].[table_id]" or "[dataset_id].[table_id]". */ - private static final String DATASET_TABLE_REGEXP = String.format( - "((?%s):)?(?%s)\\.(?

To allow such anonymous {@code *Fn}s to be written + * conveniently, {@code PTransform} is marked as {@code Serializable}, + * and includes dummy {@code writeObject()} and {@code readObject()} + * operations that do not save or restore any state. + * + * @see Applying Transformations + * + * @param the type of the input to this PTransform + * @param the type of the output of this PTransform + */ +public abstract class PTransform + implements Serializable /* See the note above */ { + + /** + * Applies this {@code PTransform} on the given {@code Input}, and returns its + * {@code Output}. + * + *

%s)", - PROJECT_ID_REGEXP, DATASET_REGEXP, TABLE_REGEXP); + private static final String DATASET_TABLE_REGEXP = + String.format("((?%s):)?(?%s)\\.(?
%s)", PROJECT_ID_REGEXP, + DATASET_REGEXP, TABLE_REGEXP); - private static final Pattern TABLE_SPEC = - Pattern.compile(DATASET_TABLE_REGEXP); + private static final Pattern TABLE_SPEC = Pattern.compile(DATASET_TABLE_REGEXP); /** * Parse a table specification in the form @@ -160,15 +158,13 @@ public static TableReference parseTableSpec(String tableSpec) { if (!match.matches()) { throw new IllegalArgumentException( "Table reference is not in [project_id]:[dataset_id].[table_id] " - + "format: " + tableSpec); + + "format: " + tableSpec); } TableReference ref = new TableReference(); ref.setProjectId(match.group("PROJECT")); - return ref - .setDatasetId(match.group("DATASET")) - .setTableId(match.group("TABLE")); + return ref.setDatasetId(match.group("DATASET")).setTableId(match.group("TABLE")); } /** @@ -181,9 +177,7 @@ public static String toTableSpec(TableReference ref) { sb.append(":"); } - sb.append(ref.getDatasetId()) - .append('.') - .append(ref.getTableId()); + sb.append(ref.getDatasetId()).append('.').append(ref.getTableId()); return sb.toString(); } @@ -241,8 +235,7 @@ public static Bound withoutValidation() { * A PTransform that reads from a BigQuery table and returns a bounded * {@code PCollection}. */ - public static class Bound - extends PTransform> { + public static class Bound extends PTransform> { TableReference table; final boolean validate; @@ -292,8 +285,7 @@ public PCollection apply(PInput input) { throw new IllegalStateException( "must set the table reference of a BigQueryIO.Read transform"); } - return PCollection.createPrimitiveOutputInternal( - new GlobalWindow()) + return PCollection.createPrimitiveOutputInternal(new GlobalWindow()) // Force the output's Coder to be what the read is using, and // unchangeable later, to ensure that we read the input in the // format specified by the Read transform. @@ -306,16 +298,16 @@ protected Coder getDefaultOutputCoder() { } @Override - protected String getKindString() { return "BigQueryIO.Read"; } + protected String getKindString() { + return "BigQueryIO.Read"; + } static { DirectPipelineRunner.registerDefaultTransformEvaluator( - Bound.class, - new DirectPipelineRunner.TransformEvaluator() { + Bound.class, new DirectPipelineRunner.TransformEvaluator() { @Override public void evaluate( - Bound transform, - DirectPipelineRunner.EvaluationContext context) { + Bound transform, DirectPipelineRunner.EvaluationContext context) { evaluateReadHelper(transform, context); } }); @@ -370,7 +362,6 @@ public boolean getValidate() { * */ public static class Write { - /** * An enumeration type for the BigQuery create disposition strings publicly * documented as {@code CREATE_NEVER}, and {@code CREATE_IF_NEEDED}. @@ -488,8 +479,7 @@ public static Bound withoutValidation() { * A PTransform that can write either a bounded or unbounded * {@code PCollection}s to a BigQuery table. */ - public static class Bound - extends PTransform, PDone> { + public static class Bound extends PTransform, PDone> { final TableReference table; // Table schema. The schema is required only if the table does not exist. @@ -515,8 +505,7 @@ public Bound() { } Bound(String name, TableReference ref, TableSchema schema, - CreateDisposition createDisposition, - WriteDisposition writeDisposition, + CreateDisposition createDisposition, WriteDisposition writeDisposition, boolean validate) { super(name); this.table = ref; @@ -530,8 +519,7 @@ public Bound() { * Sets the name associated with this transformation. */ public Bound named(String name) { - return new Bound(name, table, schema, createDisposition, - writeDisposition, validate); + return new Bound(name, table, schema, createDisposition, writeDisposition, validate); } /** @@ -547,28 +535,24 @@ public Bound to(String tableSpec) { * Specifies the table to be written to. */ public Bound to(TableReference table) { - return new Bound(name, table, schema, createDisposition, - writeDisposition, validate); + return new Bound(name, table, schema, createDisposition, writeDisposition, validate); } /** * Specifies the table schema, used if the table is created. */ public Bound withSchema(TableSchema schema) { - return new Bound(name, table, schema, createDisposition, - writeDisposition, validate); + return new Bound(name, table, schema, createDisposition, writeDisposition, validate); } /** Specifies options for creating the table. */ public Bound withCreateDisposition(CreateDisposition createDisposition) { - return new Bound(name, table, schema, createDisposition, - writeDisposition, validate); + return new Bound(name, table, schema, createDisposition, writeDisposition, validate); } /** Specifies options for writing the table. */ public Bound withWriteDisposition(WriteDisposition writeDisposition) { - return new Bound(name, table, schema, createDisposition, - writeDisposition, validate); + return new Bound(name, table, schema, createDisposition, writeDisposition, validate); } /** @@ -585,11 +569,9 @@ public PDone apply(PCollection input) { "must set the table reference of a BigQueryIO.Write transform"); } - if (createDisposition == CreateDisposition.CREATE_IF_NEEDED && - schema == null) { - throw new IllegalArgumentException( - "CreateDisposition is CREATE_IF_NEEDED, " - + "however no schema was provided."); + if (createDisposition == CreateDisposition.CREATE_IF_NEEDED && schema == null) { + throw new IllegalArgumentException("CreateDisposition is CREATE_IF_NEEDED, " + + "however no schema was provided."); } // In streaming, BigQuery write is taken care of by StreamWithDeDup transform. @@ -607,16 +589,16 @@ protected Coder getDefaultOutputCoder() { } @Override - protected String getKindString() { return "BigQueryIO.Write"; } + protected String getKindString() { + return "BigQueryIO.Write"; + } static { DirectPipelineRunner.registerDefaultTransformEvaluator( - Bound.class, - new DirectPipelineRunner.TransformEvaluator() { + Bound.class, new DirectPipelineRunner.TransformEvaluator() { @Override public void evaluate( - Bound transform, - DirectPipelineRunner.EvaluationContext context) { + Bound transform, DirectPipelineRunner.EvaluationContext context) { evaluateWriteHelper(transform, context); } }); @@ -654,9 +636,8 @@ public boolean getValidate() { /** * Implementation of DoFn to perform streaming BigQuery write. */ - private static class StreamingWriteFn extends DoFn>, Void> - implements DoFn.RequiresKeyedState { - + private static class StreamingWriteFn + extends DoFn>, Void> implements DoFn.RequiresKeyedState { /** * Class to accumulate BigQuery row data as a list of String. * DoFn implementation must be Serializable, but BigQuery classes, @@ -664,13 +645,11 @@ private static class StreamingWriteFn extends DoFn, Serializable { - /** The list where BigQuery row data is accumulated. */ private final List jsonRows = new ArrayList<>(); /** Iterator of JsonTableRows converts the row in String to TableRow. */ static class JsonTableRowIterator implements Iterator { - private final Iterator iteratorInternal; /** Constructor. */ @@ -730,9 +709,9 @@ void add(TableRow row) { /** The list of tables created so far, so we don't try the creation each time. */ - private static ThreadLocal> createdTables = - new ThreadLocal>() { - @Override protected HashSet initialValue() { + private static ThreadLocal> createdTables = new ThreadLocal>() { + @Override + protected HashSet initialValue() { return new HashSet<>(); } }; @@ -760,10 +739,9 @@ public void startBundle(Context context) { HashSet tables = createdTables.get(); if (!tables.contains(jsonTableSchema)) { try { - TableSchema tableSchema = JSON_FACTORY.fromString( - jsonTableSchema, TableSchema.class); - TableReference tableReference = JSON_FACTORY.fromString( - jsonTableReference, TableReference.class); + TableSchema tableSchema = JSON_FACTORY.fromString(jsonTableSchema, TableSchema.class); + TableReference tableReference = + JSON_FACTORY.fromString(jsonTableReference, TableReference.class); BigQueryTableInserter inserter = new BigQueryTableInserter(client, tableReference); @@ -791,8 +769,8 @@ public void finishBundle(Context context) { Bigquery client = Transport.newBigQueryClient(options).build(); try { - TableReference tableReference = JSON_FACTORY.fromString( - jsonTableReference, TableReference.class); + TableReference tableReference = + JSON_FACTORY.fromString(jsonTableReference, TableReference.class); BigQueryTableInserter inserter = new BigQueryTableInserter(client, tableReference); inserter.insertAll(jsonTableRows.iterator(), uniqueIdsForTableRows.iterator()); @@ -810,8 +788,7 @@ public void finishBundle(Context context) { * a randomUUID is generated only once per bucket of data. The actual unique * id is created by concatenating this randomUUID with a sequential number. */ - private static class TagWithUniqueIds extends DoFn>> { + private static class TagWithUniqueIds extends DoFn>> { private transient String randomUUID; private transient AtomicLong sequenceNo; @@ -828,8 +805,7 @@ public void processElement(ProcessContext context) { ThreadLocalRandom randomGenerator = ThreadLocalRandom.current(); // We output on keys 0-50 to ensure that there's enough batching for // BigQuery. - context.output(KV.of(randomGenerator.nextInt(0, 50), - KV.of(uniqueId, context.element()))); + context.output(KV.of(randomGenerator.nextInt(0, 50), KV.of(uniqueId, context.element()))); } } @@ -839,9 +815,7 @@ public void processElement(ProcessContext context) { * PTransform that performs streaming BigQuery write. To increase consistency, * it leverages BigQuery best effort de-dup mechanism. */ - private static class StreamWithDeDup - extends PTransform, PDone> { - + private static class StreamWithDeDup extends PTransform, PDone> { private final TableReference tableReference; private final TableSchema tableSchema; @@ -851,7 +825,10 @@ private static class StreamWithDeDup this.tableSchema = tableSchema; } - @Override protected Coder getDefaultOutputCoder() { return VoidCoder.of(); } + @Override + protected Coder getDefaultOutputCoder() { + return VoidCoder.of(); + } @Override public PDone apply(PCollection in) { @@ -891,8 +868,7 @@ public PDone apply(PCollection in) { * This loads the entire table into an in-memory PCollection. */ private static void evaluateReadHelper( - Read.Bound transform, - DirectPipelineRunner.EvaluationContext context) { + Read.Bound transform, DirectPipelineRunner.EvaluationContext context) { BigQueryOptions options = context.getPipelineOptions(); Bigquery client = Transport.newBigQueryClient(options).build(); TableReference ref = transform.table; @@ -901,7 +877,7 @@ private static void evaluateReadHelper( } LOG.info("Reading from BigQuery table {}", toTableSpec(ref)); - List elems = CloudSourceUtils.readElemsFromSource(new BigQuerySource(client, ref)); + List elems = ReaderUtils.readElemsFromReader(new BigQueryReader(client, ref)); LOG.info("Number of records read from BigQuery: {}", elems.size()); context.setPCollection(transform.getOutput(), elems); } @@ -913,8 +889,7 @@ private static void evaluateReadHelper( * The table will be created if necessary. */ private static void evaluateWriteHelper( - Write.Bound transform, - DirectPipelineRunner.EvaluationContext context) { + Write.Bound transform, DirectPipelineRunner.EvaluationContext context) { BigQueryOptions options = context.getPipelineOptions(); Bigquery client = Transport.newBigQueryClient(options).build(); TableReference ref = transform.table; @@ -927,8 +902,8 @@ private static void evaluateWriteHelper( try { BigQueryTableInserter inserter = new BigQueryTableInserter(client, ref); - inserter.getOrCreateTable(transform.writeDisposition, - transform.createDisposition, transform.schema); + inserter.getOrCreateTable( + transform.writeDisposition, transform.createDisposition, transform.schema); List tableRows = context.getPCollection(transform.getInput()); inserter.insertAll(tableRows.iterator()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java index c0e97ff7527b8..86512be491927 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java @@ -16,17 +16,16 @@ package com.google.cloud.dataflow.sdk.io; -import static com.google.cloud.dataflow.sdk.util.CloudSourceUtils.readElemsFromSource; - import com.google.api.client.util.Preconditions; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.worker.TextReader; import com.google.cloud.dataflow.sdk.runners.worker.TextSink; -import com.google.cloud.dataflow.sdk.runners.worker.TextSource; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.util.ReaderUtils; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.common.worker.Sink; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -160,15 +159,16 @@ public static Bound withoutValidation() { * @param the type of each of the elements of the resulting * PCollection, decoded from the lines of the text file */ - public static class Bound - extends PTransform> { + public static class Bound extends PTransform> { private static final long serialVersionUID = 0; /** The filepattern to read from. */ - @Nullable final String filepattern; + @Nullable + final String filepattern; /** The Coder to use to decode each line. */ - @Nullable final Coder coder; + @Nullable + final Coder coder; /** An option to indicate if input validation is desired. Default is true. */ final boolean validate; @@ -231,14 +231,12 @@ public Bound withoutValidation() { @Override public PCollection apply(PInput input) { if (filepattern == null) { - throw new IllegalStateException( - "need to set the filepattern of a TextIO.Read transform"); + throw new IllegalStateException("need to set the filepattern of a TextIO.Read transform"); } // Force the output's Coder to be what the read is using, and // unchangeable later, to ensure that we read the input in the // format specified by the Read transform. - return PCollection.createPrimitiveOutputInternal(new GlobalWindow()) - .setCoder(coder); + return PCollection.createPrimitiveOutputInternal(new GlobalWindow()).setCoder(coder); } @Override @@ -247,7 +245,9 @@ protected Coder getDefaultOutputCoder() { } @Override - protected String getKindString() { return "TextIO.Read"; } + protected String getKindString() { + return "TextIO.Read"; + } public String getFilepattern() { return filepattern; @@ -259,12 +259,10 @@ public boolean needsValidation() { static { DirectPipelineRunner.registerDefaultTransformEvaluator( - Bound.class, - new DirectPipelineRunner.TransformEvaluator() { + Bound.class, new DirectPipelineRunner.TransformEvaluator() { @Override public void evaluate( - Bound transform, - DirectPipelineRunner.EvaluationContext context) { + Bound transform, DirectPipelineRunner.EvaluationContext context) { evaluateReadHelper(transform, context); } }); @@ -332,8 +330,7 @@ public static Bound withNumShards(int numShards) { *

See {@link ShardNameTemplate} for a description of shard templates. */ public static Bound withShardNameTemplate(String shardTemplate) { - return new Bound<>(DEFAULT_TEXT_CODER) - .withShardNameTemplate(shardTemplate); + return new Bound<>(DEFAULT_TEXT_CODER).withShardNameTemplate(shardTemplate); } /** @@ -367,12 +364,12 @@ public static Bound withCoder(Coder coder) { * * @param the type of the elements of the input PCollection */ - public static class Bound - extends PTransform, PDone> { + public static class Bound extends PTransform, PDone> { private static final long serialVersionUID = 0; /** The filename to write to. */ - @Nullable final String filenamePrefix; + @Nullable + final String filenamePrefix; /** Suffix to use for each filename. */ final String filenameSuffix; @@ -389,9 +386,8 @@ public static class Bound this(null, null, "", coder, 0, ShardNameTemplate.INDEX_OF_MAX); } - Bound(String name, String filenamePrefix, String filenameSuffix, - Coder coder, int numShards, - String shardTemplate) { + Bound(String name, String filenamePrefix, String filenameSuffix, Coder coder, + int numShards, String shardTemplate) { super(name); this.coder = coder; this.filenamePrefix = filenamePrefix; @@ -405,8 +401,7 @@ public static class Bound * with the given step name. Does not modify this object. */ public Bound named(String name) { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, - shardTemplate); + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate); } /** @@ -419,8 +414,7 @@ public Bound named(String name) { */ public Bound to(String filenamePrefix) { validateOutputComponent(filenamePrefix); - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, - shardTemplate); + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate); } /** @@ -433,8 +427,7 @@ public Bound to(String filenamePrefix) { */ public Bound withSuffix(String nameExtension) { validateOutputComponent(nameExtension); - return new Bound<>(name, filenamePrefix, nameExtension, coder, numShards, - shardTemplate); + return new Bound<>(name, filenamePrefix, nameExtension, coder, numShards, shardTemplate); } /** @@ -453,8 +446,7 @@ public Bound withSuffix(String nameExtension) { */ public Bound withNumShards(int numShards) { Preconditions.checkArgument(numShards >= 0); - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, - shardTemplate); + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate); } /** @@ -466,8 +458,7 @@ public Bound withNumShards(int numShards) { * @see ShardNameTemplate */ public Bound withShardNameTemplate(String shardTemplate) { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, - shardTemplate); + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate); } /** @@ -492,8 +483,7 @@ public Bound withoutSharding() { * @param the type of the elements of the input PCollection */ public Bound withCoder(Coder coder) { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, - shardTemplate); + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate); } @Override @@ -518,7 +508,9 @@ protected Coder getDefaultOutputCoder() { } @Override - protected String getKindString() { return "TextIO.Write"; } + protected String getKindString() { + return "TextIO.Write"; + } public String getFilenamePrefix() { return filenamePrefix; @@ -542,12 +534,10 @@ public Coder getCoder() { static { DirectPipelineRunner.registerDefaultTransformEvaluator( - Bound.class, - new DirectPipelineRunner.TransformEvaluator() { + Bound.class, new DirectPipelineRunner.TransformEvaluator() { @Override public void evaluate( - Bound transform, - DirectPipelineRunner.EvaluationContext context) { + Bound transform, DirectPipelineRunner.EvaluationContext context) { evaluateWriteHelper(transform, context); } }); @@ -557,30 +547,27 @@ public void evaluate( // Pattern which matches old-style shard output patterns, which are now // disallowed. - private static final Pattern SHARD_OUTPUT_PATTERN = - Pattern.compile("@([0-9]+|\\*)"); + private static final Pattern SHARD_OUTPUT_PATTERN = Pattern.compile("@([0-9]+|\\*)"); private static void validateOutputComponent(String partialFilePattern) { Preconditions.checkArgument( !SHARD_OUTPUT_PATTERN.matcher(partialFilePattern).find(), "Output name components are not allowed to contain @* or @N patterns: " - + partialFilePattern); + + partialFilePattern); } ////////////////////////////////////////////////////////////////////////////// private static void evaluateReadHelper( - Read.Bound transform, - DirectPipelineRunner.EvaluationContext context) { - TextSource source = new TextSource<>( - transform.filepattern, true, null, null, transform.coder); - List elems = readElemsFromSource(source); + Read.Bound transform, DirectPipelineRunner.EvaluationContext context) { + TextReader reader = + new TextReader<>(transform.filepattern, true, null, null, transform.coder); + List elems = ReaderUtils.readElemsFromReader(reader); context.setPCollection(transform.getOutput(), elems); } private static void evaluateWriteHelper( - Write.Bound transform, - DirectPipelineRunner.EvaluationContext context) { + Write.Bound transform, DirectPipelineRunner.EvaluationContext context) { List elems = context.getPCollection(transform.getInput()); int numShards = transform.numShards; if (numShards < 1) { @@ -588,17 +575,15 @@ private static void evaluateWriteHelper( numShards = 1; } TextSink> writer = TextSink.createForDirectPipelineRunner( - transform.filenamePrefix, transform.getShardNameTemplate(), - transform.filenameSuffix, numShards, - true, null, null, transform.coder); + transform.filenamePrefix, transform.getShardNameTemplate(), transform.filenameSuffix, + numShards, true, null, null, transform.coder); try (Sink.SinkWriter> sink = writer.writer()) { for (T elem : elems) { sink.add(WindowedValue.valueInGlobalWindow(elem)); } } catch (IOException exn) { throw new RuntimeException( - "unable to write to output file \"" + transform.filenamePrefix + "\"", - exn); + "unable to write to output file \"" + transform.filenamePrefix + "\"", exn); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteReader.java similarity index 76% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSource.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteReader.java index 5a8524eb88853..a62d20775b7d7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteReader.java @@ -20,7 +20,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.WindowedValue; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.apache.avro.Schema; import org.apache.avro.generic.GenericDatumReader; @@ -38,34 +38,28 @@ * * @param the type of the elements read from the source */ -public class AvroByteSource extends Source { - - final AvroSource avroSource; +public class AvroByteReader extends Reader { + final AvroReader avroReader; final Coder coder; private final Schema schema = Schema.create(Schema.Type.BYTES); - public AvroByteSource(String filename, - @Nullable Long startPosition, - @Nullable Long endPosition, - Coder coder) { + public AvroByteReader( + String filename, @Nullable Long startPosition, @Nullable Long endPosition, Coder coder) { this.coder = coder; - avroSource = new AvroSource<>( - filename, startPosition, endPosition, + avroReader = new AvroReader<>(filename, startPosition, endPosition, WindowedValue.getValueOnlyCoder(AvroCoder.of(ByteBuffer.class, schema))); } @Override - public SourceIterator iterator() throws IOException { + public ReaderIterator iterator() throws IOException { return new AvroByteFileIterator(); } - class AvroByteFileIterator extends AbstractSourceIterator { - - private final SourceIterator> avroFileIterator; + class AvroByteFileIterator extends AbstractReaderIterator { + private final ReaderIterator> avroFileIterator; public AvroByteFileIterator() throws IOException { - avroFileIterator = avroSource.iterator( - new GenericDatumReader(schema)); + avroFileIterator = avroReader.iterator(new GenericDatumReader(schema)); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java similarity index 76% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSource.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java index 3f071cff2c7a1..6fd10b2223c02 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java @@ -16,15 +16,12 @@ package com.google.cloud.dataflow.sdk.runners.worker; -import static com.google.cloud.dataflow.sdk.util.WindowedValue.ValueOnlyWindowedValueCoder; -import static com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; - import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.IOChannelFactory; import com.google.cloud.dataflow.sdk.util.IOChannelUtils; import com.google.cloud.dataflow.sdk.util.WindowedValue; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.apache.avro.Schema; import org.apache.avro.file.DataFileReader; @@ -46,24 +43,24 @@ * * @param the type of the elements read from the source */ -public class AvroSource extends Source> { +public class AvroReader extends Reader> { private static final int BUF_SIZE = 200; final String filename; - @Nullable final Long startPosition; - @Nullable final Long endPosition; + @Nullable + final Long startPosition; + @Nullable + final Long endPosition; final AvroCoder avroCoder; private final Schema schema; - public AvroSource(String filename, - @Nullable Long startPosition, - @Nullable Long endPosition, - WindowedValueCoder coder) { - if (!(coder instanceof ValueOnlyWindowedValueCoder)) { + public AvroReader(String filename, @Nullable Long startPosition, @Nullable Long endPosition, + WindowedValue.WindowedValueCoder coder) { + if (!(coder instanceof WindowedValue.ValueOnlyWindowedValueCoder)) { throw new IllegalArgumentException("Expected ValueOnlyWindowedValueCoder"); } if (!(coder.getValueCoder() instanceof AvroCoder)) { - throw new IllegalArgumentException("AvroSource requires an AvroCoder"); + throw new IllegalArgumentException("AvroReader requires an AvroCoder"); } this.filename = filename; @@ -73,7 +70,7 @@ public AvroSource(String filename, this.schema = this.avroCoder.getSchema(); } - public SourceIterator> iterator(DatumReader datumReader) throws IOException { + public ReaderIterator> iterator(DatumReader datumReader) throws IOException { IOChannelFactory factory = IOChannelUtils.getFactory(filename); Collection inputs = factory.match(filename); @@ -85,46 +82,40 @@ public SourceIterator> iterator(DatumReader datumReader) thr } else { if (startPosition != null || endPosition != null) { throw new UnsupportedOperationException( - "Unable to apply range limits to multiple-input stream: " + - filename); + "Unable to apply range limits to multiple-input stream: " + filename); } return new AvroFileMultiIterator(datumReader, factory, inputs.iterator()); } } @Override - public SourceIterator> iterator() throws IOException { + public ReaderIterator> iterator() throws IOException { return iterator(avroCoder.createDatumReader()); } - class AvroFileMultiIterator extends LazyMultiSourceIterator> { + class AvroFileMultiIterator extends LazyMultiReaderIterator> { private final IOChannelFactory factory; private final DatumReader datumReader; - public AvroFileMultiIterator(DatumReader datumReader, - IOChannelFactory factory, - Iterator inputs) { + public AvroFileMultiIterator( + DatumReader datumReader, IOChannelFactory factory, Iterator inputs) { super(inputs); this.factory = factory; this.datumReader = datumReader; } @Override - protected SourceIterator> open(String input) throws IOException { + protected ReaderIterator> open(String input) throws IOException { return new AvroFileIterator(datumReader, input, factory.open(input), null, null); } } - class AvroFileIterator extends AbstractSourceIterator> { + class AvroFileIterator extends AbstractReaderIterator> { final DataFileReader fileReader; final Long endOffset; - public AvroFileIterator(DatumReader datumReader, - String filename, - ReadableByteChannel reader, - @Nullable Long startOffset, - @Nullable Long endOffset) - throws IOException { + public AvroFileIterator(DatumReader datumReader, String filename, ReadableByteChannel reader, + @Nullable Long startOffset, @Nullable Long endOffset) throws IOException { if (!(reader instanceof SeekableByteChannel)) { throw new UnsupportedOperationException( "Unable to seek to offset in stream for " + filename); @@ -141,8 +132,7 @@ public AvroFileIterator(DatumReader datumReader, @Override public boolean hasNext() throws IOException { - return fileReader.hasNext() - && (endOffset == null || !fileReader.pastSync(endOffset)); + return fileReader.hasNext() && (endOffset == null || !fileReader.pastSync(endOffset)); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactory.java similarity index 70% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactory.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactory.java index 740f94965dd80..c4292c5bcb535 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactory.java @@ -26,41 +26,36 @@ import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; /** - * Creates an AvroSource from a CloudObject spec. + * Creates an AvroReader from a CloudObject spec. */ @SuppressWarnings("rawtypes") -public class AvroSourceFactory { +public class AvroReaderFactory { // Do not instantiate. - private AvroSourceFactory() {} + private AvroReaderFactory() {} - public static Source create(PipelineOptions options, - CloudObject spec, - Coder coder, - ExecutionContext executionContext) - throws Exception { + public static Reader create(PipelineOptions options, CloudObject spec, Coder coder, + ExecutionContext executionContext) throws Exception { return create(spec, coder); } - static Source create(CloudObject spec, - Coder coder) - throws Exception { + static Reader create(CloudObject spec, Coder coder) throws Exception { String filename = getString(spec, PropertyNames.FILENAME); Long startOffset = getLong(spec, PropertyNames.START_OFFSET, null); Long endOffset = getLong(spec, PropertyNames.END_OFFSET, null); if (!(coder instanceof WindowedValueCoder)) { - return new AvroByteSource<>(filename, startOffset, endOffset, coder); + return new AvroByteReader<>(filename, startOffset, endOffset, coder); //throw new IllegalArgumentException("Expected WindowedValueCoder"); } WindowedValueCoder windowedCoder = (WindowedValueCoder) coder; if (windowedCoder.getValueCoder() instanceof AvroCoder) { - return new AvroSource(filename, startOffset, endOffset, windowedCoder); + return new AvroReader(filename, startOffset, endOffset, windowedCoder); } else { - return new AvroByteSource<>(filename, startOffset, endOffset, windowedCoder); + return new AvroByteReader<>(filename, startOffset, endOffset, windowedCoder); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReader.java similarity index 81% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySource.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReader.java index fa1afddf87a7f..3d442f3ffc768 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReader.java @@ -24,7 +24,7 @@ import com.google.cloud.dataflow.sdk.options.BigQueryOptions; import com.google.cloud.dataflow.sdk.util.BigQueryTableRowIterator; import com.google.cloud.dataflow.sdk.util.Transport; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import java.io.IOException; import java.util.NoSuchElementException; @@ -37,13 +37,13 @@ * progress reporting because the source is used only in situations where the entire table must be * read by each worker (i.e. the source is used as a side input). */ -public class BigQuerySource extends Source { +public class BigQueryReader extends Reader { final TableReference tableRef; final BigQueryOptions bigQueryOptions; final Bigquery bigQueryClient; /** Builds a BigQuery source using pipeline options to instantiate a Bigquery client. */ - public BigQuerySource(BigQueryOptions bigQueryOptions, TableReference tableRef) { + public BigQueryReader(BigQueryOptions bigQueryOptions, TableReference tableRef) { // Save pipeline options so that we can construct the BigQuery client on-demand whenever an // iterator gets created. this.bigQueryOptions = bigQueryOptions; @@ -51,31 +51,29 @@ public BigQuerySource(BigQueryOptions bigQueryOptions, TableReference tableRef) this.bigQueryClient = null; } - /** Builds a BigQuerySource directly using a BigQuery client. */ - public BigQuerySource(Bigquery bigQueryClient, TableReference tableRef) { + /** Builds a BigQueryReader directly using a BigQuery client. */ + public BigQueryReader(Bigquery bigQueryClient, TableReference tableRef) { this.bigQueryOptions = null; this.tableRef = tableRef; this.bigQueryClient = bigQueryClient; } @Override - public SourceIterator iterator() throws IOException { - return new BigQuerySourceIterator( + public ReaderIterator iterator() throws IOException { + return new BigQueryReaderIterator( bigQueryClient != null - ? bigQueryClient - : Transport.newBigQueryClient(bigQueryOptions).build(), + ? bigQueryClient : Transport.newBigQueryClient(bigQueryOptions).build(), tableRef); } /** - * A SourceIterator that yields TableRow objects for each row of a BigQuery table. + * A ReaderIterator that yields TableRow objects for each row of a BigQuery table. */ - class BigQuerySourceIterator extends AbstractSourceIterator { - + class BigQueryReaderIterator extends AbstractReaderIterator { private BigQueryTableRowIterator rowIterator; - public BigQuerySourceIterator(Bigquery bigQueryClient, TableReference tableRef) { - rowIterator = new BigQueryTableRowIterator(bigQueryClient, tableRef); + public BigQueryReaderIterator(Bigquery bigQueryClient, TableReference tableRef) { + rowIterator = new BigQueryTableRowIterator(bigQueryClient, tableRef); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderFactory.java similarity index 86% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactory.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderFactory.java index 682b7faa1400b..e2c00ba4064fb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderFactory.java @@ -27,16 +27,15 @@ import com.google.cloud.dataflow.sdk.util.PropertyNames; /** - * Creates a BigQuerySource from a {@link CloudObject} spec. + * Creates a BigQueryReader from a {@link CloudObject} spec. */ -public class BigQuerySourceFactory { +public class BigQueryReaderFactory { // Do not instantiate. - private BigQuerySourceFactory() {} + private BigQueryReaderFactory() {} - public static BigQuerySource create( - PipelineOptions options, CloudObject spec, Coder coder, + public static BigQueryReader create(PipelineOptions options, CloudObject spec, Coder coder, ExecutionContext executionContext) throws Exception { - return new BigQuerySource( + return new BigQueryReader( options.as(BigQueryOptions.class), new TableReference() .setProjectId(getString(spec, PropertyNames.BIGQUERY_PROJECT)) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java index f2569b1488f6e..250eb431ad4a3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java @@ -18,7 +18,7 @@ import static com.google.cloud.dataflow.sdk.runners.worker.DataflowWorker.buildStatus; import static com.google.cloud.dataflow.sdk.runners.worker.DataflowWorker.uniqueId; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudDuration; import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime; import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; @@ -54,11 +54,8 @@ public class DataflowWorkProgressUpdater extends WorkProgressUpdater { /** Options specifying information about the pipeline run by the worker.*/ private final DataflowWorkerHarnessOptions options; - public DataflowWorkProgressUpdater( - WorkItem workItem, - WorkExecutor worker, - DataflowWorker.WorkUnitClient workUnitClient, - DataflowWorkerHarnessOptions options) { + public DataflowWorkProgressUpdater(WorkItem workItem, WorkExecutor worker, + DataflowWorker.WorkUnitClient workUnitClient, DataflowWorkerHarnessOptions options) { super(worker); this.workItem = workItem; this.workUnitClient = workUnitClient; @@ -77,11 +74,9 @@ protected long getWorkUnitLeaseExpirationTimestamp() { @Override protected void reportProgressHelper() throws Exception { - WorkItemStatus status = buildStatus( - workItem, false /*completed*/, - worker.getOutputCounters(), worker.getOutputMetrics(), options, - worker.getWorkerProgress(), stopPositionToService, - null /*sourceOperationResponse*/, null /*errors*/); + WorkItemStatus status = buildStatus(workItem, false/*completed*/, worker.getOutputCounters(), + worker.getOutputMetrics(), options, worker.getWorkerProgress(), stopPositionToService, + null/*sourceOperationResponse*/, null/*errors*/); status.setRequestedLeaseDuration(toCloudDuration(Duration.millis(requestedLeaseDurationMs))); WorkItemServiceState result = workUnitClient.reportWorkItemStatus(status); @@ -95,16 +90,15 @@ protected void reportProgressHelper() throws Exception { ApproximateProgress suggestedStopPoint = result.getSuggestedStopPoint(); if (suggestedStopPoint == null && result.getSuggestedStopPosition() != null) { - suggestedStopPoint = new ApproximateProgress() - .setPosition(result.getSuggestedStopPosition()); + suggestedStopPoint = + new ApproximateProgress().setPosition(result.getSuggestedStopPosition()); } if (suggestedStopPoint != null) { LOG.info("Proposing stop progress on work unit {} at proposed stopping point {}", workString(), suggestedStopPoint); stopPositionToService = - worker.proposeStopPosition( - cloudProgressToSourceProgress(suggestedStopPoint)); + worker.proposeStopPosition(cloudProgressToReaderProgress(suggestedStopPoint)); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java index 5175d15aa882e..499653a11d0a7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java @@ -36,7 +36,7 @@ import com.google.cloud.dataflow.sdk.util.common.CounterSet; import com.google.cloud.dataflow.sdk.util.common.Metric; import com.google.cloud.dataflow.sdk.util.common.worker.CustomSourceFormat; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.cloud.dataflow.sdk.util.common.worker.WorkExecutor; import com.google.cloud.dataflow.sdk.util.common.worker.WorkProgressUpdater; @@ -63,7 +63,6 @@ * execute it, and update the work. */ public class DataflowWorker { - private static final Logger LOG = LoggerFactory.getLogger(DataflowWorker.class); /** @@ -77,8 +76,7 @@ public class DataflowWorker { */ private final DataflowWorkerHarnessOptions options; - public DataflowWorker(WorkUnitClient workUnitClient, - DataflowWorkerHarnessOptions options) { + public DataflowWorker(WorkUnitClient workUnitClient, DataflowWorkerHarnessOptions options) { this.workUnitClient = workUnitClient; this.options = options; } @@ -114,20 +112,17 @@ private boolean doWork(WorkItem workItem) throws IOException { ExecutionContext executionContext = new BatchModeExecutionContext(); if (workItem.getMapTask() != null) { - worker = MapTaskExecutorFactory.create(options, - workItem.getMapTask(), - executionContext); + worker = MapTaskExecutorFactory.create(options, workItem.getMapTask(), executionContext); } else if (workItem.getSourceOperationTask() != null) { - worker = SourceOperationExecutorFactory.create( - workItem.getSourceOperationTask()); + worker = SourceOperationExecutorFactory.create(workItem.getSourceOperationTask()); } else { throw new RuntimeException("unknown kind of work item: " + workItem.toString()); } - WorkProgressUpdater progressUpdater = new DataflowWorkProgressUpdater( - workItem, worker, workUnitClient, options); + WorkProgressUpdater progressUpdater = + new DataflowWorkProgressUpdater(workItem, worker, workUnitClient, options); progressUpdater.startReportingProgress(); // Blocks while executing the work. @@ -155,13 +150,13 @@ private boolean doWork(WorkItem workItem) throws IOException { // TODO: Find out a generic way for the WorkExecutor to report work-specific results // into the work update. - CustomSourceFormat.SourceOperationResponse sourceOperationResponse = + CustomSourceFormat.OperationResponse sourceOperationResponse = (worker instanceof SourceOperationExecutor) ? cloudSourceOperationResponseToSourceOperationResponse( - ((SourceOperationExecutor) worker).getResponse()) + ((SourceOperationExecutor) worker).getResponse()) : null; - reportStatus(options, "Success", workItem, counters, metrics, sourceOperationResponse, - null /*errors*/); + reportStatus( + options, "Success", workItem, counters, metrics, sourceOperationResponse, null/*errors*/); return true; @@ -181,8 +176,7 @@ private boolean doWork(WorkItem workItem) throws IOException { } /** Handles the exception thrown when reading and executing the work. */ - private void handleWorkError( - WorkItem workItem, WorkExecutor worker, Throwable e) + private void handleWorkError(WorkItem workItem, WorkExecutor worker, Throwable e) throws IOException { LOG.warn("Uncaught exception occurred during work unit execution:", e); @@ -190,14 +184,12 @@ private void handleWorkError( // into the client. Throwable t = e instanceof UserCodeException ? e.getCause() : e; Status error = new Status(); - error.setCode(2); // Code.UNKNOWN. TODO: Replace with a generated definition. + error.setCode(2); // Code.UNKNOWN. TODO: Replace with a generated definition. // TODO: Attach the stack trace as exception details, not to the message. error.setMessage(buildCloudStackTrace(t)); - reportStatus(options, "Failure", workItem, - worker == null ? null : worker.getOutputCounters(), - worker == null ? null : worker.getOutputMetrics(), - null /*sourceOperationResponse*/, + reportStatus(options, "Failure", workItem, worker == null ? null : worker.getOutputCounters(), + worker == null ? null : worker.getOutputMetrics(), null/*sourceOperationResponse*/, error == null ? null : Collections.singletonList(error)); } @@ -225,30 +217,21 @@ private static String buildCloudStackTrace(Throwable t) { return result.toString(); } - private void reportStatus(DataflowWorkerHarnessOptions options, - String status, - WorkItem workItem, - @Nullable CounterSet counters, - @Nullable Collection> metrics, - @Nullable CustomSourceFormat.SourceOperationResponse - sourceOperationResponse, - @Nullable List errors) - throws IOException { + private void reportStatus(DataflowWorkerHarnessOptions options, String status, WorkItem workItem, + @Nullable CounterSet counters, @Nullable Collection> metrics, + @Nullable CustomSourceFormat.OperationResponse sourceOperationResponse, + @Nullable List errors) throws IOException { LOG.info("{} processing work item {}", status, uniqueId(workItem)); - WorkItemStatus workItemStatus = buildStatus(workItem, true /*completed*/, - counters, metrics, options, null, null, sourceOperationResponse, errors); + WorkItemStatus workItemStatus = buildStatus(workItem, true/*completed*/, counters, metrics, + options, null, null, sourceOperationResponse, errors); workUnitClient.reportWorkItemStatus(workItemStatus); } - static WorkItemStatus buildStatus( - WorkItem workItem, - boolean completed, - @Nullable CounterSet counters, - @Nullable Collection> metrics, - DataflowWorkerHarnessOptions options, - @Nullable Source.Progress progress, - @Nullable Source.Position stopPosition, - @Nullable CustomSourceFormat.SourceOperationResponse sourceOperationResponse, + static WorkItemStatus buildStatus(WorkItem workItem, boolean completed, + @Nullable CounterSet counters, @Nullable Collection> metrics, + DataflowWorkerHarnessOptions options, @Nullable Reader.Progress progress, + @Nullable Reader.Position stopPosition, + @Nullable CustomSourceFormat.OperationResponse sourceOperationResponse, @Nullable List errors) { WorkItemStatus status = new WorkItemStatus(); status.setWorkItemId(Long.toString(workItem.getId())); @@ -323,8 +306,7 @@ public abstract static class WorkUnitClient { * @param workItemStatus the status to report * @return a {@link WorkServiceState} (e.g. a new stop position) */ - public abstract WorkItemServiceState reportWorkItemStatus( - WorkItemStatus workItemStatus) + public abstract WorkItemServiceState reportWorkItemStatus(WorkItemStatus workItemStatus) throws IOException; } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java similarity index 77% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedSource.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java index 2e152e794f80c..de116277fd262 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java @@ -17,8 +17,8 @@ package com.google.cloud.dataflow.sdk.runners.worker; import static com.google.api.client.util.Preconditions.checkNotNull; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToSourcePosition; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; import com.google.api.services.dataflow.model.ApproximateProgress; @@ -27,7 +27,7 @@ import com.google.cloud.dataflow.sdk.util.IOChannelFactory; import com.google.cloud.dataflow.sdk.util.IOChannelUtils; import com.google.cloud.dataflow.sdk.util.common.worker.ProgressTracker; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,21 +47,20 @@ * * @param the type of the elements read from the source */ -public abstract class FileBasedSource extends Source { +public abstract class FileBasedReader extends Reader { protected static final int BUF_SIZE = 200; protected final String filename; - @Nullable protected final Long startPosition; - @Nullable protected final Long endPosition; + @Nullable + protected final Long startPosition; + @Nullable + protected final Long endPosition; protected final Coder coder; protected final boolean useDefaultBufferSize; - private static final Logger LOG = LoggerFactory.getLogger(FileBasedSource.class); + private static final Logger LOG = LoggerFactory.getLogger(FileBasedReader.class); - protected FileBasedSource(String filename, - @Nullable Long startPosition, - @Nullable Long endPosition, - Coder coder, - boolean useDefaultBufferSize) { + protected FileBasedReader(String filename, @Nullable Long startPosition, + @Nullable Long endPosition, Coder coder, boolean useDefaultBufferSize) { this.filename = filename; this.startPosition = startPosition; this.endPosition = endPosition; @@ -80,20 +79,18 @@ protected FileBasedSource(String filename, * * @param endPosition offset of the end position; null means end-of-file */ - protected abstract SourceIterator newSourceIteratorForRangeInFile( - IOChannelFactory factory, String oneFile, long startPosition, - @Nullable Long endPosition) - throws IOException; + protected abstract ReaderIterator newReaderIteratorForRangeInFile(IOChannelFactory factory, + String oneFile, long startPosition, @Nullable Long endPosition) throws IOException; /** * Returns a new iterator for elements in the given files. Caller * must ensure that the file collection is not empty. */ - protected abstract SourceIterator newSourceIteratorForFiles( + protected abstract ReaderIterator newReaderIteratorForFiles( IOChannelFactory factory, Collection files) throws IOException; @Override - public SourceIterator iterator() throws IOException { + public ReaderIterator iterator() throws IOException { IOChannelFactory factory = IOChannelUtils.getFactory(filename); Collection inputs = factory.match(filename); if (inputs.isEmpty()) { @@ -103,22 +100,20 @@ public SourceIterator iterator() throws IOException { if (startPosition != null || endPosition != null) { if (inputs.size() != 1) { throw new UnsupportedOperationException( - "Unable to apply range limits to multiple-input stream: " - + filename); + "Unable to apply range limits to multiple-input stream: " + filename); } - return newSourceIteratorForRangeInFile( - factory, inputs.iterator().next(), + return newReaderIteratorForRangeInFile(factory, inputs.iterator().next(), startPosition == null ? 0 : startPosition, endPosition); } else { - return newSourceIteratorForFiles(factory, inputs); + return newReaderIteratorForFiles(factory, inputs); } } /** * Abstract base class for file-based source iterators. */ - protected abstract class FileBasedIterator extends AbstractSourceIterator { + protected abstract class FileBasedIterator extends AbstractReaderIterator { protected final CopyableSeekableByteChannel seeker; protected final PushbackInputStream stream; protected final Long startOffset; @@ -128,16 +123,14 @@ protected abstract class FileBasedIterator extends AbstractSourceIterator { protected boolean nextElementComputed = false; protected long offset; - FileBasedIterator(CopyableSeekableByteChannel seeker, - long startOffset, - long offset, - @Nullable Long endOffset, - ProgressTracker tracker) throws IOException { + FileBasedIterator(CopyableSeekableByteChannel seeker, long startOffset, long offset, + @Nullable Long endOffset, ProgressTracker tracker) throws IOException { this.seeker = checkNotNull(seeker); this.seeker.position(startOffset); - BufferedInputStream bufferedStream = useDefaultBufferSize - ? new BufferedInputStream(Channels.newInputStream(seeker)) - : new BufferedInputStream(Channels.newInputStream(seeker), BUF_SIZE); + BufferedInputStream bufferedStream = + useDefaultBufferSize + ? new BufferedInputStream(Channels.newInputStream(seeker)) + : new BufferedInputStream(Channels.newInputStream(seeker), BUF_SIZE); this.stream = new PushbackInputStream(bufferedStream, BUF_SIZE); this.startOffset = startOffset; this.offset = offset; @@ -153,8 +146,7 @@ protected abstract class FileBasedIterator extends AbstractSourceIterator { * has been reached. * @throws IOException if an I/O error occurs */ - protected abstract ByteArrayOutputStream readElement() - throws IOException; + protected abstract ByteArrayOutputStream readElement() throws IOException; @Override public boolean hasNext() throws IOException { @@ -179,7 +171,7 @@ void advance() throws IOException { @Override public Progress getProgress() { // Currently we assume that only a offset position is reported as - // current progress. Source writer can override this method to update + // current progress. An implementor can override this method to update // other metrics, e.g. completion percentage or remaining time. com.google.api.services.dataflow.model.Position currentPosition = new com.google.api.services.dataflow.model.Position(); @@ -188,7 +180,7 @@ public Progress getProgress() { ApproximateProgress progress = new ApproximateProgress(); progress.setPosition(currentPosition); - return cloudProgressToSourceProgress(progress); + return cloudProgressToReaderProgress(progress); } @Override @@ -196,21 +188,19 @@ public Position updateStopPosition(Progress proposedStopPosition) { checkNotNull(proposedStopPosition); // Currently we only support stop position in byte offset of - // CloudPosition in a file-based Source. If stop position in + // CloudPosition in a file-based Reader. If stop position in // other types is proposed, the end position in iterator will // not be updated, and return null. com.google.api.services.dataflow.model.ApproximateProgress stopPosition = sourceProgressToCloudProgress(proposedStopPosition); if (stopPosition == null) { - LOG.warn( - "A stop position other than CloudPosition is not supported now."); + LOG.warn("A stop position other than CloudPosition is not supported now."); return null; } Long byteOffset = stopPosition.getPosition().getByteOffset(); if (byteOffset == null) { - LOG.warn( - "A stop position other than byte offset is not supported in a " + LOG.warn("A stop position other than byte offset is not supported in a " + "file-based Source."); return null; } @@ -227,7 +217,7 @@ public Position updateStopPosition(Progress proposedStopPosition) { } this.endOffset = byteOffset; - return cloudPositionToSourcePosition(stopPosition.getPosition()); + return cloudPositionToReaderPosition(stopPosition.getPosition()); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java similarity index 75% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java index 19ce35800c6ee..7d0f92313e9fd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java @@ -17,12 +17,11 @@ package com.google.cloud.dataflow.sdk.runners.worker; import static com.google.api.client.util.Preconditions.checkNotNull; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToSourcePosition; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; import com.google.api.client.util.Preconditions; - import com.google.api.services.dataflow.model.ApproximateProgress; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.IterableCoder; @@ -37,9 +36,9 @@ import com.google.cloud.dataflow.sdk.util.common.worker.BatchingShuffleEntryReader; import com.google.cloud.dataflow.sdk.util.common.worker.GroupingShuffleEntryIterator; import com.google.cloud.dataflow.sdk.util.common.worker.KeyGroupedShuffleEntries; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntryReader; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; import com.google.cloud.dataflow.sdk.values.KV; import org.slf4j.Logger; @@ -55,10 +54,8 @@ * @param the type of the keys read from the shuffle * @param the type of the values read from the shuffle */ -public class GroupingShuffleSource - extends Source>>> { - private static final Logger LOG = - LoggerFactory.getLogger(GroupingShuffleSource.class); +public class GroupingShuffleReader extends Reader>>> { + private static final Logger LOG = LoggerFactory.getLogger(GroupingShuffleReader.class); final byte[] shuffleReaderConfig; final String startShufflePosition; @@ -68,12 +65,9 @@ public class GroupingShuffleSource Coder keyCoder; Coder valueCoder; - public GroupingShuffleSource(PipelineOptions options, - byte[] shuffleReaderConfig, - String startShufflePosition, - String stopShufflePosition, - Coder>>> coder, - BatchModeExecutionContext executionContext) + public GroupingShuffleReader(PipelineOptions options, byte[] shuffleReaderConfig, + String startShufflePosition, String stopShufflePosition, + Coder>>> coder, BatchModeExecutionContext executionContext) throws Exception { this.shuffleReaderConfig = shuffleReaderConfig; this.startShufflePosition = startShufflePosition; @@ -83,45 +77,40 @@ public GroupingShuffleSource(PipelineOptions options, } @Override - public SourceIterator>>> iterator() - throws IOException { + public ReaderIterator>>> iterator() throws IOException { Preconditions.checkArgument(shuffleReaderConfig != null); return iterator(new BatchingShuffleEntryReader( - new ChunkingShuffleBatchReader(new ApplianceShuffleReader( - shuffleReaderConfig)))); + new ChunkingShuffleBatchReader(new ApplianceShuffleReader(shuffleReaderConfig)))); } private void initCoder(Coder>>> coder) throws Exception { if (!(coder instanceof WindowedValueCoder)) { - throw new Exception( - "unexpected kind of coder for WindowedValue: " + coder); + throw new Exception("unexpected kind of coder for WindowedValue: " + coder); } Coder>> elemCoder = ((WindowedValueCoder>>) coder).getValueCoder(); if (!(elemCoder instanceof KvCoder)) { - throw new Exception( - "unexpected kind of coder for elements read from " + - "a key-grouping shuffle: " + elemCoder); + throw new Exception("unexpected kind of coder for elements read from " + + "a key-grouping shuffle: " + elemCoder); } KvCoder> kvCoder = (KvCoder>) elemCoder; this.keyCoder = kvCoder.getKeyCoder(); Coder> kvValueCoder = kvCoder.getValueCoder(); if (!(kvValueCoder instanceof IterableCoder)) { - throw new Exception( - "unexpected kind of coder for values of KVs read from " + - "a key-grouping shuffle"); + throw new Exception("unexpected kind of coder for values of KVs read from " + + "a key-grouping shuffle"); } IterableCoder iterCoder = (IterableCoder) kvValueCoder; this.valueCoder = iterCoder.getElemCoder(); } - final SourceIterator>>> iterator(ShuffleEntryReader reader) + final ReaderIterator>>> iterator(ShuffleEntryReader reader) throws IOException { - return new GroupingShuffleSourceIterator(reader); + return new GroupingShuffleReaderIterator(reader); } /** - * A SourceIterator that reads from a ShuffleEntryReader and groups + * A ReaderIterator that reads from a ShuffleEntryReader and groups * all the values with the same key. * *

A key limitation of this implementation is that all iterator accesses @@ -137,17 +126,17 @@ final SourceIterator>>> iterator(ShuffleEntryR * to the current key -- which would introduce a performance * penalty. */ - private final class GroupingShuffleSourceIterator - extends AbstractSourceIterator>>> { + private final class GroupingShuffleReaderIterator + extends AbstractReaderIterator>>> { // N.B. This class is *not* static; it uses the keyCoder, valueCoder, and - // executionContext from its enclosing GroupingShuffleSource. + // executionContext from its enclosing GroupingShuffleReader. /** The iterator over shuffle entries, grouped by common key. */ private final Iterator groups; /** The stop position. No records with a position at or after * @stopPosition will be returned. Initialized - * to @AbstractShuffleSource.stopShufflePosition but can be + * to @AbstractShuffleReader.stopShufflePosition but can be * dynamically updated via @updateStopPosition() (note that such * updates can only decrease @stopPosition). * @@ -157,7 +146,7 @@ private final class GroupingShuffleSourceIterator private ByteArrayShufflePosition stopPosition = null; /** - * Position that this @GroupingShuffleSourceIterator is guaranteed + * Position that this @GroupingShuffleReaderIterator is guaranteed * not to stop before reaching (inclusive); @promisedPosition can * only increase monotonically and is updated when advancing to a * new group of records (either in the most recent call to next() @@ -168,20 +157,18 @@ private final class GroupingShuffleSourceIterator /** The next group to be consumed, if available. */ private KeyGroupedShuffleEntries nextGroup = null; - public GroupingShuffleSourceIterator(ShuffleEntryReader reader) { - promisedPosition = ByteArrayShufflePosition.fromBase64( - startShufflePosition); + public GroupingShuffleReaderIterator(ShuffleEntryReader reader) { + promisedPosition = ByteArrayShufflePosition.fromBase64(startShufflePosition); if (promisedPosition == null) { promisedPosition = new ByteArrayShufflePosition(new byte[0]); } stopPosition = ByteArrayShufflePosition.fromBase64(stopShufflePosition); - this.groups = new GroupingShuffleEntryIterator(reader.read( - promisedPosition, stopPosition)) { - @Override - protected void notifyElementRead(long byteSize) { - GroupingShuffleSource.this.notifyElementRead(byteSize); - } - }; + this.groups = new GroupingShuffleEntryIterator(reader.read(promisedPosition, stopPosition)) { + @Override + protected void notifyElementRead(long byteSize) { + GroupingShuffleReader.this.notifyElementRead(byteSize); + } + }; } private void advanceIfNecessary() { @@ -197,8 +184,7 @@ public boolean hasNext() throws IOException { if (nextGroup == null) { return false; } - return stopPosition == null - || promisedPosition.compareTo(stopPosition) < 0; + return stopPosition == null || promisedPosition.compareTo(stopPosition) < 0; } @Override @@ -220,8 +206,8 @@ public WindowedValue>> next() throws IOException { /** * Returns the position before the next {@code KV>} to be returned by the - * {@link GroupingShuffleSourceIterator}. Returns null if the - * {@link GroupingShuffleSourceIterator} is finished. + * {@link GroupingShuffleReaderIterator}. Returns null if the + * {@link GroupingShuffleReaderIterator} is finished. */ @Override public Progress getProgress() { @@ -230,13 +216,13 @@ public Progress getProgress() { ApproximateProgress progress = new ApproximateProgress(); position.setShufflePosition(promisedPosition.encodeBase64()); progress.setPosition(position); - return cloudProgressToSourceProgress(progress); + return cloudProgressToReaderProgress(progress); } /** * Updates the stop position of the shuffle source to the position proposed. Ignores the * proposed stop position if it is smaller than or equal to the position before the next - * {@code KV>} to be returned by the {@link GroupingShuffleSourceIterator}. + * {@code KV>} to be returned by the {@link GroupingShuffleReaderIterator}. */ @Override public Position updateStopPosition(Progress proposedStopPosition) { @@ -244,14 +230,12 @@ public Position updateStopPosition(Progress proposedStopPosition) { com.google.api.services.dataflow.model.Position stopCloudPosition = sourceProgressToCloudProgress(proposedStopPosition).getPosition(); if (stopCloudPosition == null) { - LOG.warn( - "A stop position other than a Position is not supported now."); + LOG.warn("A stop position other than a Position is not supported now."); return null; } if (stopCloudPosition.getShufflePosition() == null) { - LOG.warn( - "A stop position other than shuffle position is not supported in " + LOG.warn("A stop position other than shuffle position is not supported in " + "a grouping shuffle source: " + stopCloudPosition.toString()); return null; } @@ -259,36 +243,31 @@ public Position updateStopPosition(Progress proposedStopPosition) { ByteArrayShufflePosition.fromBase64(stopCloudPosition.getShufflePosition()); if (newStopPosition.compareTo(promisedPosition) <= 0) { - LOG.warn("Proposed stop position: " - + stopCloudPosition.getShufflePosition() + " <= promised position: " - + promisedPosition.encodeBase64()); + LOG.warn("Proposed stop position: " + stopCloudPosition.getShufflePosition() + + " <= promised position: " + promisedPosition.encodeBase64()); return null; } - if (this.stopPosition != null - && newStopPosition.compareTo(this.stopPosition) >= 0) { - LOG.warn("Proposed stop position: " - + stopCloudPosition.getShufflePosition() - + " >= current stop position: " - + this.stopPosition.encodeBase64()); + if (this.stopPosition != null && newStopPosition.compareTo(this.stopPosition) >= 0) { + LOG.warn("Proposed stop position: " + stopCloudPosition.getShufflePosition() + + " >= current stop position: " + this.stopPosition.encodeBase64()); return null; } this.stopPosition = newStopPosition; - LOG.info("Updated the stop position to " - + stopCloudPosition.getShufflePosition()); + LOG.info("Updated the stop position to " + stopCloudPosition.getShufflePosition()); - return cloudPositionToSourcePosition(stopCloudPosition); + return cloudPositionToReaderPosition(stopCloudPosition); } /** * Provides the {@link Reiterable} used to iterate through the values part * of a {@code KV>} entry produced by a - * {@link GroupingShuffleSource}. + * {@link GroupingShuffleReader}. */ private final class ValuesIterable implements Reiterable { // N.B. This class is *not* static; it uses the valueCoder from - // its enclosing GroupingShuffleSource. + // its enclosing GroupingShuffleReader. private final Reiterable base; @@ -305,11 +284,11 @@ public ValuesIterator iterator() { /** * Provides the {@link Reiterator} used to iterate through the values part * of a {@code KV>} entry produced by a - * {@link GroupingShuffleSource}. + * {@link GroupingShuffleReader}. */ private final class ValuesIterator implements Reiterator { // N.B. This class is *not* static; it uses the valueCoder from - // its enclosing GroupingShuffleSource. + // its enclosing GroupingShuffleReader. private final Reiterator base; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderFactory.java similarity index 68% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceFactory.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderFactory.java index 2229a77ddc10b..29a444bb280dc 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderFactory.java @@ -29,34 +29,24 @@ import com.google.cloud.dataflow.sdk.values.KV; /** - * Creates a GroupingShuffleSource from a CloudObject spec. + * Creates a GroupingShuffleReader from a CloudObject spec. */ -public class GroupingShuffleSourceFactory { +public class GroupingShuffleReaderFactory { // Do not instantiate. - private GroupingShuffleSourceFactory() {} + private GroupingShuffleReaderFactory() {} - public static GroupingShuffleSource create( - PipelineOptions options, - CloudObject spec, - Coder>>> coder, - ExecutionContext executionContext) + public static GroupingShuffleReader create(PipelineOptions options, CloudObject spec, + Coder>>> coder, ExecutionContext executionContext) throws Exception { - return create(options, spec, coder, - (BatchModeExecutionContext) executionContext); + return create(options, spec, coder, (BatchModeExecutionContext) executionContext); } - static GroupingShuffleSource create( - PipelineOptions options, - CloudObject spec, - Coder>>> coder, - BatchModeExecutionContext executionContext) + static GroupingShuffleReader create(PipelineOptions options, CloudObject spec, + Coder>>> coder, BatchModeExecutionContext executionContext) throws Exception { - return new GroupingShuffleSource<>( - options, + return new GroupingShuffleReader<>(options, decodeBase64(getString(spec, PropertyNames.SHUFFLE_READER_CONFIG)), getString(spec, PropertyNames.START_SHUFFLE_POSITION, null), - getString(spec, PropertyNames.END_SHUFFLE_POSITION, null), - coder, - executionContext); + getString(spec, PropertyNames.END_SHUFFLE_POSITION, null), coder, executionContext); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java similarity index 76% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySource.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java index a0a524ee0c9b9..487daa1affe45 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java @@ -17,8 +17,8 @@ package com.google.cloud.dataflow.sdk.runners.worker; import static com.google.api.client.util.Preconditions.checkNotNull; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToSourcePosition; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; import static java.lang.Math.min; @@ -26,7 +26,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.StringUtils; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,18 +42,16 @@ * * @param the type of the elements read from the source */ -public class InMemorySource extends Source { - private static final Logger LOG = LoggerFactory.getLogger(InMemorySource.class); +public class InMemoryReader extends Reader { + private static final Logger LOG = LoggerFactory.getLogger(InMemoryReader.class); final List encodedElements; final int startIndex; final int endIndex; final Coder coder; - public InMemorySource(List encodedElements, - @Nullable Long startIndex, - @Nullable Long endIndex, - Coder coder) { + public InMemoryReader(List encodedElements, @Nullable Long startIndex, + @Nullable Long endIndex, Coder coder) { this.encodedElements = encodedElements; int maxIndex = encodedElements.size(); if (startIndex == null) { @@ -68,8 +66,7 @@ public InMemorySource(List encodedElements, this.endIndex = maxIndex; } else { if (endIndex < this.startIndex) { - throw new IllegalArgumentException( - "end index should be >= start index"); + throw new IllegalArgumentException("end index should be >= start index"); } this.endIndex = (int) min(endIndex, maxIndex); } @@ -77,18 +74,18 @@ public InMemorySource(List encodedElements, } @Override - public SourceIterator iterator() throws IOException { - return new InMemorySourceIterator(); + public ReaderIterator iterator() throws IOException { + return new InMemoryReaderIterator(); } /** - * A SourceIterator that yields an in-memory list of elements. + * A ReaderIterator that yields an in-memory list of elements. */ - class InMemorySourceIterator extends AbstractSourceIterator { + class InMemoryReaderIterator extends AbstractReaderIterator { int index; int endPosition; - public InMemorySourceIterator() { + public InMemoryReaderIterator() { index = startIndex; endPosition = endIndex; } @@ -106,8 +103,7 @@ public T next() throws IOException { String encodedElementString = encodedElements.get(index++); // TODO: Replace with the real encoding used by the // front end, when we know what it is. - byte[] encodedElement = - StringUtils.jsonStringToByteArray(encodedElementString); + byte[] encodedElement = StringUtils.jsonStringToByteArray(encodedElementString); notifyElementRead(encodedElement.length); return CoderUtils.decodeFromByteArray(coder, encodedElement); } @@ -115,7 +111,7 @@ public T next() throws IOException { @Override public Progress getProgress() { // Currently we assume that only a record index position is reported as - // current progress. Source writer can override this method to update + // current progress. An implementer can override this method to update // other metrics, e.g. completion percentage or remaining time. com.google.api.services.dataflow.model.Position currentPosition = new com.google.api.services.dataflow.model.Position(); @@ -124,7 +120,7 @@ public Progress getProgress() { ApproximateProgress progress = new ApproximateProgress(); progress.setPosition(currentPosition); - return cloudProgressToSourceProgress(progress); + return cloudProgressToReaderProgress(progress); } @Override @@ -132,21 +128,19 @@ public Position updateStopPosition(Progress proposedStopPosition) { checkNotNull(proposedStopPosition); // Currently we only support stop position in record index of - // an API Position in InMemorySource. If stop position in other types is + // an API Position in InMemoryReader. If stop position in other types is // proposed, the end position in iterator will not be updated, // and return null. com.google.api.services.dataflow.model.Position stopPosition = sourceProgressToCloudProgress(proposedStopPosition).getPosition(); if (stopPosition == null) { - LOG.warn( - "A stop position other than a Dataflow API Position is not currently supported."); + LOG.warn("A stop position other than a Dataflow API Position is not currently supported."); return null; } Long recordIndex = stopPosition.getRecordIndex(); if (recordIndex == null) { - LOG.warn( - "A stop position other than record index is not supported in InMemorySource."); + LOG.warn("A stop position other than record index is not supported in InMemoryReader."); return null; } if (recordIndex <= index || recordIndex >= endPosition) { @@ -157,7 +151,7 @@ public Position updateStopPosition(Progress proposedStopPosition) { } this.endPosition = recordIndex.intValue(); - return cloudPositionToSourcePosition(stopPosition); + return cloudPositionToReaderPosition(stopPosition); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderFactory.java similarity index 64% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactory.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderFactory.java index 3f2cd9c9a1dba..cc51f8f0c94c4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderFactory.java @@ -28,27 +28,21 @@ import java.util.Collections; /** - * Creates an InMemorySource from a CloudObject spec. + * Creates an InMemoryReader from a CloudObject spec. */ -public class InMemorySourceFactory { +public class InMemoryReaderFactory { // Do not instantiate. - private InMemorySourceFactory() {} + private InMemoryReaderFactory() {} - public static InMemorySource create(PipelineOptions options, - CloudObject spec, - Coder coder, - ExecutionContext executionContext) - throws Exception { + public static InMemoryReader create(PipelineOptions options, CloudObject spec, + Coder coder, ExecutionContext executionContext) throws Exception { return create(spec, coder); } - static InMemorySource create(CloudObject spec, - Coder coder) throws Exception { - return new InMemorySource<>( - getStrings(spec, - PropertyNames.ELEMENTS, Collections.emptyList()), + static InMemoryReader create(CloudObject spec, Coder coder) throws Exception { + return new InMemoryReader<>( + getStrings(spec, PropertyNames.ELEMENTS, Collections.emptyList()), getLong(spec, PropertyNames.START_INDEX, null), - getLong(spec, PropertyNames.END_INDEX, null), - coder); + getLong(spec, PropertyNames.END_INDEX, null), coder); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/LazyMultiSourceIterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/LazyMultiReaderIterator.java similarity index 78% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/LazyMultiSourceIterator.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/LazyMultiReaderIterator.java index 3ccebd5617565..aef1e9f191ec1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/LazyMultiSourceIterator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/LazyMultiReaderIterator.java @@ -16,14 +16,14 @@ package com.google.cloud.dataflow.sdk.runners.worker; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import java.io.IOException; import java.util.Iterator; import java.util.NoSuchElementException; /** - * Implements a SourceIterator over a collection of inputs. + * Implements a ReaderIterator over a collection of inputs. * * The sources are used sequentially, each consumed entirely before moving * to the next source. @@ -33,18 +33,17 @@ * be produced lazily, as an open source iterator may consume process resources * such as file descriptors. */ -abstract class LazyMultiSourceIterator - extends Source.AbstractSourceIterator { +abstract class LazyMultiReaderIterator extends Reader.AbstractReaderIterator { private final Iterator inputs; - Source.SourceIterator current; + Reader.ReaderIterator current; - public LazyMultiSourceIterator(Iterator inputs) { + public LazyMultiReaderIterator(Iterator inputs) { this.inputs = inputs; } @Override public boolean hasNext() throws IOException { - while (selectSource()) { + while (selectReader()) { if (!current.hasNext()) { current.close(); current = null; @@ -65,16 +64,15 @@ public T next() throws IOException { @Override public void close() throws IOException { - while (selectSource()) { + while (selectReader()) { current.close(); current = null; } } - protected abstract Source.SourceIterator open(String input) - throws IOException; + protected abstract Reader.ReaderIterator open(String input) throws IOException; - boolean selectSource() throws IOException { + boolean selectReader() throws IOException { if (current != null) { return true; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java index 095aa0876ee8e..dd0133e7b3763 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java @@ -45,9 +45,9 @@ import com.google.cloud.dataflow.sdk.util.common.worker.ParDoOperation; import com.google.cloud.dataflow.sdk.util.common.worker.PartialGroupByKeyOperation; import com.google.cloud.dataflow.sdk.util.common.worker.ReadOperation; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.cloud.dataflow.sdk.util.common.worker.ReceivingOperation; import com.google.cloud.dataflow.sdk.util.common.worker.Sink; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; import com.google.cloud.dataflow.sdk.util.common.worker.WriteOperation; import com.google.cloud.dataflow.sdk.values.KV; @@ -64,24 +64,19 @@ public class MapTaskExecutorFactory { /** * Creates a new MapTaskExecutor from the given MapTask definition. */ - public static MapTaskExecutor create(PipelineOptions options, - MapTask mapTask, - ExecutionContext context) - throws Exception { + public static MapTaskExecutor create( + PipelineOptions options, MapTask mapTask, ExecutionContext context) throws Exception { List operations = new ArrayList<>(); CounterSet counters = new CounterSet(); String counterPrefix = mapTask.getStageName() + "-"; - StateSampler stateSampler = new StateSampler( - counterPrefix, counters.getAddCounterMutator()); + StateSampler stateSampler = new StateSampler(counterPrefix, counters.getAddCounterMutator()); // Open-ended state. stateSampler.setState("other"); // Instantiate operations for each instruction in the graph. for (ParallelInstruction instruction : mapTask.getInstructions()) { - operations.add( - createOperation(options, instruction, context, operations, - counterPrefix, counters.getAddCounterMutator(), - stateSampler)); + operations.add(createOperation(options, instruction, context, operations, counterPrefix, + counters.getAddCounterMutator(), stateSampler)); } return new MapTaskExecutor(operations, counters, stateSampler); @@ -90,130 +85,87 @@ public static MapTaskExecutor create(PipelineOptions options, /** * Creates an Operation from the given ParallelInstruction definition. */ - static Operation createOperation( - PipelineOptions options, - ParallelInstruction instruction, - ExecutionContext executionContext, - List priorOperations, - String counterPrefix, - CounterSet.AddCounterMutator addCounterMutator, - StateSampler stateSampler) - throws Exception { + static Operation createOperation(PipelineOptions options, ParallelInstruction instruction, + ExecutionContext executionContext, List priorOperations, String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { if (instruction.getRead() != null) { - return createReadOperation( - options, instruction, executionContext, priorOperations, + return createReadOperation(options, instruction, executionContext, priorOperations, counterPrefix, addCounterMutator, stateSampler); } else if (instruction.getWrite() != null) { - return createWriteOperation( - options, instruction, executionContext, priorOperations, + return createWriteOperation(options, instruction, executionContext, priorOperations, counterPrefix, addCounterMutator, stateSampler); } else if (instruction.getParDo() != null) { - return createParDoOperation( - options, instruction, executionContext, priorOperations, + return createParDoOperation(options, instruction, executionContext, priorOperations, counterPrefix, addCounterMutator, stateSampler); } else if (instruction.getPartialGroupByKey() != null) { - return createPartialGroupByKeyOperation( - options, instruction, executionContext, priorOperations, - counterPrefix, addCounterMutator, stateSampler); + return createPartialGroupByKeyOperation(options, instruction, executionContext, + priorOperations, counterPrefix, addCounterMutator, stateSampler); } else if (instruction.getFlatten() != null) { - return createFlattenOperation( - options, instruction, executionContext, priorOperations, + return createFlattenOperation(options, instruction, executionContext, priorOperations, counterPrefix, addCounterMutator, stateSampler); } else { throw new Exception("Unexpected instruction: " + instruction); } } - static ReadOperation createReadOperation( - PipelineOptions options, - ParallelInstruction instruction, - ExecutionContext executionContext, - List priorOperations, - String counterPrefix, - CounterSet.AddCounterMutator addCounterMutator, - StateSampler stateSampler) - throws Exception { + static ReadOperation createReadOperation(PipelineOptions options, ParallelInstruction instruction, + ExecutionContext executionContext, List priorOperations, String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { ReadInstruction read = instruction.getRead(); - Source source = - SourceFactory.create(options, read.getSource(), executionContext); + Reader reader = ReaderFactory.create(options, read.getSource(), executionContext); - OutputReceiver[] receivers = createOutputReceivers( - instruction, counterPrefix, addCounterMutator, stateSampler, 1); + OutputReceiver[] receivers = + createOutputReceivers(instruction, counterPrefix, addCounterMutator, stateSampler, 1); - return new ReadOperation(instruction.getSystemName(), source, receivers, - counterPrefix, addCounterMutator, stateSampler); + return new ReadOperation(instruction.getSystemName(), reader, receivers, counterPrefix, + addCounterMutator, stateSampler); } - static WriteOperation createWriteOperation( - PipelineOptions options, - ParallelInstruction instruction, - ExecutionContext executionContext, - List priorOperations, - String counterPrefix, - CounterSet.AddCounterMutator addCounterMutator, - StateSampler stateSampler) - throws Exception { + static WriteOperation createWriteOperation(PipelineOptions options, + ParallelInstruction instruction, ExecutionContext executionContext, + List priorOperations, String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { WriteInstruction write = instruction.getWrite(); Sink sink = SinkFactory.create(options, write.getSink(), executionContext); - OutputReceiver[] receivers = createOutputReceivers( - instruction, counterPrefix, addCounterMutator, stateSampler, 0); + OutputReceiver[] receivers = + createOutputReceivers(instruction, counterPrefix, addCounterMutator, stateSampler, 0); - WriteOperation operation = - new WriteOperation(instruction.getSystemName(), sink, receivers, - counterPrefix, addCounterMutator, stateSampler); + WriteOperation operation = new WriteOperation(instruction.getSystemName(), sink, receivers, + counterPrefix, addCounterMutator, stateSampler); attachInput(operation, write.getInput(), priorOperations); return operation; } - static ParDoOperation createParDoOperation( - PipelineOptions options, - ParallelInstruction instruction, - ExecutionContext executionContext, - List priorOperations, - String counterPrefix, - CounterSet.AddCounterMutator addCounterMutator, - StateSampler stateSampler) - throws Exception { + static ParDoOperation createParDoOperation(PipelineOptions options, + ParallelInstruction instruction, ExecutionContext executionContext, + List priorOperations, String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { ParDoInstruction parDo = instruction.getParDo(); - ParDoFn fn = ParDoFnFactory.create( - options, - CloudObject.fromSpec(parDo.getUserFn()), - instruction.getSystemName(), - parDo.getSideInputs(), - parDo.getMultiOutputInfos(), - parDo.getNumOutputs(), - executionContext, - addCounterMutator, - stateSampler); + ParDoFn fn = ParDoFnFactory.create(options, CloudObject.fromSpec(parDo.getUserFn()), + instruction.getSystemName(), parDo.getSideInputs(), parDo.getMultiOutputInfos(), + parDo.getNumOutputs(), executionContext, addCounterMutator, stateSampler); - OutputReceiver[] receivers = - createOutputReceivers(instruction, counterPrefix, addCounterMutator, - stateSampler, parDo.getNumOutputs()); + OutputReceiver[] receivers = createOutputReceivers( + instruction, counterPrefix, addCounterMutator, stateSampler, parDo.getNumOutputs()); - ParDoOperation operation = - new ParDoOperation(instruction.getSystemName(), fn, receivers, - counterPrefix, addCounterMutator, stateSampler); + ParDoOperation operation = new ParDoOperation( + instruction.getSystemName(), fn, receivers, counterPrefix, addCounterMutator, stateSampler); attachInput(operation, parDo.getInput(), priorOperations); return operation; } - static PartialGroupByKeyOperation createPartialGroupByKeyOperation( - PipelineOptions options, - ParallelInstruction instruction, - ExecutionContext executionContext, - List priorOperations, - String counterPrefix, - CounterSet.AddCounterMutator addCounterMutator, - StateSampler stateSampler) - throws Exception { + static PartialGroupByKeyOperation createPartialGroupByKeyOperation(PipelineOptions options, + ParallelInstruction instruction, ExecutionContext executionContext, + List priorOperations, String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { PartialGroupByKeyInstruction pgbk = instruction.getPartialGroupByKey(); Coder coder = Serializer.deserialize(pgbk.getInputElementCodec(), Coder.class); @@ -230,19 +182,14 @@ static PartialGroupByKeyOperation createPartialGroupByKeyOperation( Coder keyCoder = kvCoder.getKeyCoder(); Coder valueCoder = kvCoder.getValueCoder(); - OutputReceiver[] receivers = createOutputReceivers( - instruction, counterPrefix, addCounterMutator, stateSampler, 1); + OutputReceiver[] receivers = + createOutputReceivers(instruction, counterPrefix, addCounterMutator, stateSampler, 1); PartialGroupByKeyOperation operation = new PartialGroupByKeyOperation(instruction.getSystemName(), - new CoderGroupingKeyCreator(keyCoder), - new CoderSizeEstimator(keyCoder), - new CoderSizeEstimator(valueCoder), - 0.001 /*sizeEstimatorSampleRate*/, - PairInfo.create(), - receivers, - counterPrefix, addCounterMutator, - stateSampler); + new CoderGroupingKeyCreator(keyCoder), new CoderSizeEstimator(keyCoder), + new CoderSizeEstimator(valueCoder), 0.001/*sizeEstimatorSampleRate*/, PairInfo.create(), + receivers, counterPrefix, addCounterMutator, stateSampler); attachInput(operation, pgbk.getInput(), priorOperations); @@ -254,7 +201,9 @@ static PartialGroupByKeyOperation createPartialGroupByKeyOperation( */ public static class PairInfo implements PartialGroupByKeyOperation.PairInfo { private static PairInfo theInstance = new PairInfo(); - public static PairInfo create() { return theInstance; } + public static PairInfo create() { + return theInstance; + } private PairInfo() {} @Override public Object getKeyFromInputPair(Object pair) { @@ -293,8 +242,7 @@ public Object createGroupingKey(Object value) throws Exception { /** * Implements PGBKOp.SizeEstimator via Coder. */ - public static class CoderSizeEstimator - implements PartialGroupByKeyOperation.SizeEstimator { + public static class CoderSizeEstimator implements PartialGroupByKeyOperation.SizeEstimator { final Coder coder; public CoderSizeEstimator(Coder coder) { @@ -307,24 +255,17 @@ public long estimateSize(Object value) throws Exception { } } - static FlattenOperation createFlattenOperation( - PipelineOptions options, - ParallelInstruction instruction, - ExecutionContext executionContext, - List priorOperations, - String counterPrefix, - CounterSet.AddCounterMutator addCounterMutator, - StateSampler stateSampler) - throws Exception { + static FlattenOperation createFlattenOperation(PipelineOptions options, + ParallelInstruction instruction, ExecutionContext executionContext, + List priorOperations, String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { FlattenInstruction flatten = instruction.getFlatten(); OutputReceiver[] receivers = - createOutputReceivers(instruction, counterPrefix, addCounterMutator, - stateSampler, 1); + createOutputReceivers(instruction, counterPrefix, addCounterMutator, stateSampler, 1); - FlattenOperation operation = - new FlattenOperation(instruction.getSystemName(), receivers, - counterPrefix, addCounterMutator, stateSampler); + FlattenOperation operation = new FlattenOperation( + instruction.getSystemName(), receivers, counterPrefix, addCounterMutator, stateSampler); for (InstructionInput input : flatten.getInputs()) { attachInput(operation, input, priorOperations); @@ -337,30 +278,23 @@ static FlattenOperation createFlattenOperation( * Returns an array of OutputReceivers for the given * ParallelInstruction definition. */ - static OutputReceiver[] createOutputReceivers( - ParallelInstruction instruction, - String counterPrefix, - CounterSet.AddCounterMutator addCounterMutator, - StateSampler stateSampler, - int expectedNumOutputs) - throws Exception { + static OutputReceiver[] createOutputReceivers(ParallelInstruction instruction, + String counterPrefix, CounterSet.AddCounterMutator addCounterMutator, + StateSampler stateSampler, int expectedNumOutputs) throws Exception { int numOutputs = 0; if (instruction.getOutputs() != null) { numOutputs = instruction.getOutputs().size(); } if (numOutputs != expectedNumOutputs) { - throw new AssertionError( - "ParallelInstruction.Outputs has an unexpected length"); + throw new AssertionError("ParallelInstruction.Outputs has an unexpected length"); } OutputReceiver[] receivers = new OutputReceiver[numOutputs]; for (int i = 0; i < numOutputs; i++) { InstructionOutput cloudOutput = instruction.getOutputs().get(i); - receivers[i] = new OutputReceiver( - cloudOutput.getName(), - new ElementByteSizeObservableCoder( - Serializer.deserialize(cloudOutput.getCodec(), Coder.class)), - counterPrefix, - addCounterMutator); + receivers[i] = new OutputReceiver(cloudOutput.getName(), + new ElementByteSizeObservableCoder(Serializer.deserialize( + cloudOutput.getCodec(), Coder.class)), + counterPrefix, addCounterMutator); } return receivers; } @@ -368,8 +302,7 @@ static OutputReceiver[] createOutputReceivers( /** * Adapts a Coder to the ElementByteSizeObservable interface. */ - public static class ElementByteSizeObservableCoder - implements ElementByteSizeObservable { + public static class ElementByteSizeObservableCoder implements ElementByteSizeObservable { final Coder coder; public ElementByteSizeObservableCoder(Coder coder) { @@ -382,8 +315,7 @@ public boolean isRegisterByteSizeObserverCheap(T value) { } @Override - public void registerByteSizeObserver(T value, - ElementByteSizeObserver observer) + public void registerByteSizeObserver(T value, ElementByteSizeObserver observer) throws Exception { coder.registerByteSizeObserver(value, observer, Coder.Context.OUTER); } @@ -393,9 +325,8 @@ public void registerByteSizeObserver(T value, * Adds an input to the given Operation, coming from the given * producer instruction output. */ - static void attachInput(ReceivingOperation operation, - @Nullable InstructionInput input, - List priorOperations) { + static void attachInput(ReceivingOperation operation, @Nullable InstructionInput input, + List priorOperations) { Integer producerInstructionIndex = 0; Integer outputNum = 0; if (input != null) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleReader.java similarity index 75% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSource.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleReader.java index e1401b6cd83ad..771bd2baadba5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleReader.java @@ -24,9 +24,9 @@ import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; import com.google.cloud.dataflow.sdk.util.common.worker.BatchingShuffleEntryReader; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntryReader; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; import com.google.cloud.dataflow.sdk.values.KV; import java.io.IOException; @@ -39,19 +39,15 @@ * @param the type of the keys read from the shuffle * @param the type of the values read from the shuffle */ -public class PartitioningShuffleSource extends Source>> { - +public class PartitioningShuffleReader extends Reader>> { final byte[] shuffleReaderConfig; final String startShufflePosition; final String stopShufflePosition; Coder keyCoder; WindowedValueCoder windowedValueCoder; - public PartitioningShuffleSource(PipelineOptions options, - byte[] shuffleReaderConfig, - String startShufflePosition, - String stopShufflePosition, - Coder>> coder) + public PartitioningShuffleReader(PipelineOptions options, byte[] shuffleReaderConfig, + String startShufflePosition, String stopShufflePosition, Coder>> coder) throws Exception { this.shuffleReaderConfig = shuffleReaderConfig; this.startShufflePosition = startShufflePosition; @@ -65,14 +61,12 @@ public PartitioningShuffleSource(PipelineOptions options, */ private void initCoder(Coder>> coder) throws Exception { if (!(coder instanceof WindowedValueCoder)) { - throw new Exception( - "unexpected kind of coder for WindowedValue: " + coder); + throw new Exception("unexpected kind of coder for WindowedValue: " + coder); } WindowedValueCoder> windowedElemCoder = ((WindowedValueCoder>) coder); Coder> elemCoder = windowedElemCoder.getValueCoder(); if (!(elemCoder instanceof KvCoder)) { - throw new Exception( - "unexpected kind of coder for elements read from " + throw new Exception("unexpected kind of coder for elements read from " + "a key-partitioning shuffle: " + elemCoder); } KvCoder kvCoder = (KvCoder) elemCoder; @@ -81,28 +75,25 @@ private void initCoder(Coder>> coder) throws Exception { } @Override - public com.google.cloud.dataflow.sdk.util.common.worker.Source.SourceIterator< - WindowedValue>> iterator() throws IOException { + public ReaderIterator>> iterator() throws IOException { Preconditions.checkArgument(shuffleReaderConfig != null); return iterator(new BatchingShuffleEntryReader( - new ChunkingShuffleBatchReader(new ApplianceShuffleReader( - shuffleReaderConfig)))); + new ChunkingShuffleBatchReader(new ApplianceShuffleReader(shuffleReaderConfig)))); } - SourceIterator>> iterator(ShuffleEntryReader reader) throws IOException { - return new PartitioningShuffleSourceIterator(reader); + ReaderIterator>> iterator(ShuffleEntryReader reader) throws IOException { + return new PartitioningShuffleReaderIterator(reader); } /** - * A SourceIterator that reads from a ShuffleEntryReader, + * A ReaderIterator that reads from a ShuffleEntryReader, * extracts K and {@code WindowedValue}, and returns a constructed * {@code WindowedValue}. */ - class PartitioningShuffleSourceIterator - extends AbstractSourceIterator>> { + class PartitioningShuffleReaderIterator extends AbstractReaderIterator>> { Iterator iterator; - PartitioningShuffleSourceIterator(ShuffleEntryReader reader) { + PartitioningShuffleReaderIterator(ShuffleEntryReader reader) { this.iterator = reader.read( ByteArrayShufflePosition.fromBase64(startShufflePosition), ByteArrayShufflePosition.fromBase64(stopShufflePosition)); @@ -120,9 +111,8 @@ public WindowedValue> next() throws IOException { WindowedValue windowedValue = CoderUtils.decodeFromByteArray(windowedValueCoder, record.getValue()); notifyElementRead(record.length()); - return WindowedValue.of(KV.of(key, windowedValue.getValue()), - windowedValue.getTimestamp(), - windowedValue.getWindows()); + return WindowedValue.of(KV.of(key, windowedValue.getValue()), windowedValue.getTimestamp(), + windowedValue.getWindows()); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleReaderFactory.java similarity index 65% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceFactory.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleReaderFactory.java index f97d1d5b82988..72d7d6c7cb5ca 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleReaderFactory.java @@ -28,23 +28,18 @@ import com.google.cloud.dataflow.sdk.values.KV; /** - * Creates a PartitioningShuffleSource from a CloudObject spec. + * Creates a PartitioningShuffleReader from a CloudObject spec. */ -public class PartitioningShuffleSourceFactory { +public class PartitioningShuffleReaderFactory { // Do not instantiate. - private PartitioningShuffleSourceFactory() {} + private PartitioningShuffleReaderFactory() {} - public static PartitioningShuffleSource create( - PipelineOptions options, - CloudObject spec, - Coder>> coder, - ExecutionContext executionContext) + public static PartitioningShuffleReader create(PipelineOptions options, + CloudObject spec, Coder>> coder, ExecutionContext executionContext) throws Exception { - return new PartitioningShuffleSource( - options, - decodeBase64(getString(spec, PropertyNames.SHUFFLE_READER_CONFIG)), - getString(spec, PropertyNames.START_SHUFFLE_POSITION, null), - getString(spec, PropertyNames.END_SHUFFLE_POSITION, null), - coder); + return new PartitioningShuffleReader(options, + decodeBase64(getString(spec, PropertyNames.SHUFFLE_READER_CONFIG)), + getString(spec, PropertyNames.START_SHUFFLE_POSITION, null), + getString(spec, PropertyNames.END_SHUFFLE_POSITION, null), coder); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderFactory.java similarity index 59% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactory.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderFactory.java index d4726094a3ea6..5d42970c16000 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderFactory.java @@ -16,6 +16,7 @@ package com.google.cloud.dataflow.sdk.runners.worker; +import com.google.api.services.dataflow.model.Source; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.util.CloudObject; @@ -23,7 +24,7 @@ import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.InstanceBuilder; import com.google.cloud.dataflow.sdk.util.Serializer; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.common.reflect.TypeToken; import java.util.HashMap; @@ -32,71 +33,57 @@ import javax.annotation.Nullable; /** - * Constructs a Source from a Dataflow API Source definition. + * Constructs a Reader from a Dataflow API Source definition. * - * A SourceFactory concrete "subclass" should define a method with the following + * A ReaderFactory concrete "subclass" should define a method with the following * signature: *

 {@code
- * static SomeSourceSubclass create(PipelineOptions, CloudObject,
+ * static SomeReaderSubclass create(PipelineOptions, CloudObject,
  *                                     Coder, ExecutionContext);
  * } 
*/ -public final class SourceFactory { +public final class ReaderFactory { // Do not instantiate. - private SourceFactory() {} + private ReaderFactory() {} /** * A map from the short names of predefined sources to * their full factory class names. */ - static Map predefinedSourceFactories = new HashMap<>(); + static Map predefinedReaderFactories = new HashMap<>(); static { - predefinedSourceFactories.put( - "TextSource", - TextSourceFactory.class.getName()); - predefinedSourceFactories.put( - "AvroSource", - AvroSourceFactory.class.getName()); - predefinedSourceFactories.put( - "UngroupedShuffleSource", - UngroupedShuffleSourceFactory.class.getName()); - predefinedSourceFactories.put( - "PartitioningShuffleSource", - PartitioningShuffleSourceFactory.class.getName()); - predefinedSourceFactories.put( - "GroupingShuffleSource", - GroupingShuffleSourceFactory.class.getName()); - predefinedSourceFactories.put( - "InMemorySource", - InMemorySourceFactory.class.getName()); - predefinedSourceFactories.put( - "BigQuerySource", - BigQuerySourceFactory.class.getName()); + predefinedReaderFactories.put("TextSource", TextReaderFactory.class.getName()); + predefinedReaderFactories.put("AvroSource", AvroReaderFactory.class.getName()); + predefinedReaderFactories.put( + "UngroupedShuffleSource", UngroupedShuffleReaderFactory.class.getName()); + predefinedReaderFactories.put( + "PartitioningShuffleSource", PartitioningShuffleReaderFactory.class.getName()); + predefinedReaderFactories.put( + "GroupingShuffleSource", GroupingShuffleReaderFactory.class.getName()); + predefinedReaderFactories.put("InMemorySource", InMemoryReaderFactory.class.getName()); + predefinedReaderFactories.put("BigQuerySource", BigQueryReaderFactory.class.getName()); } /** - * Creates a Source from a Dataflow API Source definition. + * Creates a Reader from a Dataflow API Source definition. * * @throws Exception if the source could not be decoded and * constructed */ - public static Source create( - @Nullable PipelineOptions options, - com.google.api.services.dataflow.model.Source cloudSource, - @Nullable ExecutionContext executionContext) - throws Exception { + public static Reader create(@Nullable PipelineOptions options, Source cloudSource, + @Nullable ExecutionContext executionContext) throws Exception { cloudSource = CloudSourceUtils.flattenBaseSpecs(cloudSource); Coder coder = Serializer.deserialize(cloudSource.getCodec(), Coder.class); CloudObject object = CloudObject.fromSpec(cloudSource.getSpec()); - String sourceFactoryClassName = predefinedSourceFactories.get(object.getClassName()); + String sourceFactoryClassName = predefinedReaderFactories.get(object.getClassName()); if (sourceFactoryClassName == null) { sourceFactoryClassName = object.getClassName(); } try { - return InstanceBuilder.ofType(new TypeToken>() {}) + return InstanceBuilder.ofType(new TypeToken>() {}) .fromClassName(sourceFactoryClassName) .fromFactoryMethod("create") .withArg(PipelineOptions.class, options) @@ -106,8 +93,7 @@ public static Source create( .build(); } catch (ClassNotFoundException exn) { - throw new Exception( - "unable to create a source from " + cloudSource, exn); + throw new Exception("unable to create a source from " + cloudSource, exn); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSink.java index 72ea16fc99b43..e581528598ec9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSink.java @@ -37,8 +37,12 @@ * @param the type of the elements written to the sink */ public class ShuffleSink extends Sink> { - - enum ShuffleKind { UNGROUPED, PARTITION_KEYS, GROUP_KEYS, GROUP_KEYS_AND_SORT_VALUES } + enum ShuffleKind { + UNGROUPED, + PARTITION_KEYS, + GROUP_KEYS, + GROUP_KEYS_AND_SORT_VALUES + } static final long SHUFFLE_WRITER_BUFFER_SIZE = 128 << 20; @@ -58,8 +62,7 @@ enum ShuffleKind { UNGROUPED, PARTITION_KEYS, GROUP_KEYS, GROUP_KEYS_AND_SORT_VA Coder sortKeyCoder; Coder sortValueCoder; - public static ShuffleKind parseShuffleKind(String shuffleKind) - throws Exception { + public static ShuffleKind parseShuffleKind(String shuffleKind) throws Exception { try { return Enum.valueOf(ShuffleKind.class, shuffleKind.trim().toUpperCase()); } catch (IllegalArgumentException e) { @@ -67,11 +70,8 @@ public static ShuffleKind parseShuffleKind(String shuffleKind) } } - public ShuffleSink(PipelineOptions options, - byte[] shuffleWriterConfig, - ShuffleKind shuffleKind, - Coder> coder) - throws Exception { + public ShuffleSink(PipelineOptions options, byte[] shuffleWriterConfig, ShuffleKind shuffleKind, + Coder> coder) throws Exception { this.shuffleWriterConfig = shuffleWriterConfig; this.shuffleKind = shuffleKind; initCoder(coder); @@ -107,8 +107,7 @@ private void initCoder(Coder> coder) throws Exception { this.elemCoder = windowedElemCoder.getValueCoder(); if (shardByKey) { if (!(elemCoder instanceof KvCoder)) { - throw new Exception( - "unexpected kind of coder for elements written to " + throw new Exception("unexpected kind of coder for elements written to " + "a key-grouping shuffle"); } KvCoder kvCoder = (KvCoder) elemCoder; @@ -118,8 +117,7 @@ private void initCoder(Coder> coder) throws Exception { // TODO: Decide the representation of sort-keyed values. // For now, we'll just use KVs. if (!(valueCoder instanceof KvCoder)) { - throw new Exception( - "unexpected kind of coder for values written to " + throw new Exception("unexpected kind of coder for values written to " + "a value-sorting shuffle"); } KvCoder kvValueCoder = (KvCoder) valueCoder; @@ -168,8 +166,7 @@ public long add(WindowedValue windowedElem) throws IOException { T elem = windowedElem.getValue(); if (shardByKey) { if (!(elem instanceof KV)) { - throw new AssertionError( - "expecting the values written to a key-grouping shuffle " + throw new AssertionError("expecting the values written to a key-grouping shuffle " + "to be KVs"); } KV kv = (KV) elem; @@ -180,8 +177,7 @@ public long add(WindowedValue windowedElem) throws IOException { if (sortValues) { if (!(value instanceof KV)) { - throw new AssertionError( - "expecting the value parts of the KVs written to " + throw new AssertionError("expecting the value parts of the KVs written to " + "a value-sorting shuffle to also be KVs"); } KV kvValue = (KV) value; @@ -189,12 +185,11 @@ public long add(WindowedValue windowedElem) throws IOException { Object sortValue = kvValue.getValue(); // TODO: Need to coordinate with the - // GroupingShuffleSource, to make sure it knows how to + // GroupingShuffleReader, to make sure it knows how to // reconstruct the value from the sortKeyBytes and // sortValueBytes. Right now, it doesn't know between // sorting and non-sorting GBKs. - secondaryKeyBytes = - CoderUtils.encodeToByteArray(sortKeyCoder, sortKey); + secondaryKeyBytes = CoderUtils.encodeToByteArray(sortKeyCoder, sortKey); valueBytes = CoderUtils.encodeToByteArray(sortValueCoder, sortValue); } else if (groupValues) { @@ -222,15 +217,13 @@ public long add(WindowedValue windowedElem) throws IOException { // for writing a single-sharded ordered PCollection through a // shuffle, since the order of elements in the input will be // preserved in the output. - keyBytes = - CoderUtils.encodeToByteArray(BigEndianLongCoder.of(), seqNum++); + keyBytes = CoderUtils.encodeToByteArray(BigEndianLongCoder.of(), seqNum++); secondaryKeyBytes = null; valueBytes = CoderUtils.encodeToByteArray(windowedElemCoder, windowedElem); } - return writer.put(new ShuffleEntry( - keyBytes, secondaryKeyBytes, valueBytes)); + return writer.put(new ShuffleEntry(keyBytes, secondaryKeyBytes, valueBytes)); } @Override @@ -242,7 +235,7 @@ public void close() throws IOException { @Override public SinkWriter> writer() throws IOException { Preconditions.checkArgument(shuffleWriterConfig != null); - return writer(new ChunkingShuffleEntryWriter(new ApplianceShuffleWriter( - shuffleWriterConfig, SHUFFLE_WRITER_BUFFER_SIZE))); + return writer(new ChunkingShuffleEntryWriter( + new ApplianceShuffleWriter(shuffleWriterConfig, SHUFFLE_WRITER_BUFFER_SIZE))); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtils.java index f3fc1cf3f3ef3..374fd65a969f6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtils.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtils.java @@ -19,10 +19,11 @@ import static com.google.cloud.dataflow.sdk.util.Structs.getString; import com.google.api.services.dataflow.model.SideInputInfo; +import com.google.api.services.dataflow.model.Source; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.PropertyNames; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import java.util.ArrayList; import java.util.Iterator; @@ -41,20 +42,15 @@ public class SideInputUtils { * Reads the given side input, producing the contents associated * with a a {@link PCollectionView}. */ - public static Object readSideInput(PipelineOptions options, - SideInputInfo sideInputInfo, - ExecutionContext executionContext) - throws Exception { + public static Object readSideInput(PipelineOptions options, SideInputInfo sideInputInfo, + ExecutionContext executionContext) throws Exception { Iterable elements = readSideInputSources(options, sideInputInfo.getSources(), executionContext); return readSideInputValue(sideInputInfo.getKind(), elements); } - static Iterable readSideInputSources( - PipelineOptions options, - List sideInputSources, - ExecutionContext executionContext) - throws Exception { + static Iterable readSideInputSources(PipelineOptions options, + List sideInputSources, ExecutionContext executionContext) throws Exception { int numSideInputSources = sideInputSources.size(); if (numSideInputSources == 0) { throw new Exception("expecting at least one side input Source"); @@ -62,25 +58,19 @@ static Iterable readSideInputSources( return readSideInputSource(options, sideInputSources.get(0), executionContext); } else { List> shards = new ArrayList<>(); - for (com.google.api.services.dataflow.model.Source sideInputSource - : sideInputSources) { + for (Source sideInputSource : sideInputSources) { shards.add(readSideInputSource(options, sideInputSource, executionContext)); } return new ShardedIterable<>(shards); } } - static Iterable readSideInputSource( - PipelineOptions options, - com.google.api.services.dataflow.model.Source sideInputSource, - ExecutionContext executionContext) - throws Exception { - return new SourceIterable<>( - SourceFactory.create(options, sideInputSource, executionContext)); + static Iterable readSideInputSource(PipelineOptions options, Source sideInputSource, + ExecutionContext executionContext) throws Exception { + return new ReaderIterable<>(ReaderFactory.create(options, sideInputSource, executionContext)); } - static Object readSideInputValue(Map sideInputKind, - Iterable elements) + static Object readSideInputValue(Map sideInputKind, Iterable elements) throws Exception { String className = getString(sideInputKind, PropertyNames.OBJECT_TYPE_NAME); if (SINGLETON_KIND.equals(className)) { @@ -91,8 +81,7 @@ static Object readSideInputValue(Map sideInputKind, return elem; } } - throw new Exception( - "expecting a singleton side input to have a single value"); + throw new Exception("expecting a singleton side input to have a single value"); } else if (COLLECTION_KIND.equals(className)) { return elements; @@ -106,27 +95,27 @@ static Object readSideInputValue(Map sideInputKind, ///////////////////////////////////////////////////////////////////////////// - static class SourceIterable implements Iterable { - final Source source; + static class ReaderIterable implements Iterable { + final Reader reader; - public SourceIterable(Source source) { - this.source = source; + public ReaderIterable(Reader reader) { + this.reader = reader; } @Override public Iterator iterator() { try { - return new SourceIterator<>(source.iterator()); + return new ReaderIterator<>(reader.iterator()); } catch (Exception exn) { throw new RuntimeException(exn); } } } - static class SourceIterator implements Iterator { - final Source.SourceIterator iterator; + static class ReaderIterator implements Iterator { + final Reader.ReaderIterator iterator; - public SourceIterator(Source.SourceIterator iterator) { + public ReaderIterator(Reader.ReaderIterator iterator) { this.iterator = iterator; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java index 1e0c8aa234918..0f7ce18c480f5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java @@ -23,12 +23,13 @@ import com.google.api.services.dataflow.model.ApproximateProgress; import com.google.api.services.dataflow.model.Position; +import com.google.api.services.dataflow.model.Source; import com.google.api.services.dataflow.model.SourceMetadata; import com.google.api.services.dataflow.model.SourceOperationRequest; import com.google.api.services.dataflow.model.SourceOperationResponse; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.common.worker.CustomSourceFormat; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import java.util.HashMap; import java.util.Map; @@ -36,97 +37,81 @@ import javax.annotation.Nullable; /** - * Utilities for representing Source-specific objects + * Utilities for representing input-specific objects * using Dataflow model protos. */ public class SourceTranslationUtils { - public static Source.Progress cloudProgressToSourceProgress( + public static Reader.Progress cloudProgressToReaderProgress( @Nullable ApproximateProgress cloudProgress) { - return cloudProgress == null ? null - : new DataflowSourceProgress(cloudProgress); + return cloudProgress == null ? null : new DataflowReaderProgress(cloudProgress); } - public static Source.Position cloudPositionToSourcePosition( - @Nullable Position cloudPosition) { - return cloudPosition == null ? null - : new DataflowSourcePosition(cloudPosition); + public static Reader.Position cloudPositionToReaderPosition(@Nullable Position cloudPosition) { + return cloudPosition == null ? null : new DataflowReaderPosition(cloudPosition); } - public static CustomSourceFormat.SourceOperationRequest - cloudSourceOperationRequestToSourceOperationRequest( - @Nullable SourceOperationRequest request) { - return request == null ? null - : new DataflowSourceOperationRequest(request); + public static CustomSourceFormat.OperationRequest + cloudSourceOperationRequestToSourceOperationRequest( + @Nullable SourceOperationRequest request) { + return request == null ? null : new DataflowSourceOperationRequest(request); } - public static CustomSourceFormat.SourceOperationResponse - cloudSourceOperationResponseToSourceOperationResponse( - @Nullable SourceOperationResponse response) { - return response == null ? null - : new DataflowSourceOperationResponse(response); + public static CustomSourceFormat.OperationResponse + cloudSourceOperationResponseToSourceOperationResponse( + @Nullable SourceOperationResponse response) { + return response == null ? null : new DataflowSourceOperationResponse(response); } public static CustomSourceFormat.SourceSpec cloudSourceToSourceSpec( - @Nullable com.google.api.services.dataflow.model.Source cloudSource) { - return cloudSource == null ? null - : new DataflowSourceSpec(cloudSource); + @Nullable Source cloudSource) { + return cloudSource == null ? null : new DataflowSourceSpec(cloudSource); } public static ApproximateProgress sourceProgressToCloudProgress( - @Nullable Source.Progress sourceProgress) { - return sourceProgress == null ? null - : ((DataflowSourceProgress) sourceProgress).cloudProgress; + @Nullable Reader.Progress sourceProgress) { + return sourceProgress == null ? null : ((DataflowReaderProgress) sourceProgress).cloudProgress; } - public static Position sourcePositionToCloudPosition( - @Nullable Source.Position sourcePosition) { - return sourcePosition == null ? null - : ((DataflowSourcePosition) sourcePosition).cloudPosition; + public static Position sourcePositionToCloudPosition(@Nullable Reader.Position sourcePosition) { + return sourcePosition == null ? null : ((DataflowReaderPosition) sourcePosition).cloudPosition; } - public static SourceOperationRequest - sourceOperationRequestToCloudSourceOperationRequest( - @Nullable CustomSourceFormat.SourceOperationRequest request) { - return (request == null) ? null - : ((DataflowSourceOperationRequest) request).cloudRequest; + public static SourceOperationRequest sourceOperationRequestToCloudSourceOperationRequest( + @Nullable CustomSourceFormat.OperationRequest request) { + return (request == null) ? null : ((DataflowSourceOperationRequest) request).cloudRequest; } - public static SourceOperationResponse - sourceOperationResponseToCloudSourceOperationResponse( - @Nullable CustomSourceFormat.SourceOperationResponse response) { - return (response == null) ? null - : ((DataflowSourceOperationResponse) response).cloudResponse; + public static SourceOperationResponse sourceOperationResponseToCloudSourceOperationResponse( + @Nullable CustomSourceFormat.OperationResponse response) { + return (response == null) ? null : ((DataflowSourceOperationResponse) response).cloudResponse; } - public static com.google.api.services.dataflow.model.Source sourceSpecToCloudSource( - @Nullable CustomSourceFormat.SourceSpec spec) { - return (spec == null) ? null - : ((DataflowSourceSpec) spec).cloudSource; + public static Source sourceSpecToCloudSource(@Nullable CustomSourceFormat.SourceSpec spec) { + return (spec == null) ? null : ((DataflowSourceSpec) spec).cloudSource; } - static class DataflowSourceProgress implements Source.Progress { + static class DataflowReaderProgress implements Reader.Progress { public final ApproximateProgress cloudProgress; - public DataflowSourceProgress(ApproximateProgress cloudProgress) { + public DataflowReaderProgress(ApproximateProgress cloudProgress) { this.cloudProgress = cloudProgress; } } - static class DataflowSourcePosition implements Source.Position { + static class DataflowReaderPosition implements Reader.Position { public final Position cloudPosition; - public DataflowSourcePosition(Position cloudPosition) { + public DataflowReaderPosition(Position cloudPosition) { this.cloudPosition = cloudPosition; } } - static class DataflowSourceOperationRequest implements CustomSourceFormat.SourceOperationRequest { + static class DataflowSourceOperationRequest implements CustomSourceFormat.OperationRequest { public final SourceOperationRequest cloudRequest; public DataflowSourceOperationRequest(SourceOperationRequest cloudRequest) { this.cloudRequest = cloudRequest; } } - static class DataflowSourceOperationResponse - implements CustomSourceFormat.SourceOperationResponse { + static class DataflowSourceOperationResponse implements CustomSourceFormat.OperationResponse { public final SourceOperationResponse cloudResponse; public DataflowSourceOperationResponse(SourceOperationResponse cloudResponse) { this.cloudResponse = cloudResponse; @@ -134,16 +119,15 @@ public DataflowSourceOperationResponse(SourceOperationResponse cloudResponse) { } static class DataflowSourceSpec implements CustomSourceFormat.SourceSpec { - public final com.google.api.services.dataflow.model.Source cloudSource; - public DataflowSourceSpec(com.google.api.services.dataflow.model.Source cloudSource) { + public final Source cloudSource; + public DataflowSourceSpec(Source cloudSource) { this.cloudSource = cloudSource; } } // Represents a cloud Source as a dictionary for encoding inside the CUSTOM_SOURCE // property of CloudWorkflowStep.input. - public static Map cloudSourceToDictionary( - com.google.api.services.dataflow.model.Source source) { + public static Map cloudSourceToDictionary(Source source) { // Do not translate encoding - the source's encoding is translated elsewhere // to the step's output info. Map res = new HashMap<>(); @@ -159,28 +143,24 @@ public static Map cloudSourceToDictionary( return res; } - private static Map cloudSourceMetadataToDictionary( - SourceMetadata metadata) { + private static Map cloudSourceMetadataToDictionary(SourceMetadata metadata) { Map res = new HashMap<>(); if (metadata.getProducesSortedKeys() != null) { - addBoolean(res, PropertyNames.CUSTOM_SOURCE_PRODUCES_SORTED_KEYS, - metadata.getProducesSortedKeys()); + addBoolean( + res, PropertyNames.CUSTOM_SOURCE_PRODUCES_SORTED_KEYS, metadata.getProducesSortedKeys()); } if (metadata.getEstimatedSizeBytes() != null) { - addLong(res, PropertyNames.CUSTOM_SOURCE_ESTIMATED_SIZE_BYTES, - metadata.getEstimatedSizeBytes()); + addLong( + res, PropertyNames.CUSTOM_SOURCE_ESTIMATED_SIZE_BYTES, metadata.getEstimatedSizeBytes()); } if (metadata.getInfinite() != null) { - addBoolean(res, PropertyNames.CUSTOM_SOURCE_IS_INFINITE, - metadata.getInfinite()); + addBoolean(res, PropertyNames.CUSTOM_SOURCE_IS_INFINITE, metadata.getInfinite()); } return res; } - public static com.google.api.services.dataflow.model.Source dictionaryToCloudSource( - Map params) throws Exception { - com.google.api.services.dataflow.model.Source res = - new com.google.api.services.dataflow.model.Source(); + public static Source dictionaryToCloudSource(Map params) throws Exception { + Source res = new Source(); res.setSpec(getDictionary(params, PropertyNames.CUSTOM_SOURCE_SPEC)); // CUSTOM_SOURCE_METADATA and CUSTOM_SOURCE_DOES_NOT_NEED_SPLITTING do not have to be // translated, because they only make sense in cloud Source objects produced by the user. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReader.java similarity index 73% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSource.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReader.java index 5bbcba0e6b91d..f46eca2427356 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReader.java @@ -36,37 +36,27 @@ * * @param the type of the elements read from the source */ -public class TextSource extends FileBasedSource { +public class TextReader extends FileBasedReader { final boolean stripTrailingNewlines; - public TextSource(String filename, - boolean stripTrailingNewlines, - @Nullable Long startPosition, - @Nullable Long endPosition, - Coder coder) { - this(filename, stripTrailingNewlines, - startPosition, endPosition, coder, true); + public TextReader(String filename, boolean stripTrailingNewlines, @Nullable Long startPosition, + @Nullable Long endPosition, Coder coder) { + this(filename, stripTrailingNewlines, startPosition, endPosition, coder, true); } - protected TextSource(String filename, - boolean stripTrailingNewlines, - @Nullable Long startPosition, - @Nullable Long endPosition, - Coder coder, - boolean useDefaultBufferSize) { + protected TextReader(String filename, boolean stripTrailingNewlines, @Nullable Long startPosition, + @Nullable Long endPosition, Coder coder, boolean useDefaultBufferSize) { super(filename, startPosition, endPosition, coder, useDefaultBufferSize); this.stripTrailingNewlines = stripTrailingNewlines; } @Override - protected SourceIterator newSourceIteratorForRangeInFile( - IOChannelFactory factory, String oneFile, long startPosition, - @Nullable Long endPosition) - throws IOException { + protected ReaderIterator newReaderIteratorForRangeInFile(IOChannelFactory factory, + String oneFile, long startPosition, @Nullable Long endPosition) throws IOException { // Position before the first record, so we can find the record beginning. final long start = startPosition > 0 ? startPosition - 1 : 0; - TextFileIterator iterator = newSourceIteratorForRangeWithStrictStart( + TextFileIterator iterator = newReaderIteratorForRangeWithStrictStart( factory, oneFile, stripTrailingNewlines, start, endPosition); // Skip the initial record if start position was set. @@ -78,22 +68,18 @@ protected SourceIterator newSourceIteratorForRangeInFile( } @Override - protected SourceIterator newSourceIteratorForFiles( + protected ReaderIterator newReaderIteratorForFiles( IOChannelFactory factory, Collection files) throws IOException { if (files.size() == 1) { - return newSourceIteratorForFile( - factory, files.iterator().next(), stripTrailingNewlines); + return newReaderIteratorForFile(factory, files.iterator().next(), stripTrailingNewlines); } - return new TextFileMultiIterator( - factory, files.iterator(), stripTrailingNewlines); + return new TextFileMultiIterator(factory, files.iterator(), stripTrailingNewlines); } - private TextFileIterator newSourceIteratorForFile( - IOChannelFactory factory, String input, boolean stripTrailingNewlines) - throws IOException { - return newSourceIteratorForRangeWithStrictStart( - factory, input, stripTrailingNewlines, 0, null); + private TextFileIterator newReaderIteratorForFile( + IOChannelFactory factory, String input, boolean stripTrailingNewlines) throws IOException { + return newReaderIteratorForRangeWithStrictStart(factory, input, stripTrailingNewlines, 0, null); } /** @@ -102,36 +88,34 @@ private TextFileIterator newSourceIteratorForFile( * middle of a line (instead, the latter half that starts at * startOffset will be returned as the first element). */ - private TextFileIterator newSourceIteratorForRangeWithStrictStart( - IOChannelFactory factory, String input, boolean stripTrailingNewlines, - long startOffset, @Nullable Long endOffset) throws IOException { + private TextFileIterator newReaderIteratorForRangeWithStrictStart(IOChannelFactory factory, + String input, boolean stripTrailingNewlines, long startOffset, @Nullable Long endOffset) + throws IOException { ReadableByteChannel reader = factory.open(input); if (!(reader instanceof SeekableByteChannel)) { - throw new UnsupportedOperationException( - "Unable to seek in stream for " + input); + throw new UnsupportedOperationException("Unable to seek in stream for " + input); } SeekableByteChannel seeker = (SeekableByteChannel) reader; return new TextFileIterator( - new CopyableSeekableByteChannel(seeker), - stripTrailingNewlines, startOffset, endOffset); + new CopyableSeekableByteChannel(seeker), stripTrailingNewlines, startOffset, endOffset); } - class TextFileMultiIterator extends LazyMultiSourceIterator { + class TextFileMultiIterator extends LazyMultiReaderIterator { private final IOChannelFactory factory; private final boolean stripTrailingNewlines; - public TextFileMultiIterator(IOChannelFactory factory, - Iterator inputs, boolean stripTrailingNewlines) { + public TextFileMultiIterator( + IOChannelFactory factory, Iterator inputs, boolean stripTrailingNewlines) { super(inputs); this.factory = factory; this.stripTrailingNewlines = stripTrailingNewlines; } @Override - protected SourceIterator open(String input) throws IOException { - return newSourceIteratorForFile(factory, input, stripTrailingNewlines); + protected ReaderIterator open(String input) throws IOException { + return newReaderIteratorForFile(factory, input, stripTrailingNewlines); } } @@ -139,25 +123,20 @@ class TextFileIterator extends FileBasedIterator { private final boolean stripTrailingNewlines; private ScanState state; - TextFileIterator(CopyableSeekableByteChannel seeker, - boolean stripTrailingNewlines, - long startOffset, - @Nullable Long endOffset) throws IOException { + TextFileIterator(CopyableSeekableByteChannel seeker, boolean stripTrailingNewlines, + long startOffset, @Nullable Long endOffset) throws IOException { this(seeker, stripTrailingNewlines, startOffset, startOffset, endOffset, new ProgressTrackerGroup() { @Override protected void report(Integer lineLength) { notifyElementRead(lineLength.longValue()); } - }.start(), new ScanState(BUF_SIZE, !stripTrailingNewlines)); + }.start(), + new ScanState(BUF_SIZE, !stripTrailingNewlines)); } - private TextFileIterator(CopyableSeekableByteChannel seeker, - boolean stripTrailingNewlines, - long startOffset, - long offset, - @Nullable Long endOffset, - ProgressTracker tracker, + private TextFileIterator(CopyableSeekableByteChannel seeker, boolean stripTrailingNewlines, + long startOffset, long offset, @Nullable Long endOffset, ProgressTracker tracker, ScanState state) throws IOException { super(seeker, startOffset, offset, endOffset, tracker); @@ -166,17 +145,15 @@ private TextFileIterator(CopyableSeekableByteChannel seeker, } private TextFileIterator(TextFileIterator it) throws IOException { - this(it.seeker.copy(), it.stripTrailingNewlines, - /* Correctly adjust the start position of the seeker given - * that it may hold bytes that have been read and now reside - * in the read buffer (that is copied during cloning) */ - it.startOffset + it.state.totalBytesRead, - it.offset, - it.endOffset, it.tracker.copy(), it.state.copy()); + // Correctly adjust the start position of the seeker given + // that it may hold bytes that have been read and now reside + // in the read buffer (that is copied during cloning). + this(it.seeker.copy(), it.stripTrailingNewlines, it.startOffset + it.state.totalBytesRead, + it.offset, it.endOffset, it.tracker.copy(), it.state.copy()); } @Override - public SourceIterator copy() throws IOException { + public ReaderIterator copy() throws IOException { return new TextFileIterator(this); } @@ -193,8 +170,7 @@ public SourceIterator copy() throws IOException { * @throws IOException if an I/O error occurs */ @Override - protected ByteArrayOutputStream readElement() - throws IOException { + protected ByteArrayOutputStream readElement() throws IOException { ByteArrayOutputStream buffer = new ByteArrayOutputStream(BUF_SIZE); int charsConsumed = 0; @@ -233,7 +209,7 @@ protected ByteArrayOutputStream readElement() * being scanned. */ private static class ScanState { - private int start; // Valid bytes in buf start at this index + private int start; // Valid bytes in buf start at this index private int pos; // Where the separator is in the buf (if one was found) private int end; // the index of the end of bytes in buf private byte[] buf; @@ -251,16 +227,14 @@ public ScanState(int size, boolean keepNewlines) { } public ScanState copy() { - byte[] bufCopy = new byte[buf.length]; // copy :( + byte[] bufCopy = new byte[buf.length]; // copy :( System.arraycopy(buf, start, bufCopy, start, end - start); return new ScanState( - this.keepNewlines, this.start, this.pos, this.end, - bufCopy, this.lastByteRead, 0); + this.keepNewlines, this.start, this.pos, this.end, bufCopy, this.lastByteRead, 0); } - private ScanState( - boolean keepNewlines, int start, int pos, int end, - byte[] buf, byte lastByteRead, long totalBytesRead) { + private ScanState(boolean keepNewlines, int start, int pos, int end, byte[] buf, + byte lastByteRead, long totalBytesRead) { this.start = start; this.pos = pos; this.end = end; @@ -299,7 +273,7 @@ public int consumeUntilSeparator(ByteArrayOutputStream out) { if (separatorFound()) { int charsConsumed = (pos - start + 1); // The separator is consumed copyToOutputBuffer(out); - start = pos + 1; // skip the separator + start = pos + 1; // skip the separator return charsConsumed; } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactory.java similarity index 64% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactory.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactory.java index a15c2d505c47f..4ec8973b87a12 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactory.java @@ -29,46 +29,35 @@ import com.google.cloud.dataflow.sdk.util.Serializer; /** - * Creates a TextSource from a CloudObject spec. + * Creates a TextReader from a CloudObject spec. */ -public class TextSourceFactory { +public class TextReaderFactory { // Do not instantiate. - private TextSourceFactory() {} + private TextReaderFactory() {} - public static TextSource create(PipelineOptions options, - CloudObject spec, - Coder coder, - ExecutionContext executionContext) - throws Exception { + public static TextReader create(PipelineOptions options, CloudObject spec, Coder coder, + ExecutionContext executionContext) throws Exception { return create(spec, coder); } - static TextSource create(CloudObject spec, - Coder coder) - throws Exception { + static TextReader create(CloudObject spec, Coder coder) throws Exception { return create(spec, coder, true); } - public static TextSource create(Source spec) - throws Exception { + public static TextReader create(Source spec) throws Exception { return create( - CloudObject.fromSpec(spec.getSpec()), - Serializer.deserialize(spec.getCodec(), Coder.class)); + CloudObject.fromSpec(spec.getSpec()), Serializer.deserialize(spec.getCodec(), Coder.class)); } - static TextSource create(CloudObject spec, - Coder coder, - boolean useDefaultBufferSize) throws Exception { + static TextReader create(CloudObject spec, Coder coder, boolean useDefaultBufferSize) + throws Exception { String filenameOrPattern = getString(spec, PropertyNames.FILENAME, null); if (filenameOrPattern == null) { filenameOrPattern = getString(spec, PropertyNames.FILEPATTERN, null); } - return new TextSource<>( - filenameOrPattern, + return new TextReader<>(filenameOrPattern, getBoolean(spec, PropertyNames.STRIP_TRAILING_NEWLINES, true), getLong(spec, PropertyNames.START_OFFSET, null), - getLong(spec, PropertyNames.END_OFFSET, null), - coder, - useDefaultBufferSize); + getLong(spec, PropertyNames.END_OFFSET, null), coder, useDefaultBufferSize); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleReader.java similarity index 75% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSource.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleReader.java index d7d0cf7cf841e..4291d94cd974d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleReader.java @@ -21,9 +21,9 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.common.worker.BatchingShuffleEntryReader; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntryReader; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; import java.io.IOException; import java.util.Iterator; @@ -36,17 +36,14 @@ * * @param the type of the elements read from the source */ -public class UngroupedShuffleSource extends Source { +public class UngroupedShuffleReader extends Reader { final byte[] shuffleReaderConfig; final String startShufflePosition; final String stopShufflePosition; final Coder coder; - public UngroupedShuffleSource(PipelineOptions options, - byte[] shuffleReaderConfig, - @Nullable String startShufflePosition, - @Nullable String stopShufflePosition, - Coder coder) { + public UngroupedShuffleReader(PipelineOptions options, byte[] shuffleReaderConfig, + @Nullable String startShufflePosition, @Nullable String stopShufflePosition, Coder coder) { this.shuffleReaderConfig = shuffleReaderConfig; this.startShufflePosition = startShufflePosition; this.stopShufflePosition = stopShufflePosition; @@ -54,26 +51,24 @@ public UngroupedShuffleSource(PipelineOptions options, } @Override - public SourceIterator iterator() throws IOException { + public ReaderIterator iterator() throws IOException { Preconditions.checkArgument(shuffleReaderConfig != null); return iterator(new BatchingShuffleEntryReader( - new ChunkingShuffleBatchReader(new ApplianceShuffleReader( - shuffleReaderConfig)))); + new ChunkingShuffleBatchReader(new ApplianceShuffleReader(shuffleReaderConfig)))); } - SourceIterator iterator(ShuffleEntryReader reader) throws IOException { - return new UngroupedShuffleSourceIterator(reader); + ReaderIterator iterator(ShuffleEntryReader reader) throws IOException { + return new UngroupedShuffleReaderIterator(reader); } /** - * A SourceIterator that reads from a ShuffleEntryReader and extracts + * A ReaderIterator that reads from a ShuffleEntryReader and extracts * just the values. */ - class UngroupedShuffleSourceIterator extends AbstractSourceIterator { + class UngroupedShuffleReaderIterator extends AbstractReaderIterator { Iterator iterator; - UngroupedShuffleSourceIterator(ShuffleEntryReader reader) - throws IOException { + UngroupedShuffleReaderIterator(ShuffleEntryReader reader) throws IOException { this.iterator = reader.read( ByteArrayShufflePosition.fromBase64(startShufflePosition), ByteArrayShufflePosition.fromBase64(stopShufflePosition)); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleReaderFactory.java similarity index 70% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceFactory.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleReaderFactory.java index adff71226d6b8..3237bb8349bb2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleReaderFactory.java @@ -26,31 +26,22 @@ import com.google.cloud.dataflow.sdk.util.PropertyNames; /** - * Creates an UngroupedShuffleSource from a CloudObject spec. + * Creates an UngroupedShuffleReader from a CloudObject spec. */ -public class UngroupedShuffleSourceFactory { +public class UngroupedShuffleReaderFactory { // Do not instantiate. - private UngroupedShuffleSourceFactory() {} + private UngroupedShuffleReaderFactory() {} - public static UngroupedShuffleSource create( - PipelineOptions options, - CloudObject spec, - Coder coder, - ExecutionContext executionContext) - throws Exception { + public static UngroupedShuffleReader create(PipelineOptions options, CloudObject spec, + Coder coder, ExecutionContext executionContext) throws Exception { return create(options, spec, coder); } - static UngroupedShuffleSource create( - PipelineOptions options, - CloudObject spec, - Coder coder) - throws Exception { - return new UngroupedShuffleSource<>( - options, + static UngroupedShuffleReader create( + PipelineOptions options, CloudObject spec, Coder coder) throws Exception { + return new UngroupedShuffleReader<>(options, decodeBase64(getString(spec, PropertyNames.SHUFFLE_READER_CONFIG)), getString(spec, PropertyNames.START_SHUFFLE_POSITION, null), - getString(spec, PropertyNames.END_SHUFFLE_POSITION, null), - coder); + getString(spec, PropertyNames.END_SHUFFLE_POSITION, null), coder); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtils.java index 7d97948af437e..bcb4d343a35a4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtils.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtils.java @@ -16,17 +16,16 @@ package com.google.cloud.dataflow.sdk.util; -import com.google.cloud.dataflow.sdk.runners.worker.SourceFactory; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.api.services.dataflow.model.Source; +import com.google.cloud.dataflow.sdk.runners.worker.ReaderFactory; -import java.io.IOException; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; /** - * Utilities for working with Source Dataflow API definitions and {@link Source} + * Utilities for working with Source Dataflow API definitions and + * {@link com.google.cloud.dataflow.sdk.util.common.worker.Reader} * objects. */ public class CloudSourceUtils { @@ -35,8 +34,7 @@ public class CloudSourceUtils { * On conflict for a parameter name, values in {@code spec} override values in {@code baseSpecs}, * and later values in {@code baseSpecs} override earlier ones. */ - public static com.google.api.services.dataflow.model.Source - flattenBaseSpecs(com.google.api.services.dataflow.model.Source source) { + public static Source flattenBaseSpecs(Source source) { if (source.getBaseSpecs() == null) { return source; } @@ -46,33 +44,19 @@ public class CloudSourceUtils { } params.putAll(source.getSpec()); - com.google.api.services.dataflow.model.Source result = source.clone(); + Source result = source.clone(); result.setSpec(params); result.setBaseSpecs(null); return result; } - /** Reads all elements from the given {@link Source}. */ - public static List readElemsFromSource(Source source) { - List elems = new ArrayList<>(); - try (Source.SourceIterator it = source.iterator()) { - while (it.hasNext()) { - elems.add(it.next()); - } - } catch (IOException e) { - throw new RuntimeException("Failed to read from source: " + source, e); - } - return elems; - } - /** - * Creates a {@link Source} from the given Dataflow Source API definition and - * reads all elements from it. + * Creates a {@link com.google.cloud.dataflow.sdk.util.common.worker.Reader} + * from the given Dataflow Source API definition and reads all elements from it. */ - public static List readElemsFromSource( - com.google.api.services.dataflow.model.Source source) { + public static List readElemsFromSource(Source source) { try { - return readElemsFromSource(SourceFactory.create(null, source, null)); + return ReaderUtils.readElemsFromReader(ReaderFactory.create(null, source, null)); } catch (Exception e) { throw new RuntimeException("Failed to read from source: " + source.toString(), e); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReaderUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReaderUtils.java new file mode 100644 index 0000000000000..4c2f9bf35380a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReaderUtils.java @@ -0,0 +1,44 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Utilities for working with {@link com.google.cloud.dataflow.sdk.util.common.worker.Reader} + * objects. + */ +public class ReaderUtils { + /** + * Reads all elements from the given + * {@link com.google.cloud.dataflow.sdk.util.common.worker.Reader}. + */ + public static List readElemsFromReader(Reader reader) { + List elems = new ArrayList<>(); + try (Reader.ReaderIterator it = reader.iterator()) { + while (it.hasNext()) { + elems.add(it.next()); + } + } catch (IOException e) { + throw new RuntimeException("Failed to read from source: " + reader, e); + } + return elems; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CustomSourceFormat.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CustomSourceFormat.java index 4fc67d60f3c3f..12cdf30ac468a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CustomSourceFormat.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CustomSourceFormat.java @@ -25,8 +25,7 @@ public interface CustomSourceFormat { * Performs an operation on the specification of a source. * See {@code SourceOperationRequest} for details. */ - public SourceOperationResponse performSourceOperation(SourceOperationRequest operation) - throws Exception; + public OperationResponse performSourceOperation(OperationRequest operation) throws Exception; /** * A representation of an operation on the specification of a source, @@ -38,24 +37,21 @@ public SourceOperationResponse performSourceOperation(SourceOperationRequest ope * about the implementation, and so the concrete Source subclasses used * by a tool-specific framework should match. */ - public interface SourceOperationRequest { - } + public interface OperationRequest {} /** * A representation of the result of a SourceOperationRequest. * - *

See the comment on {@link SourceOperationRequest} for how instances of this + *

See the comment on {@link OperationRequest} for how instances of this * interface are used by the rest of the framework. */ - public interface SourceOperationResponse { - } + public interface OperationResponse {} /** * A representation of a specification of a source. * - *

See the comment on {@link SourceOperationRequest} for how instances of this + *

See the comment on {@link OperationRequest} for how instances of this * interface are used by the rest of the framework. */ - public interface SourceSpec { - } + public interface SourceSpec {} } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java index 45d5e8c6715e3..793343d6d1bb4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java @@ -28,8 +28,7 @@ * An executor for a map task, defined by a list of Operations. */ public class MapTaskExecutor extends WorkExecutor { - private static final Logger LOG = - LoggerFactory.getLogger(MapTaskExecutor.class); + private static final Logger LOG = LoggerFactory.getLogger(MapTaskExecutor.class); /** The operations in the map task, in execution order. */ public final List operations; @@ -45,9 +44,8 @@ public class MapTaskExecutor extends WorkExecutor { * operations, which may get extended during execution * @param stateSampler a state sampler for tracking where time is being spent */ - public MapTaskExecutor(List operations, - CounterSet counters, - StateSampler stateSampler) { + public MapTaskExecutor( + List operations, CounterSet counters, StateSampler stateSampler) { super(counters); this.operations = operations; this.stateSampler = stateSampler; @@ -62,8 +60,7 @@ public void execute() throws Exception { // Starting a root operation such as a ReadOperation does the work // of processing the input dataset. LOG.debug("starting operations"); - ListIterator iterator = - operations.listIterator(operations.size()); + ListIterator iterator = operations.listIterator(operations.size()); while (iterator.hasPrevious()) { Operation op = iterator.previous(); op.start(); @@ -83,26 +80,24 @@ public void execute() throws Exception { } @Override - public Source.Progress getWorkerProgress() throws Exception { + public Reader.Progress getWorkerProgress() throws Exception { return getReadOperation().getProgress(); } @Override - public Source.Position proposeStopPosition( - Source.Progress proposedStopPosition) throws Exception { + public Reader.Position proposeStopPosition(Reader.Progress proposedStopPosition) + throws Exception { return getReadOperation().proposeStopPosition(proposedStopPosition); } ReadOperation getReadOperation() throws Exception { if (operations == null || operations.isEmpty()) { - throw new IllegalStateException( - "Map task has no operation."); + throw new IllegalStateException("Map task has no operation."); } Operation readOperation = operations.get(0); if (!(readOperation instanceof ReadOperation)) { - throw new IllegalStateException( - "First operation in the map task is not a ReadOperation."); + throw new IllegalStateException("First operation in the map task is not a ReadOperation."); } return (ReadOperation) readOperation; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java index f4caef51d0d1e..acd5d6468b691 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java @@ -41,20 +41,20 @@ public class ReadOperation extends Operation { private static final Logger LOG = LoggerFactory.getLogger(ReadOperation.class); private static final long DEFAULT_PROGRESS_UPDATE_PERIOD_MS = TimeUnit.SECONDS.toMillis(1); - /** The Source this operation reads from. */ - public final Source source; + /** The Reader this operation reads from. */ + public final Reader reader; /** The total byte counter for all data read by this operation. */ final Counter byteCount; - /** StateSampler state for advancing the SourceIterator. */ + /** StateSampler state for advancing the ReaderIterator. */ private final int readState; /** - * The Source's reader this operation reads from, created by start(). + * The Reader's iterator this operation reads from, created by start(). * Guarded by sourceIteratorLock. */ - volatile Source.SourceIterator sourceIterator = null; + volatile Reader.ReaderIterator readerIterator = null; private final Object sourceIteratorLock = new Object(); /** @@ -64,7 +64,7 @@ public class ReadOperation extends Operation { * wait for a read to complete (which can take an unbounded time, delay a worker progress update, * and cause lease expiration and all sorts of trouble). */ - private AtomicReference progress = new AtomicReference<>(); + private AtomicReference progress = new AtomicReference<>(); /** * On every iteration of the read loop, "progress" is fetched from sourceIterator if requested. @@ -78,21 +78,21 @@ public class ReadOperation extends Operation { private AtomicBoolean isProgressUpdateRequested = new AtomicBoolean(true); - public ReadOperation(String operationName, Source source, OutputReceiver[] receivers, + public ReadOperation(String operationName, Reader reader, OutputReceiver[] receivers, String counterPrefix, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) { super(operationName, receivers, counterPrefix, addCounterMutator, stateSampler); - this.source = source; + this.reader = reader; this.byteCount = addCounterMutator.addCounter( Counter.longs(bytesCounterName(counterPrefix, operationName), SUM)); readState = stateSampler.stateForName(operationName + "-read"); } /** Invoked by tests. */ - ReadOperation(Source source, OutputReceiver outputReceiver, String counterPrefix, + ReadOperation(Reader reader, OutputReceiver outputReceiver, String counterPrefix, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) { - this("ReadOperation", source, new OutputReceiver[] {outputReceiver}, counterPrefix, - addCounterMutator, stateSampler); + this("ReadOperation", reader, new OutputReceiver[] {outputReceiver}, counterPrefix, + addCounterMutator, stateSampler); } /** @@ -107,8 +107,8 @@ protected String bytesCounterName(String counterPrefix, String operationName) { return operationName + "-ByteCount"; } - public Source getSource() { - return source; + public Reader getReader() { + return reader; } @Override @@ -127,12 +127,12 @@ protected void runReadLoop() throws Exception { return; } - source.addObserver(new SourceObserver()); + reader.addObserver(new ReaderObserver()); try (StateSampler.ScopedState process = stateSampler.scopedState(processState)) { assert process != null; synchronized (sourceIteratorLock) { - sourceIterator = source.iterator(); + readerIterator = reader.iterator(); } // TODO: Consider using the ExecutorService from PipelineOptions instead. @@ -157,7 +157,7 @@ public void run() { try { // Force a progress update at the beginning and at the end. synchronized (sourceIteratorLock) { - progress.set(sourceIterator.getProgress()); + progress.set(readerIterator.getProgress()); } while (true) { Object value; @@ -166,24 +166,24 @@ public void run() { try (StateSampler.ScopedState read = stateSampler.scopedState(readState)) { assert read != null; synchronized (sourceIteratorLock) { - if (!sourceIterator.hasNext()) { + if (!readerIterator.hasNext()) { break; } - value = sourceIterator.next(); + value = readerIterator.next(); if (isProgressUpdateRequested.getAndSet(false) || progressUpdatePeriodMs == 0) { - progress.set(sourceIterator.getProgress()); + progress.set(readerIterator.getProgress()); } } } receiver.process(value); } synchronized (sourceIteratorLock) { - progress.set(sourceIterator.getProgress()); + progress.set(readerIterator.getProgress()); } } finally { synchronized (sourceIteratorLock) { - sourceIterator.close(); + readerIterator.close(); } if (progressUpdatePeriodMs != 0) { updateRequester.interrupt(); @@ -200,24 +200,24 @@ public void run() { * @return the task progress, or {@code null} if the source iterator has not * been initialized */ - public Source.Progress getProgress() { + public Reader.Progress getProgress() { return progress.get(); } /** - * Relays the request to update the stop position to {@code SourceIterator}. + * Relays the request to update the stop position to {@code ReaderIterator}. * * @param proposedStopPosition the proposed stop position - * @return the new stop position updated in {@code SourceIterator}, or + * @return the new stop position updated in {@code ReaderIterator}, or * {@code null} if the source iterator has not been initialized */ - public Source.Position proposeStopPosition(Source.Progress proposedStopPosition) { + public Reader.Position proposeStopPosition(Reader.Progress proposedStopPosition) { synchronized (sourceIteratorLock) { - if (sourceIterator == null) { + if (readerIterator == null) { LOG.warn("Iterator has not been initialized, returning null stop position."); return null; } - return sourceIterator.updateStopPosition(proposedStopPosition); + return readerIterator.updateStopPosition(proposedStopPosition); } } @@ -226,10 +226,10 @@ public Source.Position proposeStopPosition(Source.Progress proposedStopPosition) * an element, update() gets called with the byte size of the element, which * gets added up into the ReadOperation's byte counter. */ - private class SourceObserver implements Observer { + private class ReaderObserver implements Observer { @Override public void update(Observable obs, Object obj) { - Preconditions.checkArgument(obs == source, "unexpected observable" + obs); + Preconditions.checkArgument(obs == reader, "unexpected observable" + obs); Preconditions.checkArgument(obj instanceof Long, "unexpected parameter object: " + obj); byteCount.addValue((long) obj); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Source.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java similarity index 78% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Source.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java index d50b93dc54193..2ecef5b4cc869 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Source.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java @@ -21,26 +21,26 @@ import java.util.Observable; /** - * Abstract base class for Sources. + * Abstract base class for readers. * *

A Source is read from by getting an Iterator-like value and * iterating through it. * * @param the type of the elements read from the source */ -public abstract class Source extends Observable { +public abstract class Reader extends Observable { /** - * Returns a SourceIterator that allows reading from this source. + * Returns a ReaderIterator that allows reading from this source. */ - public abstract SourceIterator iterator() throws IOException; + public abstract ReaderIterator iterator() throws IOException; /** - * A stateful iterator over the data in a Source. + * A stateful iterator over the data in a Reader. */ - public interface SourceIterator extends AutoCloseable { + public interface ReaderIterator extends AutoCloseable { /** * Returns whether the source has any more elements. Some sources, - * such as GroupingShuffleSource, invalidate the return value of + * such as GroupingShuffleReader, invalidate the return value of * the previous next() call during the call to hasNext(). */ public boolean hasNext() throws IOException; @@ -53,13 +53,13 @@ public interface SourceIterator extends AutoCloseable { public T next() throws IOException; /** - * Copies the current SourceIterator. + * Copies the current ReaderIterator. * * @throws UnsupportedOperationException if the particular implementation * does not support copy * @throws IOException if copying the iterator involves IO that fails */ - public SourceIterator copy() throws IOException; + public ReaderIterator copy() throws IOException; @Override public void close() throws IOException; @@ -80,7 +80,7 @@ public interface SourceIterator extends AutoCloseable { * position and returns the actual new stop position. * *

If the source finds the proposed one is not a convenient position to - * stop, it can pick a different stop position. The {@code SourceIterator} + * stop, it can pick a different stop position. The {@code ReaderIterator} * should start returning {@code false} from {@code hasNext()} once it has * passed its stop position. Subsequent stop position updates must be in * non-increasing order within a task. @@ -96,11 +96,10 @@ public interface SourceIterator extends AutoCloseable { public Position updateStopPosition(Progress proposedStopPosition); } - /** An abstract base class for SourceIterator implementations. */ - public abstract static class AbstractSourceIterator - implements SourceIterator { + /** An abstract base class for ReaderIterator implementations. */ + public abstract static class AbstractReaderIterator implements ReaderIterator { @Override - public SourceIterator copy() throws IOException { + public ReaderIterator copy() throws IOException { throw new UnsupportedOperationException(); } @@ -121,32 +120,30 @@ public Position updateStopPosition(Progress proposedStopPosition) { } /** - * A representation of how far a {@code SourceIterator} is through a - * {@code Source}. + * A representation of how far a {@code ReaderIterator} is through a + * {@code Reader}. * *

The common worker framework does not interpret instances of * this interface. But a tool-specific framework can make assumptions - * about the implementation, and so the concrete Source subclasses used + * about the implementation, and so the concrete Reader subclasses used * by a tool-specific framework should match. */ - public interface Progress { - } + public interface Progress {} /** * A representation of a position in an iteration through a - * {@code Source}. + * {@code Reader}. * *

See the comment on {@link Progress} for how instances of this * interface are used by the rest of the framework. */ - public interface Position { - } + public interface Position {} /** * Utility method to notify observers about a new element, which has - * been read by this Source, and its size in bytes. Normally, there + * been read by this Reader, and its size in bytes. Normally, there * is only one observer, which is a ReadOperation that encapsules - * this Source. Derived classes must call this method whenever they + * this Reader. Derived classes must call this method whenever they * read additional data, even if that element may never be returned * from the corresponding source iterator. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java index c4a603f70caad..18ab4762d171d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java @@ -49,8 +49,7 @@ public abstract class WorkExecutor implements AutoCloseable { */ public WorkExecutor(CounterSet outputCounters) { this.outputCounters = outputCounters; - this.os = - (OperatingSystemMXBean) ManagementFactory.getOperatingSystemMXBean(); + this.os = (OperatingSystemMXBean) ManagementFactory.getOperatingSystemMXBean(); } /** @@ -78,7 +77,7 @@ public Collection> getOutputMetrics() { /** * Returns the worker's current progress. */ - public Source.Progress getWorkerProgress() throws Exception { + public Reader.Progress getWorkerProgress() throws Exception { // By default, return null indicating worker progress not available. return null; } @@ -87,8 +86,8 @@ public Source.Progress getWorkerProgress() throws Exception { * Proposes that the worker changes the stop position for the current work. * Returns the new position if accepted, otherwise {@code null}. */ - public Source.Position proposeStopPosition( - Source.Progress proposedStopPosition) throws Exception { + public Reader.Position proposeStopPosition(Reader.Progress proposedStopPosition) + throws Exception { // By default, returns null indicating that no task splitting happens. return null; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java index c5222eb04a2f1..b50afb4b5f886 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java @@ -49,15 +49,15 @@ public abstract class WorkProgressUpdater { private static final long DEFAULT_LEASE_DURATION_MILLIS = 3 * 60 * 1000; /** The lease renewal RPC latency margin. */ - private static final long LEASE_RENEWAL_LATENCY_MARGIN = Long.valueOf( - System.getProperty("worker_lease_renewal_latency_margin", "5000")); + private static final long LEASE_RENEWAL_LATENCY_MARGIN = + Long.valueOf(System.getProperty("worker_lease_renewal_latency_margin", "5000")); /** * The minimum period between two consecutive progress updates. Ensures the * {@link WorkProgressUpdater} does not generate update storms. */ - private static final long MIN_REPORTING_INTERVAL_MILLIS = Long.valueOf( - System.getProperty("minimum_worker_update_interval_millis", "5000")); + private static final long MIN_REPORTING_INTERVAL_MILLIS = + Long.valueOf(System.getProperty("minimum_worker_update_interval_millis", "5000")); /** * The maximum period between two consecutive progress updates. Ensures the @@ -87,15 +87,12 @@ public abstract class WorkProgressUpdater { * we'll send the {@code null} as a stop position update, which is a no-op * for the service. */ - protected Source.Position stopPositionToService; + protected Reader.Position stopPositionToService; public WorkProgressUpdater(WorkExecutor worker) { this.worker = worker; this.executor = Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("WorkProgressUpdater-%d") - .build()); + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("WorkProgressUpdater-%d").build()); } /** @@ -104,10 +101,9 @@ public WorkProgressUpdater(WorkExecutor worker) { public void startReportingProgress() { // Send the initial work progress report half-way through the lease // expiration. Subsequent intervals adapt to hints from the service. - long leaseRemainingTime = - leaseRemainingTime(getWorkUnitLeaseExpirationTimestamp()); - progressReportIntervalMs = nextProgressReportInterval( - leaseRemainingTime / 2, leaseRemainingTime); + long leaseRemainingTime = leaseRemainingTime(getWorkUnitLeaseExpirationTimestamp()); + progressReportIntervalMs = + nextProgressReportInterval(leaseRemainingTime / 2, leaseRemainingTime); requestedLeaseDurationMs = DEFAULT_LEASE_DURATION_MILLIS; LOG.info("Started reporting progress for work item: {}", workString()); @@ -131,7 +127,7 @@ public void stopReportingProgress() throws Exception { // We send a final progress report in case there was an unreported stop position update. if (stopPositionToService != null) { LOG.info("Sending final progress update with unreported stop position."); - reportProgressHelper(); // This call can fail with an exception + reportProgressHelper(); // This call can fail with an exception } LOG.info("Stopped reporting progress for work item: {}", workString()); @@ -148,19 +144,19 @@ public void stopReportingProgress() throws Exception { * @param leaseRemainingTime milliseconds left before the work lease expires * @return the time in milliseconds before sending the next progress update */ - protected static long nextProgressReportInterval(long suggestedInterval, - long leaseRemainingTime) { + protected static long nextProgressReportInterval( + long suggestedInterval, long leaseRemainingTime) { // Sanitize input in case we get a negative suggested time interval. suggestedInterval = Math.max(0, suggestedInterval); // Try to send the next progress update before the next lease expiration // allowing some RPC latency margin. - suggestedInterval = Math.min(suggestedInterval, - leaseRemainingTime - LEASE_RENEWAL_LATENCY_MARGIN); + suggestedInterval = + Math.min(suggestedInterval, leaseRemainingTime - LEASE_RENEWAL_LATENCY_MARGIN); // Bound reporting interval to avoid staleness and progress update storms. - return Math.min(Math.max(MIN_REPORTING_INTERVAL_MILLIS, suggestedInterval), - MAX_REPORTING_INTERVAL_MILLIS); + return Math.min( + Math.max(MIN_REPORTING_INTERVAL_MILLIS, suggestedInterval), MAX_REPORTING_INTERVAL_MILLIS); } /** @@ -181,7 +177,8 @@ public void run() { reportProgress(); } } - }, progressReportIntervalMs, TimeUnit.MILLISECONDS); + }, + progressReportIntervalMs, TimeUnit.MILLISECONDS); LOG.debug("Next work progress update for work item {} scheduled to occur in {} ms.", workString(), progressReportIntervalMs); } @@ -211,13 +208,13 @@ protected long leaseRemainingTime(long leaseExpirationTimestamp) { LOG.debug("Lease remaining time for {} is 0 ms.", workString()); return 0; } - LOG.debug("Lease remaining time for {} is {} ms.", - workString(), leaseExpirationTimestamp - now); + LOG.debug( + "Lease remaining time for {} is {} ms.", workString(), leaseExpirationTimestamp - now); return leaseExpirationTimestamp - now; } // Visible for testing. - public Source.Position getStopPosition() { + public Reader.Position getStopPosition() { return stopPositionToService; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java index fa09bd09106af..e9a6cbde3773f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java @@ -42,7 +42,7 @@ * an unbounded {@code PCollection}. * *

Each element in a {@code PCollection} may have an associated implicit - * timestamp. Sources assign timestamps to elements when they create + * timestamp. Readers assign timestamps to elements when they create * {@code PCollection}s, and other {@code PTransform}s propagate these * timestamps from their input to their output. For example, PubsubIO.Read * assigns pubsub message timestamps to elements, and TextIO.Read assigns @@ -150,8 +150,7 @@ public PCollection setOrdered(boolean isOrdered) { if (this.isOrdered != isOrdered) { if (isFinishedSpecifyingInternal()) { throw new IllegalStateException( - "cannot change the orderedness of " + this + - " once it's been used"); + "cannot change the orderedness of " + this + " once it's been used"); } this.isOrdered = isOrdered; } @@ -162,8 +161,7 @@ public PCollection setOrdered(boolean isOrdered) { * Applies the given PTransform to this input PCollection, and * returns the PTransform's Output. */ - public Output apply( - PTransform, Output> t) { + public Output apply(PTransform, Output> t) { return Pipeline.applyTransform(this, t); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteReaderTest.java similarity index 74% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSourceTest.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteReaderTest.java index e6bfffdcb68ab..f9739dc2d6e8e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteReaderTest.java @@ -24,7 +24,7 @@ import com.google.cloud.dataflow.sdk.util.IOChannelUtils; import com.google.cloud.dataflow.sdk.util.MimeTypes; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.apache.avro.Schema; import org.apache.avro.file.DataFileWriter; @@ -49,23 +49,21 @@ import javax.annotation.Nullable; /** - * Tests for AvroByteSource. + * Tests for AvroByteReader. */ @RunWith(JUnit4.class) -public class AvroByteSourceTest { +public class AvroByteReaderTest { @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); - private void runTestRead(List> elemsList, - Coder coder, - boolean requireExactMatch) + private void runTestRead(List> elemsList, Coder coder, boolean requireExactMatch) throws Exception { File tmpFile = tmpFolder.newFile("file.avro"); String filename = tmpFile.getPath(); // Write the data. - OutputStream outStream = Channels.newOutputStream( - IOChannelUtils.create(filename, MimeTypes.BINARY)); + OutputStream outStream = + Channels.newOutputStream(IOChannelUtils.create(filename, MimeTypes.BINARY)); Schema schema = Schema.create(Schema.Type.BYTES); DatumWriter datumWriter = new GenericDatumWriter<>(schema); DataFileWriter fileWriter = new DataFileWriter<>(datumWriter); @@ -126,19 +124,14 @@ private void runTestRead(List> elemsList, Assert.assertEquals(expectedSizes, actualSizes); } - private List readElems(String filename, - @Nullable Long startOffset, - @Nullable Long endOffset, - Coder coder, - List actualSizes) - throws Exception { - AvroByteSource avroSource = - new AvroByteSource<>(filename, startOffset, endOffset, coder); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(avroSource, actualSizes); + private List readElems(String filename, @Nullable Long startOffset, + @Nullable Long endOffset, Coder coder, List actualSizes) throws Exception { + AvroByteReader avroReader = new AvroByteReader<>(filename, startOffset, endOffset, coder); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(avroReader, actualSizes); List actualElems = new ArrayList<>(); - try (Source.SourceIterator iterator = avroSource.iterator()) { + try (Reader.ReaderIterator iterator = avroReader.iterator()) { while (iterator.hasNext()) { actualElems.add(iterator.next()); } @@ -148,21 +141,18 @@ private List readElems(String filename, @Test public void testRead() throws Exception { - runTestRead(Collections.singletonList(TestUtils.INTS), - BigEndianIntegerCoder.of(), - true /* require exact match */); + runTestRead(Collections.singletonList(TestUtils.INTS), BigEndianIntegerCoder.of(), + true/* require exact match */); } @Test public void testReadEmpty() throws Exception { - runTestRead(Collections.singletonList(TestUtils.NO_INTS), - BigEndianIntegerCoder.of(), - true /* require exact match */); + runTestRead(Collections.singletonList(TestUtils.NO_INTS), BigEndianIntegerCoder.of(), + true/* require exact match */); } - private List> generateInputBlocks(int numBlocks, - int blockSizeBytes, - int averageLineSizeBytes) { + private List> generateInputBlocks( + int numBlocks, int blockSizeBytes, int averageLineSizeBytes) { Random random = new Random(0); List> blocks = new ArrayList<>(numBlocks); for (int blockNum = 0; blockNum < numBlocks; blockNum++) { @@ -183,16 +173,13 @@ private List> generateInputBlocks(int numBlocks, @Test public void testReadSmallRanges() throws Exception { - runTestRead(generateInputBlocks(3, 50, 5), - StringUtf8Coder.of(), - true /* require exact match */); + runTestRead(generateInputBlocks(3, 50, 5), StringUtf8Coder.of(), true/* require exact match */); } @Test public void testReadBigRanges() throws Exception { - runTestRead(generateInputBlocks(10, 128 * 1024, 100), - StringUtf8Coder.of(), - false /* don't require exact match */); + runTestRead(generateInputBlocks(10, 128 * 1024, 100), StringUtf8Coder.of(), + false/* don't require exact match */); } // TODO: sharded filenames diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSinkTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSinkTest.java index 3c7f29b40a8c1..83366800389e6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSinkTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroByteSinkTest.java @@ -65,18 +65,16 @@ void runTestWriteFile(List elems, Coder coder) throws Exception { // Read back the file. - SeekableByteChannel inChannel = (SeekableByteChannel) - IOChannelUtils.getFactory(filename).open(filename); + SeekableByteChannel inChannel = + (SeekableByteChannel) IOChannelUtils.getFactory(filename).open(filename); - SeekableInput seekableInput = - new AvroSource.SeekableByteChannelInput(inChannel); + SeekableInput seekableInput = new AvroReader.SeekableByteChannelInput(inChannel); Schema schema = Schema.create(Schema.Type.BYTES); DatumReader datumReader = new GenericDatumReader<>(schema); - DataFileReader fileReader = new DataFileReader<>( - seekableInput, datumReader); + DataFileReader fileReader = new DataFileReader<>(seekableInput, datumReader); List actual = new ArrayList<>(); List expectedSizes = new ArrayList<>(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactoryTest.java new file mode 100644 index 0000000000000..a71cc8ea510a2 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactoryTest.java @@ -0,0 +1,107 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.addLong; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.api.services.dataflow.model.Source; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; + +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import javax.annotation.Nullable; + +/** + * Tests for AvroReaderFactory. + */ +@RunWith(JUnit4.class) +@SuppressWarnings("rawtypes") +public class AvroReaderFactoryTest { + private final String pathToAvroFile = "/path/to/file.avro"; + + Reader runTestCreateAvroReader(String filename, @Nullable Long start, @Nullable Long end, + CloudObject encoding) throws Exception { + CloudObject spec = CloudObject.forClassName("AvroSource"); + addString(spec, "filename", filename); + if (start != null) { + addLong(spec, "start_offset", start); + } + if (end != null) { + addLong(spec, "end_offset", end); + } + + Source cloudSource = new Source(); + cloudSource.setSpec(spec); + cloudSource.setCodec(encoding); + + Reader reader = ReaderFactory.create( + PipelineOptionsFactory.create(), cloudSource, new BatchModeExecutionContext()); + return reader; + } + + @Test + public void testCreatePlainAvroByteReader() throws Exception { + Coder coder = WindowedValue.getValueOnlyCoder(BigEndianIntegerCoder.of()); + Reader reader = runTestCreateAvroReader(pathToAvroFile, null, null, coder.asCloudObject()); + + Assert.assertThat(reader, new IsInstanceOf(AvroByteReader.class)); + AvroByteReader avroReader = (AvroByteReader) reader; + Assert.assertEquals(pathToAvroFile, avroReader.avroReader.filename); + Assert.assertEquals(null, avroReader.avroReader.startPosition); + Assert.assertEquals(null, avroReader.avroReader.endPosition); + Assert.assertEquals(coder, avroReader.coder); + } + + @Test + public void testCreateRichAvroByteReader() throws Exception { + Coder coder = WindowedValue.getValueOnlyCoder(BigEndianIntegerCoder.of()); + Reader reader = runTestCreateAvroReader(pathToAvroFile, 200L, 500L, coder.asCloudObject()); + + Assert.assertThat(reader, new IsInstanceOf(AvroByteReader.class)); + AvroByteReader avroReader = (AvroByteReader) reader; + Assert.assertEquals(pathToAvroFile, avroReader.avroReader.filename); + Assert.assertEquals(200L, (long) avroReader.avroReader.startPosition); + Assert.assertEquals(500L, (long) avroReader.avroReader.endPosition); + Assert.assertEquals(coder, avroReader.coder); + } + + @Test + public void testCreateRichAvroReader() throws Exception { + WindowedValue.WindowedValueCoder coder = + WindowedValue.getValueOnlyCoder(AvroCoder.of(Integer.class)); + Reader reader = runTestCreateAvroReader(pathToAvroFile, 200L, 500L, coder.asCloudObject()); + + Assert.assertThat(reader, new IsInstanceOf(AvroReader.class)); + AvroReader avroReader = (AvroReader) reader; + Assert.assertEquals(pathToAvroFile, avroReader.filename); + Assert.assertEquals(200L, (long) avroReader.startPosition); + Assert.assertEquals(500L, (long) avroReader.endPosition); + Assert.assertEquals(coder.getValueCoder(), avroReader.avroCoder); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderTest.java similarity index 73% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceTest.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderTest.java index 4855ef92e4d96..4e5a4a71e4c6b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderTest.java @@ -24,11 +24,10 @@ import com.google.cloud.dataflow.sdk.util.MimeTypes; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.apache.avro.file.DataFileWriter; import org.apache.avro.io.DatumWriter; - import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -47,23 +46,21 @@ import javax.annotation.Nullable; /** - * Tests for AvroSource. + * Tests for AvroReader. */ @RunWith(JUnit4.class) -public class AvroSourceTest { +public class AvroReaderTest { @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); - private void runTestRead(List> elemsList, - AvroCoder coder, - boolean requireExactMatch) - throws Exception { + private void runTestRead( + List> elemsList, AvroCoder coder, boolean requireExactMatch) throws Exception { File tmpFile = tmpFolder.newFile("file.avro"); String filename = tmpFile.getPath(); // Write the data. - OutputStream outStream = Channels.newOutputStream( - IOChannelUtils.create(filename, MimeTypes.BINARY)); + OutputStream outStream = + Channels.newOutputStream(IOChannelUtils.create(filename, MimeTypes.BINARY)); DatumWriter datumWriter = coder.createDatumWriter(); DataFileWriter fileWriter = new DataFileWriter<>(datumWriter); fileWriter.create(coder.getSchema(), outStream); @@ -122,19 +119,15 @@ private void runTestRead(List> elemsList, Assert.assertEquals(expectedSizes, actualSizes); } - private List readElems(String filename, - @Nullable Long startOffset, - @Nullable Long endOffset, - Coder coder, - List actualSizes) - throws Exception { - AvroSource avroSource = - new AvroSource<>(filename, startOffset, endOffset, WindowedValue.getValueOnlyCoder(coder)); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(avroSource, actualSizes); + private List readElems(String filename, @Nullable Long startOffset, + @Nullable Long endOffset, Coder coder, List actualSizes) throws Exception { + AvroReader avroReader = + new AvroReader<>(filename, startOffset, endOffset, WindowedValue.getValueOnlyCoder(coder)); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(avroReader, actualSizes); List actualElems = new ArrayList<>(); - try (Source.SourceIterator> iterator = avroSource.iterator()) { + try (Reader.ReaderIterator> iterator = avroReader.iterator()) { while (iterator.hasNext()) { actualElems.add(iterator.next().getValue()); } @@ -144,21 +137,18 @@ private List readElems(String filename, @Test public void testRead() throws Exception { - runTestRead(Collections.singletonList(TestUtils.INTS), - AvroCoder.of(Integer.class), - true /* require exact match */); + runTestRead(Collections.singletonList(TestUtils.INTS), AvroCoder.of(Integer.class), + true/* require exact match */); } @Test public void testReadEmpty() throws Exception { - runTestRead(Collections.singletonList(TestUtils.NO_INTS), - AvroCoder.of(Integer.class), - true /* require exact match */); + runTestRead(Collections.singletonList(TestUtils.NO_INTS), AvroCoder.of(Integer.class), + true/* require exact match */); } - private List> generateInputBlocks(int numBlocks, - int blockSizeBytes, - int averageLineSizeBytes) { + private List> generateInputBlocks( + int numBlocks, int blockSizeBytes, int averageLineSizeBytes) { Random random = new Random(0); List> blocks = new ArrayList<>(numBlocks); for (int blockNum = 0; blockNum < numBlocks; blockNum++) { @@ -179,16 +169,14 @@ private List> generateInputBlocks(int numBlocks, @Test public void testReadSmallRanges() throws Exception { - runTestRead(generateInputBlocks(3, 50, 5), - AvroCoder.of(String.class), - true /* require exact match */); + runTestRead( + generateInputBlocks(3, 50, 5), AvroCoder.of(String.class), true/* require exact match */); } @Test public void testReadBigRanges() throws Exception { - runTestRead(generateInputBlocks(10, 128 * 1024, 100), - AvroCoder.of(String.class), - false /* don't require exact match */); + runTestRead(generateInputBlocks(10, 128 * 1024, 100), AvroCoder.of(String.class), + false/* don't require exact match */); } // TODO: sharded filenames diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkTest.java index 5f22d2774f4be..f2199e6781da5 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkTest.java @@ -63,16 +63,14 @@ void runTestWriteFile(List elems, AvroCoder coder) throws Exception { // Read back the file. - SeekableByteChannel inChannel = (SeekableByteChannel) - IOChannelUtils.getFactory(filename).open(filename); + SeekableByteChannel inChannel = + (SeekableByteChannel) IOChannelUtils.getFactory(filename).open(filename); - SeekableInput seekableInput = - new AvroSource.SeekableByteChannelInput(inChannel); + SeekableInput seekableInput = new AvroReader.SeekableByteChannelInput(inChannel); DatumReader datumReader = new GenericDatumReader<>(coder.getSchema()); - DataFileReader fileReader = new DataFileReader<>( - seekableInput, datumReader); + DataFileReader fileReader = new DataFileReader<>(seekableInput, datumReader); List actual = new ArrayList<>(); List expectedSizes = new ArrayList<>(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactoryTest.java deleted file mode 100644 index 1db06de456057..0000000000000 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSourceFactoryTest.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * Copyright (C) 2014 Google Inc. - * - * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; - -import static com.google.cloud.dataflow.sdk.util.Structs.addLong; -import static com.google.cloud.dataflow.sdk.util.Structs.addString; - -import com.google.cloud.dataflow.sdk.coders.AvroCoder; -import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; -import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; -import com.google.cloud.dataflow.sdk.util.CloudObject; -import com.google.cloud.dataflow.sdk.util.WindowedValue; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; - -import org.hamcrest.core.IsInstanceOf; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -import javax.annotation.Nullable; - -/** - * Tests for AvroSourceFactory. - */ -@RunWith(JUnit4.class) -@SuppressWarnings("rawtypes") -public class AvroSourceFactoryTest { - private final String pathToAvroFile = "/path/to/file.avro"; - - Source runTestCreateAvroSource(String filename, - @Nullable Long start, - @Nullable Long end, - CloudObject encoding) - throws Exception { - CloudObject spec = CloudObject.forClassName("AvroSource"); - addString(spec, "filename", filename); - if (start != null) { - addLong(spec, "start_offset", start); - } - if (end != null) { - addLong(spec, "end_offset", end); - } - - com.google.api.services.dataflow.model.Source cloudSource = - new com.google.api.services.dataflow.model.Source(); - cloudSource.setSpec(spec); - cloudSource.setCodec(encoding); - - Source source = SourceFactory.create(PipelineOptionsFactory.create(), - cloudSource, - new BatchModeExecutionContext()); - return source; - } - - @Test - public void testCreatePlainAvroByteSource() throws Exception { - Coder coder = - WindowedValue.getValueOnlyCoder(BigEndianIntegerCoder.of()); - Source source = runTestCreateAvroSource( - pathToAvroFile, null, null, coder.asCloudObject()); - - Assert.assertThat(source, new IsInstanceOf(AvroByteSource.class)); - AvroByteSource avroSource = (AvroByteSource) source; - Assert.assertEquals(pathToAvroFile, avroSource.avroSource.filename); - Assert.assertEquals(null, avroSource.avroSource.startPosition); - Assert.assertEquals(null, avroSource.avroSource.endPosition); - Assert.assertEquals(coder, avroSource.coder); - } - - @Test - public void testCreateRichAvroByteSource() throws Exception { - Coder coder = - WindowedValue.getValueOnlyCoder(BigEndianIntegerCoder.of()); - Source source = runTestCreateAvroSource( - pathToAvroFile, 200L, 500L, coder.asCloudObject()); - - Assert.assertThat(source, new IsInstanceOf(AvroByteSource.class)); - AvroByteSource avroSource = (AvroByteSource) source; - Assert.assertEquals(pathToAvroFile, avroSource.avroSource.filename); - Assert.assertEquals(200L, (long) avroSource.avroSource.startPosition); - Assert.assertEquals(500L, (long) avroSource.avroSource.endPosition); - Assert.assertEquals(coder, avroSource.coder); - } - - @Test - public void testCreateRichAvroSource() throws Exception { - WindowedValue.WindowedValueCoder coder = - WindowedValue.getValueOnlyCoder(AvroCoder.of(Integer.class)); - Source source = runTestCreateAvroSource( - pathToAvroFile, 200L, 500L, coder.asCloudObject()); - - Assert.assertThat(source, new IsInstanceOf(AvroSource.class)); - AvroSource avroSource = (AvroSource) source; - Assert.assertEquals(pathToAvroFile, avroSource.filename); - Assert.assertEquals(200L, (long) avroSource.startPosition); - Assert.assertEquals(500L, (long) avroSource.endPosition); - Assert.assertEquals(coder.getValueCoder(), avroSource.avroCoder); - } -} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderFactoryTest.java similarity index 53% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactoryTest.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderFactoryTest.java index 0eb95c70205ca..f8a87c85d2c60 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderFactoryTest.java @@ -19,10 +19,11 @@ import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; import static com.google.cloud.dataflow.sdk.util.Structs.addString; +import com.google.api.services.dataflow.model.Source; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CloudObject; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.hamcrest.core.IsInstanceOf; import org.junit.Assert; @@ -31,48 +32,41 @@ import org.junit.runners.JUnit4; /** - * Tests for BigQuerySourceFactory. + * Tests for BigQueryReaderFactory. */ @RunWith(JUnit4.class) -public class BigQuerySourceFactoryTest { - void runTestCreateBigQuerySource(String project, - String dataset, - String table, - CloudObject encoding) - throws Exception { +public class BigQueryReaderFactoryTest { + void runTestCreateBigQueryReader( + String project, String dataset, String table, CloudObject encoding) throws Exception { CloudObject spec = CloudObject.forClassName("BigQuerySource"); addString(spec, "project", project); addString(spec, "dataset", dataset); addString(spec, "table", table); - com.google.api.services.dataflow.model.Source cloudSource = - new com.google.api.services.dataflow.model.Source(); + Source cloudSource = new Source(); cloudSource.setSpec(spec); cloudSource.setCodec(encoding); - Source source = SourceFactory.create(PipelineOptionsFactory.create(), - cloudSource, - new BatchModeExecutionContext()); - Assert.assertThat(source, new IsInstanceOf(BigQuerySource.class)); - BigQuerySource bigQuerySource = (BigQuerySource) source; - Assert.assertEquals(project, bigQuerySource.tableRef.getProjectId()); - Assert.assertEquals(dataset, bigQuerySource.tableRef.getDatasetId()); - Assert.assertEquals(table, bigQuerySource.tableRef.getTableId()); + Reader reader = ReaderFactory.create( + PipelineOptionsFactory.create(), cloudSource, new BatchModeExecutionContext()); + Assert.assertThat(reader, new IsInstanceOf(BigQueryReader.class)); + BigQueryReader bigQueryReader = (BigQueryReader) reader; + Assert.assertEquals(project, bigQueryReader.tableRef.getProjectId()); + Assert.assertEquals(dataset, bigQueryReader.tableRef.getDatasetId()); + Assert.assertEquals(table, bigQueryReader.tableRef.getTableId()); } @Test - public void testCreateBigQuerySource() throws Exception { - runTestCreateBigQuerySource( - "someproject", "somedataset", "sometable", - makeCloudEncoding("TableRowJsonCoder")); + public void testCreateBigQueryReader() throws Exception { + runTestCreateBigQueryReader( + "someproject", "somedataset", "sometable", makeCloudEncoding("TableRowJsonCoder")); } @Test - public void testCreateBigQuerySourceCoderIgnored() throws Exception { + public void testCreateBigQueryReaderCoderIgnored() throws Exception { // BigQuery sources do not need a coder because the TableRow objects are read directly from // the table using the BigQuery API. - runTestCreateBigQuerySource( - "someproject", "somedataset", "sometable", - makeCloudEncoding("BigEndianIntegerCoder")); + runTestCreateBigQueryReader( + "someproject", "somedataset", "sometable", makeCloudEncoding("BigEndianIntegerCoder")); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java similarity index 69% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceTest.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java index 2ed4635e8c10b..18248ef3183ad 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQuerySourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java @@ -32,6 +32,7 @@ import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.junit.After; import org.junit.Assert; @@ -48,20 +49,24 @@ import java.util.List; /** - * Tests for BigQuerySource. + * Tests for BigQueryReader. * *

The tests just make sure a basic scenario of reading works because the class itself is a * thin wrapper over {@code BigQueryTableRowIterator}. The tests for the wrapped class have * comprehensive coverage. */ @RunWith(JUnit4.class) -public class BigQuerySourceTest { - - @Mock private Bigquery mockClient; - @Mock private Bigquery.Tables mockTables; - @Mock private Bigquery.Tables.Get mockTablesGet; - @Mock private Bigquery.Tabledata mockTabledata; - @Mock private Bigquery.Tabledata.List mockTabledataList; +public class BigQueryReaderTest { + @Mock + private Bigquery mockClient; + @Mock + private Bigquery.Tables mockTables; + @Mock + private Bigquery.Tables.Get mockTablesGet; + @Mock + private Bigquery.Tabledata mockTabledata; + @Mock + private Bigquery.Tabledata.List mockTabledataList; @Before public void setUp() { @@ -78,12 +83,9 @@ public void tearDown() { } private void onTableGet(Table table) throws IOException { - when(mockClient.tables()) - .thenReturn(mockTables); - when(mockTables.get(anyString(), anyString(), anyString())) - .thenReturn(mockTablesGet); - when(mockTablesGet.execute()) - .thenReturn(table); + when(mockClient.tables()).thenReturn(mockTables); + when(mockTables.get(anyString(), anyString(), anyString())).thenReturn(mockTablesGet); + when(mockTablesGet.execute()).thenReturn(table); } private void verifyTableGet() throws IOException { @@ -93,12 +95,9 @@ private void verifyTableGet() throws IOException { } private void onTableList(TableDataList result) throws IOException { - when(mockClient.tabledata()) - .thenReturn(mockTabledata); - when(mockTabledata.list(anyString(), anyString(), anyString())) - .thenReturn(mockTabledataList); - when(mockTabledataList.execute()) - .thenReturn(result); + when(mockClient.tabledata()).thenReturn(mockTabledata); + when(mockTabledata.list(anyString(), anyString(), anyString())).thenReturn(mockTabledataList); + when(mockTabledataList.execute()).thenReturn(result); } private void verifyTabledataList() throws IOException { @@ -110,25 +109,14 @@ private void verifyTabledataList() throws IOException { } private Table basicTableSchema() { - return new Table() - .setSchema(new TableSchema() - .setFields(Arrays.asList( - new TableFieldSchema() - .setName("name") - .setType("STRING"), - new TableFieldSchema() - .setName("integer") - .setType("INTEGER"), - new TableFieldSchema() - .setName("float") - .setType("FLOAT"), - new TableFieldSchema() - .setName("bool") - .setType("BOOLEAN") - ))); + return new Table().setSchema(new TableSchema().setFields(Arrays.asList( + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("integer").setType("INTEGER"), + new TableFieldSchema().setName("float").setType("FLOAT"), + new TableFieldSchema().setName("bool").setType("BOOLEAN")))); } - private TableRow rawRow(Object...args) { + private TableRow rawRow(Object... args) { List cells = new LinkedList<>(); for (Object a : args) { cells.add(new TableCell().setV(a)); @@ -136,9 +124,8 @@ private TableRow rawRow(Object...args) { return new TableRow().setF(cells); } - private TableDataList rawDataList(TableRow...rows) { - return new TableDataList() - .setRows(Arrays.asList(rows)); + private TableDataList rawDataList(TableRow... rows) { + return new TableDataList().setRows(Arrays.asList(rows)); } @Test @@ -147,19 +134,14 @@ public void testRead() throws IOException { // BQ API data is always encoded as a string TableDataList dataList = rawDataList( - rawRow("Arthur", "42", "3.14159", "false"), - rawRow("Allison", "79", "2.71828", "true") - ); + rawRow("Arthur", "42", "3.14159", "false"), rawRow("Allison", "79", "2.71828", "true")); onTableList(dataList); - BigQuerySource source = new BigQuerySource( + BigQueryReader reader = new BigQueryReader( mockClient, - new TableReference() - .setProjectId("project") - .setDatasetId("dataset") - .setTableId("table")); + new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table")); - BigQuerySource.SourceIterator iterator = source.iterator(); + Reader.ReaderIterator iterator = reader.iterator(); Assert.assertTrue(iterator.hasNext()); TableRow row = iterator.next(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java index 952a51bbce9c6..081e0b8c14ed9 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java @@ -16,8 +16,8 @@ package com.google.cloud.dataflow.sdk.runners.worker; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToSourcePosition; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; import static com.google.cloud.dataflow.sdk.util.CloudCounterUtils.extractCounter; @@ -52,7 +52,7 @@ import com.google.cloud.dataflow.sdk.util.common.Metric.DoubleMetric; import com.google.cloud.dataflow.sdk.util.common.worker.MapTaskExecutor; import com.google.cloud.dataflow.sdk.util.common.worker.Operation; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; import org.hamcrest.Description; @@ -83,24 +83,23 @@ static class TestMapTaskExecutor extends MapTaskExecutor { ApproximateProgress progress = null; public TestMapTaskExecutor(CounterSet counters) { - super(new ArrayList(), - counters, - new StateSampler("test", counters.getAddCounterMutator())); + super(new ArrayList(), counters, + new StateSampler("test", counters.getAddCounterMutator())); } @Override - public Source.Progress getWorkerProgress() { - return cloudProgressToSourceProgress(progress); + public Reader.Progress getWorkerProgress() { + return cloudProgressToReaderProgress(progress); } @Override - public Source.Position proposeStopPosition( - Source.Progress suggestedStopPoint) { - @Nullable ApproximateProgress progress = sourceProgressToCloudProgress(suggestedStopPoint); + public Reader.Position proposeStopPosition(Reader.Progress suggestedStopPoint) { + @Nullable + ApproximateProgress progress = sourceProgressToCloudProgress(suggestedStopPoint); if (progress == null) { return null; } - return cloudPositionToSourcePosition(progress.getPosition()); + return cloudPositionToReaderPosition(progress.getPosition()); } public void setWorkerProgress(ApproximateProgress progress) { @@ -124,8 +123,10 @@ public void setWorkerProgress(ApproximateProgress progress) { private static final Double COUNTER_VALUE2 = Math.PI; private static final String COUNTER_VALUE3 = "value"; - @Rule public final ExpectedException thrown = ExpectedException.none(); - @Mock private DataflowWorker.WorkUnitClient workUnitClient; + @Rule + public final ExpectedException thrown = ExpectedException.none(); + @Mock + private DataflowWorker.WorkUnitClient workUnitClient; private CounterSet counters; private List> metrics; private TestMapTaskExecutor worker; @@ -160,8 +161,7 @@ public Collection> getOutputMetrics() { workItem.setLeaseExpireTime(toCloudTime(new Instant(nowMillis + 1000))); workItem.setReportStatusInterval(toCloudDuration(Duration.millis(500))); - progressUpdater = new DataflowWorkProgressUpdater( - workItem, worker, workUnitClient, options); + progressUpdater = new DataflowWorkProgressUpdater(workItem, worker, workUnitClient, options); } // TODO: Remove sleeps from this test by using a mock sleeper. This @@ -169,18 +169,17 @@ public Collection> getOutputMetrics() { // not use a ScheduledThreadExecutor which relies on real time passing. @Test(timeout = 2000) public void workProgressUpdaterUpdates() throws Exception { - when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))).thenReturn( - generateServiceState(nowMillis + 2000, 1000, null)); + when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) + .thenReturn(generateServiceState(nowMillis + 2000, 1000, null)); setUpCounters(2); setUpMetrics(3); setUpProgress(makeRecordIndexProgress(1L)); progressUpdater.startReportingProgress(); // The initial update should be sent after leaseRemainingTime / 2. - verify(workUnitClient, timeout(600)).reportWorkItemStatus(argThat( - new ExpectedDataflowProgress() - .withCounters(2) - .withMetrics(3) - .withProgress(makeRecordIndexProgress(1L)))); + verify(workUnitClient, timeout(600)) + .reportWorkItemStatus( + argThat(new ExpectedDataflowProgress().withCounters(2).withMetrics(3).withProgress( + makeRecordIndexProgress(1L)))); progressUpdater.stopReportingProgress(); } @@ -192,8 +191,7 @@ public void workProgressUpdaterAdaptsProgressInterval() throws Exception { // us to truncate the task at index 3, and the next two will not ask us to // truncate at all. when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) - .thenReturn(generateServiceState(nowMillis + 2000, 1000, - makeRecordIndexPosition(3L))) + .thenReturn(generateServiceState(nowMillis + 2000, 1000, makeRecordIndexPosition(3L))) .thenReturn(generateServiceState(nowMillis + 3000, 2000, null)) .thenReturn(generateServiceState(nowMillis + 4000, 3000, null)); @@ -203,22 +201,22 @@ public void workProgressUpdaterAdaptsProgressInterval() throws Exception { progressUpdater.startReportingProgress(); // The initial update should be sent after // leaseRemainingTime (1000) / 2 = 500. - verify(workUnitClient, timeout(600)).reportWorkItemStatus(argThat( - new ExpectedDataflowProgress() - .withCounters(3) - .withMetrics(2) - .withProgress(makeRecordIndexProgress(1L)))); + verify(workUnitClient, timeout(600)) + .reportWorkItemStatus( + argThat(new ExpectedDataflowProgress().withCounters(3).withMetrics(2).withProgress( + makeRecordIndexProgress(1L)))); setUpCounters(5); setUpMetrics(6); setUpProgress(makeRecordIndexProgress(2L)); // The second update should be sent after one second (2000 / 2). - verify(workUnitClient, timeout(1100)).reportWorkItemStatus(argThat( - new ExpectedDataflowProgress() - .withCounters(5) - .withMetrics(6) - .withProgress(makeRecordIndexProgress(2L)) - .withStopPosition(makeRecordIndexPosition(3L)))); + verify(workUnitClient, timeout(1100)) + .reportWorkItemStatus(argThat( + new ExpectedDataflowProgress() + .withCounters(5) + .withMetrics(6) + .withProgress(makeRecordIndexProgress(2L)) + .withStopPosition(makeRecordIndexPosition(3L)))); // After the request is sent, reset stop position cache to null. assertNull(progressUpdater.getStopPosition()); @@ -226,9 +224,9 @@ public void workProgressUpdaterAdaptsProgressInterval() throws Exception { setUpProgress(makeRecordIndexProgress(3L)); // The third update should be sent after one and half seconds (3000 / 2). - verify(workUnitClient, timeout(1600)).reportWorkItemStatus(argThat( - new ExpectedDataflowProgress() - .withProgress(makeRecordIndexProgress(3L)))); + verify(workUnitClient, timeout(1600)) + .reportWorkItemStatus( + argThat(new ExpectedDataflowProgress().withProgress(makeRecordIndexProgress(3L)))); progressUpdater.stopReportingProgress(); } @@ -237,29 +235,30 @@ public void workProgressUpdaterAdaptsProgressInterval() throws Exception { @Test(timeout = 3000) public void workProgressUpdaterLastUpdate() throws Exception { when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) - .thenReturn(generateServiceState(nowMillis + 2000, 1000, - makeRecordIndexPosition(2L))) + .thenReturn(generateServiceState(nowMillis + 2000, 1000, makeRecordIndexPosition(2L))) .thenReturn(generateServiceState(nowMillis + 3000, 2000, null)); setUpProgress(makeRecordIndexProgress(1L)); progressUpdater.startReportingProgress(); // The initial update should be sent after leaseRemainingTime / 2 = 500 msec. Thread.sleep(600); - verify(workUnitClient, timeout(200)).reportWorkItemStatus(argThat( - new ExpectedDataflowProgress() - .withProgress(makeRecordIndexProgress(1L)))); + verify(workUnitClient, timeout(200)) + .reportWorkItemStatus( + argThat(new ExpectedDataflowProgress().withProgress(makeRecordIndexProgress(1L)))); // The first update should include the new actual stop position. // Verify that the progressUpdater has recorded it. - assertEquals(makeRecordIndexPosition(2L), + assertEquals( + makeRecordIndexPosition(2L), sourcePositionToCloudPosition(progressUpdater.getStopPosition())); setUpProgress(makeRecordIndexProgress(2L)); // The second update should be sent after one second (2000 / 2). - Thread.sleep(200); // not enough time for an update so the latest stop position is not - // acknowledged. + Thread.sleep(200); // not enough time for an update so the latest stop position is not + // acknowledged. // Check that the progressUpdater still has a pending stop position to send - assertEquals(makeRecordIndexPosition(2L), + assertEquals( + makeRecordIndexPosition(2L), sourcePositionToCloudPosition(progressUpdater.getStopPosition())); progressUpdater.stopReportingProgress(); // should send the last update @@ -267,9 +266,9 @@ public void workProgressUpdaterLastUpdate() throws Exception { assertNull(progressUpdater.getStopPosition()); // Verify that the last update contained the latest stop position - verify(workUnitClient, timeout(1000)).reportWorkItemStatus(argThat( - new ExpectedDataflowProgress() - .withStopPosition(makeRecordIndexPosition(2L)))); + verify(workUnitClient, timeout(1000)) + .reportWorkItemStatus( + argThat(new ExpectedDataflowProgress().withStopPosition(makeRecordIndexPosition(2L)))); } private void setUpCounters(int n) { @@ -282,13 +281,16 @@ private void setUpCounters(int n) { private static Counter makeCounter(int i) { if (i % 3 == 0) { return Counter.longs(COUNTER_NAME + i, COUNTER_KINDS[0]) - .addValue(COUNTER_VALUE1 + i).addValue(COUNTER_VALUE1 + i * 2); + .addValue(COUNTER_VALUE1 + i) + .addValue(COUNTER_VALUE1 + i * 2); } else if (i % 3 == 1) { return Counter.doubles(COUNTER_NAME + i, COUNTER_KINDS[1]) - .addValue(COUNTER_VALUE2 + i).addValue(COUNTER_VALUE2 + i * 3); + .addValue(COUNTER_VALUE2 + i) + .addValue(COUNTER_VALUE2 + i * 3); } else { return Counter.strings(COUNTER_NAME + i, COUNTER_KINDS[2]) - .addValue(COUNTER_VALUE3 + i).addValue(COUNTER_NAME + i * 5); + .addValue(COUNTER_VALUE3 + i) + .addValue(COUNTER_NAME + i * 5); } } @@ -318,10 +320,8 @@ private ApproximateProgress makeRecordIndexProgress(Long index) { return new ApproximateProgress().setPosition(makeRecordIndexPosition(index)); } - private WorkItemServiceState generateServiceState( - long leaseExpirationTimestamp, int progressReportIntervalMs, - Position suggestedStopPosition) - throws IOException { + private WorkItemServiceState generateServiceState(long leaseExpirationTimestamp, + int progressReportIntervalMs, Position suggestedStopPosition) throws IOException { WorkItemServiceState responseState = new WorkItemServiceState(); responseState.setFactory(Transport.getJsonFactory()); responseState.setLeaseExpireTime(toCloudTime(new Instant(leaseExpirationTimestamp))); @@ -336,10 +336,14 @@ private WorkItemServiceState generateServiceState( } private static final class ExpectedDataflowProgress extends ArgumentMatcher { - @Nullable Integer counterCount; - @Nullable Integer metricCount; - @Nullable ApproximateProgress expectedProgress; - @Nullable Position expectedStopPosition; + @Nullable + Integer counterCount; + @Nullable + Integer metricCount; + @Nullable + ApproximateProgress expectedProgress; + @Nullable + Position expectedStopPosition; public ExpectedDataflowProgress withCounters(Integer counterCount) { this.counterCount = counterCount; @@ -388,9 +392,7 @@ public void describeTo(Description description) { @Override public boolean matches(Object status) { WorkItemStatus st = (WorkItemStatus) status; - return matchCountersAndMetrics(st) - && matchProgress(st) - && matchStopPosition(st); + return matchCountersAndMetrics(st) && matchProgress(st) && matchStopPosition(st); } private boolean matchCountersAndMetrics(WorkItemStatus status) { @@ -405,8 +407,7 @@ private boolean matchCountersAndMetrics(WorkItemStatus status) { } for (int i = 0; i < counterCount; i++) { - if (!sentUpdates.contains( - CounterTestUtils.extractCounterUpdate(makeCounter(i), false))) { + if (!sentUpdates.contains(CounterTestUtils.extractCounterUpdate(makeCounter(i), false))) { return false; } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java similarity index 68% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java index 21ecb0af04f2c..ff3abd3dbe426 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java @@ -17,7 +17,7 @@ package com.google.cloud.dataflow.sdk.runners.worker; import static com.google.api.client.util.Base64.encodeBase64URLSafeString; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; @@ -35,10 +35,9 @@ import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.common.Reiterable; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; import com.google.cloud.dataflow.sdk.util.common.worker.Sink; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; -import com.google.cloud.dataflow.sdk.util.common.worker.Source.SourceIterator; import com.google.cloud.dataflow.sdk.values.KV; import com.google.common.collect.Lists; @@ -58,20 +57,18 @@ import java.util.NoSuchElementException; /** - * Tests for GroupingShuffleSource. + * Tests for GroupingShuffleReader. */ @RunWith(JUnit4.class) -public class GroupingShuffleSourceTest { +public class GroupingShuffleReaderTest { private static final List>> NO_KVS = Collections.emptyList(); private static final Instant timestamp = new Instant(123000); private static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); private static final List>> KVS = Arrays.asList( - KV.of(1, Arrays.asList("in 1a", "in 1b")), - KV.of(2, Arrays.asList("in 2a", "in 2b")), - KV.of(3, Arrays.asList("in 3")), - KV.of(4, Arrays.asList("in 4a", "in 4b", "in 4c", "in 4d")), + KV.of(1, Arrays.asList("in 1a", "in 1b")), KV.of(2, Arrays.asList("in 2a", "in 2b")), + KV.of(3, Arrays.asList("in 3")), KV.of(4, Arrays.asList("in 4a", "in 4b", "in 4c", "in 4d")), KV.of(5, Arrays.asList("in 5"))); /** How many of the values with each key are to be read. */ @@ -86,8 +83,9 @@ private enum ValuesToRead { READ_ALL_VALUES } - private void runTestReadShuffleSource(List>> input, - ValuesToRead valuesToRead) + private void runTestReadFromShuffle( + List>> input, + ValuesToRead valuesToRead) throws Exception { Coder>> sinkElemCoder = WindowedValue.getFullCoder( @@ -129,16 +127,16 @@ private void runTestReadShuffleSource(List>> input, Assert.assertEquals(kvCount, records.size()); Assert.assertEquals(shuffleWriter.getSizes(), actualSizes); - // Read from shuffle with GroupingShuffleSource. + // Read from shuffle with GroupingShuffleReader. BatchModeExecutionContext context = new BatchModeExecutionContext(); - GroupingShuffleSource shuffleSource = - new GroupingShuffleSource<>( + GroupingShuffleReader groupingShuffleReader = + new GroupingShuffleReader<>( PipelineOptionsFactory.create(), null, null, null, sourceElemCoder, context); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(shuffleSource); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(groupingShuffleReader); TestShuffleReader shuffleReader = new TestShuffleReader(); List expectedSizes = new ArrayList<>(); @@ -148,8 +146,8 @@ private void runTestReadShuffleSource(List>> input, } List>> actual = new ArrayList<>(); - try (SourceIterator>>> iter = - shuffleSource.iterator(shuffleReader)) { + try (Reader.ReaderIterator>>> iter = + groupingShuffleReader.iterator(shuffleReader)) { Iterable prevValuesIterable = null; Iterator prevValuesIterator = null; while (iter.hasNext()) { @@ -166,10 +164,10 @@ private void runTestReadShuffleSource(List>> input, List values = new ArrayList<>(); if (valuesToRead.ordinal() > ValuesToRead.SKIP_VALUES.ordinal()) { if (prevValuesIterable != null) { - prevValuesIterable.iterator(); // Verifies that this does not throw. + prevValuesIterable.iterator(); // Verifies that this does not throw. } if (prevValuesIterator != null) { - prevValuesIterator.hasNext(); // Verifies that this does not throw. + prevValuesIterator.hasNext(); // Verifies that this does not throw. } Iterable valuesIterable = elem.getValue(); @@ -179,8 +177,7 @@ private void runTestReadShuffleSource(List>> input, while (valuesIterator.hasNext()) { Assert.assertTrue(valuesIterator.hasNext()); Assert.assertTrue(valuesIterator.hasNext()); - Assert.assertEquals("BatchModeExecutionContext key", - key, context.getKey()); + Assert.assertEquals("BatchModeExecutionContext key", key, context.getKey()); values.add(valuesIterator.next()); if (valuesToRead == ValuesToRead.READ_ONE_VALUE) { break; @@ -196,7 +193,7 @@ private void runTestReadShuffleSource(List>> input, } catch (NoSuchElementException exn) { // As expected. } - valuesIterable.iterator(); // Verifies that this does not throw. + valuesIterable.iterator(); // Verifies that this does not throw. } } @@ -235,33 +232,33 @@ private void runTestReadShuffleSource(List>> input, } @Test - public void testReadEmptyShuffleSource() throws Exception { - runTestReadShuffleSource(NO_KVS, ValuesToRead.READ_ALL_VALUES); + public void testReadEmptyShuffleData() throws Exception { + runTestReadFromShuffle(NO_KVS, ValuesToRead.READ_ALL_VALUES); } @Test - public void testReadEmptyShuffleSourceSkippingValues() throws Exception { - runTestReadShuffleSource(NO_KVS, ValuesToRead.SKIP_VALUES); + public void testReadEmptyShuffleDataSkippingValues() throws Exception { + runTestReadFromShuffle(NO_KVS, ValuesToRead.SKIP_VALUES); } @Test - public void testReadNonEmptyShuffleSource() throws Exception { - runTestReadShuffleSource(KVS, ValuesToRead.READ_ALL_VALUES); + public void testReadNonEmptyShuffleData() throws Exception { + runTestReadFromShuffle(KVS, ValuesToRead.READ_ALL_VALUES); } @Test - public void testReadNonEmptyShuffleSourceReadingOneValue() throws Exception { - runTestReadShuffleSource(KVS, ValuesToRead.READ_ONE_VALUE); + public void testReadNonEmptyShuffleDataReadingOneValue() throws Exception { + runTestReadFromShuffle(KVS, ValuesToRead.READ_ONE_VALUE); } @Test - public void testReadNonEmptyShuffleSourceReadingNoValues() throws Exception { - runTestReadShuffleSource(KVS, ValuesToRead.READ_NO_VALUES); + public void testReadNonEmptyShuffleDataReadingNoValues() throws Exception { + runTestReadFromShuffle(KVS, ValuesToRead.READ_NO_VALUES); } @Test - public void testReadNonEmptyShuffleSourceSkippingValues() throws Exception { - runTestReadShuffleSource(KVS, ValuesToRead.SKIP_VALUES); + public void testReadNonEmptyShuffleDataSkippingValues() throws Exception { + runTestReadFromShuffle(KVS, ValuesToRead.SKIP_VALUES); } static byte[] fabricatePosition(int shard, byte[] key) throws Exception { @@ -275,24 +272,17 @@ static byte[] fabricatePosition(int shard, byte[] key) throws Exception { } @Test - public void testReadFromEmptyShuffleSourceAndUpdateStopPosition() - throws Exception { + public void testReadFromEmptyShuffleDataAndUpdateStopPosition() throws Exception { BatchModeExecutionContext context = new BatchModeExecutionContext(); - GroupingShuffleSource shuffleSource = - new GroupingShuffleSource<>( - PipelineOptionsFactory.create(), - null, null, null, - WindowedValue.getFullCoder( - KvCoder.of( - BigEndianIntegerCoder.of(), - IterableCoder.of(BigEndianIntegerCoder.of())), - IntervalWindow.getCoder()), - context); + GroupingShuffleReader groupingShuffleReader = new GroupingShuffleReader<>( + PipelineOptionsFactory.create(), null, null, null, + WindowedValue.getFullCoder( + KvCoder.of(BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), + IntervalWindow.getCoder()), + context); TestShuffleReader shuffleReader = new TestShuffleReader(); - try (Source.SourceIterator>>> iter = - shuffleSource.iterator(shuffleReader)) { - - + try (Reader.ReaderIterator>>> iter = + groupingShuffleReader.iterator(shuffleReader)) { // Can update the stop position, the source range spans all interval Position proposedStopPosition = new Position(); String stop = encodeBase64URLSafeString(fabricatePosition(0, null)); @@ -301,25 +291,23 @@ public void testReadFromEmptyShuffleSourceAndUpdateStopPosition() Assert.assertEquals( stop, sourcePositionToCloudPosition( - iter.updateStopPosition( - cloudProgressToSourceProgress( - createApproximateProgress(proposedStopPosition)))) - .getShufflePosition()); + iter.updateStopPosition(cloudProgressToReaderProgress( + createApproximateProgress(proposedStopPosition)))).getShufflePosition()); // Cannot update stop position to a position >= the current stop position stop = encodeBase64URLSafeString(fabricatePosition(1, null)); proposedStopPosition.setShufflePosition(stop); - Assert.assertEquals(null, iter.updateStopPosition( - cloudProgressToSourceProgress( - createApproximateProgress(proposedStopPosition)))); + Assert.assertEquals( + null, + iter.updateStopPosition( + cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); } } @Test - public void testReadFromShuffleSourceAndFailToUpdateStopPosition() - throws Exception { + public void testReadFromShuffleDataAndFailToUpdateStopPosition() throws Exception { BatchModeExecutionContext context = new BatchModeExecutionContext(); final int kFirstShard = 0; @@ -327,37 +315,32 @@ public void testReadFromShuffleSourceAndFailToUpdateStopPosition() final int kNumRecords = 2; for (int i = 0; i < kNumRecords; ++i) { byte[] key = CoderUtils.encodeToByteArray(BigEndianIntegerCoder.of(), i); - shuffleReader.addEntry(new ShuffleEntry( - fabricatePosition(kFirstShard, key), key, null, key)); + shuffleReader.addEntry(new ShuffleEntry(fabricatePosition(kFirstShard, key), key, null, key)); } // Note that TestShuffleReader start/end positions are in the // space of keys not the positions (TODO: should probably always // use positions instead). - String stop = encodeBase64URLSafeString( - fabricatePosition(kNumRecords, null)); - GroupingShuffleSource shuffleSource = - new GroupingShuffleSource<>( - PipelineOptionsFactory.create(), - null, null, stop, - WindowedValue.getFullCoder( - KvCoder.of( - BigEndianIntegerCoder.of(), - IterableCoder.of(BigEndianIntegerCoder.of())), - IntervalWindow.getCoder()), - context); - - try (Source.SourceIterator>>> iter = - shuffleSource.iterator(shuffleReader)) { + String stop = encodeBase64URLSafeString(fabricatePosition(kNumRecords, null)); + GroupingShuffleReader groupingShuffleReader = new GroupingShuffleReader<>( + PipelineOptionsFactory.create(), null, null, stop, + WindowedValue.getFullCoder( + KvCoder.of(BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), + IntervalWindow.getCoder()), + context); + try (Reader.ReaderIterator>>> iter = + groupingShuffleReader.iterator(shuffleReader)) { Position proposedStopPosition = new Position(); proposedStopPosition.setShufflePosition( encodeBase64URLSafeString(fabricatePosition(kNumRecords + 1, null))); // Cannot update the stop position since the value provided is // past the current stop position. - Assert.assertEquals(null, iter.updateStopPosition( - cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + Assert.assertEquals( + null, + iter.updateStopPosition( + cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); int i = 0; for (; iter.hasNext(); ++i) { @@ -369,14 +352,18 @@ public void testReadFromShuffleSourceAndFailToUpdateStopPosition() encodeBase64URLSafeString(fabricatePosition(kFirstShard, key))); // Cannot update stop position since it is identical with // the position of the record that was just returned. - Assert.assertEquals(null, iter.updateStopPosition( - cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + Assert.assertEquals( + null, + iter.updateStopPosition( + cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); proposedStopPosition.setShufflePosition( encodeBase64URLSafeString(fabricatePosition(kFirstShard, null))); // Cannot update stop position since it comes before current position - Assert.assertEquals(null, iter.updateStopPosition( - cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + Assert.assertEquals( + null, + iter.updateStopPosition( + cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); } } Assert.assertEquals(kNumRecords, i); @@ -384,25 +371,22 @@ public void testReadFromShuffleSourceAndFailToUpdateStopPosition() proposedStopPosition.setShufflePosition( encodeBase64URLSafeString(fabricatePosition(kFirstShard, null))); // Cannot update stop position since all input was consumed. - Assert.assertEquals(null, iter.updateStopPosition( - cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + Assert.assertEquals( + null, + iter.updateStopPosition( + cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); } } @Test - public void testReadFromShuffleSourceAndUpdateStopPosition() - throws Exception { + public void testReadFromShuffleAndUpdateStopPosition() throws Exception { BatchModeExecutionContext context = new BatchModeExecutionContext(); - GroupingShuffleSource shuffleSource = - new GroupingShuffleSource<>( - PipelineOptionsFactory.create(), - null, null, null, - WindowedValue.getFullCoder( - KvCoder.of( - BigEndianIntegerCoder.of(), - IterableCoder.of(BigEndianIntegerCoder.of())), - IntervalWindow.getCoder()), - context); + GroupingShuffleReader groupingShuffleReader = new GroupingShuffleReader<>( + PipelineOptionsFactory.create(), null, null, null, + WindowedValue.getFullCoder( + KvCoder.of(BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), + IntervalWindow.getCoder()), + context); TestShuffleReader shuffleReader = new TestShuffleReader(); final int kNumRecords = 10; @@ -413,30 +397,27 @@ public void testReadFromShuffleSourceAndUpdateStopPosition() // (hence groups of values for the same key are singletons) // therefore each record comes with a unique position constructed. for (int i = 0; i < kNumRecords; ++i) { - byte[] keyByte = CoderUtils.encodeToByteArray( - BigEndianIntegerCoder.of(), i); - ShuffleEntry entry = new ShuffleEntry( - fabricatePosition(kFirstShard, keyByte), keyByte, null, keyByte); + byte[] keyByte = CoderUtils.encodeToByteArray(BigEndianIntegerCoder.of(), i); + ShuffleEntry entry = + new ShuffleEntry(fabricatePosition(kFirstShard, keyByte), keyByte, null, keyByte); shuffleReader.addEntry(entry); } for (int i = kNumRecords; i < 2 * kNumRecords; ++i) { - byte[] keyByte = CoderUtils.encodeToByteArray( - BigEndianIntegerCoder.of(), i); + byte[] keyByte = CoderUtils.encodeToByteArray(BigEndianIntegerCoder.of(), i); - ShuffleEntry entry = new ShuffleEntry( - fabricatePosition(kSecondShard, keyByte), keyByte, null, keyByte); + ShuffleEntry entry = + new ShuffleEntry(fabricatePosition(kSecondShard, keyByte), keyByte, null, keyByte); shuffleReader.addEntry(entry); } int i = 0; - try (Source.SourceIterator>>> iter = - shuffleSource.iterator(shuffleReader)) { - + try (Reader.ReaderIterator>>> iter = + groupingShuffleReader.iterator(shuffleReader)) { Position proposedStopPosition = new Position(); Assert.assertNull(iter.updateStopPosition( - cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); // Stop at the shard boundary String stop = encodeBase64URLSafeString(fabricatePosition(kSecondShard, null)); @@ -445,9 +426,8 @@ public void testReadFromShuffleSourceAndUpdateStopPosition() Assert.assertEquals( stop, sourcePositionToCloudPosition( - iter.updateStopPosition( - cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))) - .getShufflePosition()); + iter.updateStopPosition(cloudProgressToReaderProgress( + createApproximateProgress(proposedStopPosition)))).getShufflePosition()); while (iter.hasNext()) { Assert.assertTrue(iter.hasNext()); @@ -482,16 +462,12 @@ public void testGetApproximateProgress() throws Exception { List positionsList = new ArrayList(); BatchModeExecutionContext context = new BatchModeExecutionContext(); - GroupingShuffleSource shuffleSource = - new GroupingShuffleSource<>( - PipelineOptionsFactory.create(), - null, null, null, - WindowedValue.getFullCoder( - KvCoder.of( - BigEndianIntegerCoder.of(), - IterableCoder.of(BigEndianIntegerCoder.of())), - IntervalWindow.getCoder()), - context); + GroupingShuffleReader groupingShuffleReader = new GroupingShuffleReader<>( + PipelineOptionsFactory.create(), null, null, null, + WindowedValue.getFullCoder( + KvCoder.of(BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), + IntervalWindow.getCoder()), + context); TestShuffleReader shuffleReader = new TestShuffleReader(); final int kNumRecords = 10; @@ -504,30 +480,33 @@ public void testGetApproximateProgress() throws Exception { shuffleReader.addEntry(entry); } - try (Source.SourceIterator>>> sourceIterator = - shuffleSource.iterator(shuffleReader)) { + try (Reader.ReaderIterator>>> readerIterator = + groupingShuffleReader.iterator(shuffleReader)) { Integer i = 0; - while (sourceIterator.hasNext()) { - Assert.assertTrue(sourceIterator.hasNext()); - ApproximateProgress progress = sourceProgressToCloudProgress(sourceIterator.getProgress()); + while (readerIterator.hasNext()) { + Assert.assertTrue(readerIterator.hasNext()); + ApproximateProgress progress = sourceProgressToCloudProgress(readerIterator.getProgress()); Assert.assertNotNull(progress.getPosition().getShufflePosition()); // Compare returned position with the expected position. - Assert.assertEquals(ByteArrayShufflePosition.of(positionsList.get(i)).encodeBase64(), + Assert.assertEquals( + ByteArrayShufflePosition.of(positionsList.get(i)).encodeBase64(), progress.getPosition().getShufflePosition()); - WindowedValue>> elem = sourceIterator.next(); + WindowedValue>> elem = readerIterator.next(); Assert.assertEquals(i, elem.getValue().getKey()); i++; } - Assert.assertFalse(sourceIterator.hasNext()); + Assert.assertFalse(readerIterator.hasNext()); // Cannot update stop position since all input was consumed. Position proposedStopPosition = new Position(); String stop = encodeBase64URLSafeString(fabricatePosition(0, null)); proposedStopPosition.setShufflePosition(stop); - Assert.assertEquals(null, sourceIterator.updateStopPosition( - cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + Assert.assertEquals( + null, + readerIterator.updateStopPosition( + cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderFactoryTest.java similarity index 50% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactoryTest.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderFactoryTest.java index 82d2c82e99761..c10792d6cc37a 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderFactoryTest.java @@ -16,10 +16,11 @@ package com.google.cloud.dataflow.sdk.runners.worker; -import static com.google.cloud.dataflow.sdk.runners.worker.InMemorySourceTest.encodedElements; +import static com.google.cloud.dataflow.sdk.runners.worker.InMemoryReaderTest.encodedElements; import static com.google.cloud.dataflow.sdk.util.Structs.addLong; import static com.google.cloud.dataflow.sdk.util.Structs.addStringList; +import com.google.api.services.dataflow.model.Source; import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; @@ -27,7 +28,7 @@ import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.PropertyNames; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.hamcrest.core.IsInstanceOf; import org.junit.Assert; @@ -39,16 +40,12 @@ import java.util.List; /** - * Tests for InMemorySourceFactory. + * Tests for InMemoryReaderFactory. */ @RunWith(JUnit4.class) -public class InMemorySourceFactoryTest { - static com.google.api.services.dataflow.model.Source createInMemoryCloudSource( - List elements, - Long start, - Long end, - Coder coder) - throws Exception { +public class InMemoryReaderFactoryTest { + static Source createInMemoryCloudSource( + List elements, Long start, Long end, Coder coder) throws Exception { List encodedElements = encodedElements(elements, coder); CloudObject spec = CloudObject.forClassName("InMemorySource"); @@ -61,50 +58,36 @@ static com.google.api.services.dataflow.model.Source createInMemoryCloudSour addLong(spec, PropertyNames.END_INDEX, end); } - com.google.api.services.dataflow.model.Source cloudSource = - new com.google.api.services.dataflow.model.Source(); + Source cloudSource = new Source(); cloudSource.setSpec(spec); cloudSource.setCodec(coder.asCloudObject()); return cloudSource; } - void runTestCreateInMemorySource(List elements, - Long start, - Long end, - int expectedStart, - int expectedEnd, - Coder coder) - throws Exception { - com.google.api.services.dataflow.model.Source cloudSource = - createInMemoryCloudSource(elements, start, end, coder); + void runTestCreateInMemoryReader(List elements, Long start, Long end, int expectedStart, + int expectedEnd, Coder coder) throws Exception { + Source cloudSource = createInMemoryCloudSource(elements, start, end, coder); - Source source = SourceFactory.create(PipelineOptionsFactory.create(), cloudSource, - new BatchModeExecutionContext()); - Assert.assertThat(source, new IsInstanceOf(InMemorySource.class)); - InMemorySource inMemorySource = (InMemorySource) source; - Assert.assertEquals(encodedElements(elements, coder), - inMemorySource.encodedElements); - Assert.assertEquals(expectedStart, inMemorySource.startIndex); - Assert.assertEquals(expectedEnd, inMemorySource.endIndex); - Assert.assertEquals(coder, inMemorySource.coder); + Reader reader = ReaderFactory.create( + PipelineOptionsFactory.create(), cloudSource, new BatchModeExecutionContext()); + Assert.assertThat(reader, new IsInstanceOf(InMemoryReader.class)); + InMemoryReader inMemoryReader = (InMemoryReader) reader; + Assert.assertEquals(encodedElements(elements, coder), inMemoryReader.encodedElements); + Assert.assertEquals(expectedStart, inMemoryReader.startIndex); + Assert.assertEquals(expectedEnd, inMemoryReader.endIndex); + Assert.assertEquals(coder, inMemoryReader.coder); } @Test - public void testCreatePlainInMemorySource() throws Exception { - runTestCreateInMemorySource( - Arrays.asList("hi", "there", "bob"), - null, null, - 0, 3, - StringUtf8Coder.of()); + public void testCreatePlainInMemoryReader() throws Exception { + runTestCreateInMemoryReader( + Arrays.asList("hi", "there", "bob"), null, null, 0, 3, StringUtf8Coder.of()); } @Test - public void testCreateRichInMemorySource() throws Exception { - runTestCreateInMemorySource( - Arrays.asList(33, 44, 55, 66, 77, 88), - 1L, 3L, - 1, 3, - BigEndianIntegerCoder.of()); + public void testCreateRichInMemoryReader() throws Exception { + runTestCreateInMemoryReader( + Arrays.asList(33, 44, 55, 66, 77, 88), 1L, 3L, 1, 3, BigEndianIntegerCoder.of()); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java new file mode 100644 index 0000000000000..0068e5e275172 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java @@ -0,0 +1,190 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray; +import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString; + +import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.Position; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Tests for InMemoryReader. + */ +@RunWith(JUnit4.class) +public class InMemoryReaderTest { + static List encodedElements(List elements, Coder coder) throws Exception { + List encodedElements = new ArrayList<>(); + for (T element : elements) { + byte[] encodedElement = encodeToByteArray(coder, element); + String encodedElementString = byteArrayToJsonString(encodedElement); + encodedElements.add(encodedElementString); + } + return encodedElements; + } + + void runTestReadInMemory(List elements, Long startIndex, Long endIndex, + List expectedElements, List expectedSizes, Coder coder) throws Exception { + InMemoryReader inMemoryReader = + new InMemoryReader<>(encodedElements(elements, coder), startIndex, endIndex, coder); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(inMemoryReader); + List actualElements = new ArrayList<>(); + try (Reader.ReaderIterator iterator = inMemoryReader.iterator()) { + for (long i = inMemoryReader.startIndex; iterator.hasNext(); i++) { + Assert.assertEquals( + new ApproximateProgress().setPosition(makeIndexPosition(i)), + sourceProgressToCloudProgress(iterator.getProgress())); + actualElements.add(iterator.next()); + } + } + Assert.assertEquals(expectedElements, actualElements); + Assert.assertEquals(expectedSizes, observer.getActualSizes()); + } + + @Test + public void testReadAllElements() throws Exception { + runTestReadInMemory(Arrays.asList(33, 44, 55, 66, 77, 88), null, null, + Arrays.asList(33, 44, 55, 66, 77, 88), Arrays.asList(4, 4, 4, 4, 4, 4), + BigEndianIntegerCoder.of()); + } + + @Test + public void testReadElementsFromStart() throws Exception { + runTestReadInMemory(Arrays.asList(33, 44, 55, 66, 77, 88), 2L, null, + Arrays.asList(55, 66, 77, 88), Arrays.asList(4, 4, 4, 4), BigEndianIntegerCoder.of()); + } + + @Test + public void testReadElementsToEnd() throws Exception { + runTestReadInMemory(Arrays.asList(33, 44, 55, 66, 77, 88), null, 3L, Arrays.asList(33, 44, 55), + Arrays.asList(4, 4, 4), BigEndianIntegerCoder.of()); + } + + @Test + public void testReadElementsFromStartToEnd() throws Exception { + runTestReadInMemory(Arrays.asList(33, 44, 55, 66, 77, 88), 2L, 5L, Arrays.asList(55, 66, 77), + Arrays.asList(4, 4, 4), BigEndianIntegerCoder.of()); + } + + @Test + public void testReadElementsOffEnd() throws Exception { + runTestReadInMemory(Arrays.asList(33, 44, 55, 66, 77, 88), null, 30L, + Arrays.asList(33, 44, 55, 66, 77, 88), Arrays.asList(4, 4, 4, 4, 4, 4), + BigEndianIntegerCoder.of()); + } + + @Test + public void testReadElementsFromStartPastEnd() throws Exception { + runTestReadInMemory(Arrays.asList(33, 44, 55, 66, 77, 88), 20L, null, Arrays.asList(), + Arrays.asList(), BigEndianIntegerCoder.of()); + } + + @Test + public void testReadElementsFromStartToEndEmptyRange() throws Exception { + runTestReadInMemory(Arrays.asList(33, 44, 55, 66, 77, 88), 2L, 2L, Arrays.asList(), + Arrays.asList(), BigEndianIntegerCoder.of()); + } + + @Test + public void testReadNoElements() throws Exception { + runTestReadInMemory(Arrays.asList(), null, null, Arrays.asList(), + Arrays.asList(), BigEndianIntegerCoder.of()); + } + + @Test + public void testReadNoElementsFromStartToEndEmptyRange() throws Exception { + runTestReadInMemory(Arrays.asList(), 0L, 0L, Arrays.asList(), + Arrays.asList(), BigEndianIntegerCoder.of()); + } + + @Test + public void testUpdatePosition() throws Exception { + List elements = Arrays.asList(33, 44, 55, 66, 77, 88); + final long start = 1L; + final long stop = 3L; + final long end = 4L; + + Coder coder = BigEndianIntegerCoder.of(); + InMemoryReader inMemoryReader = + new InMemoryReader<>(encodedElements(elements, coder), start, end, coder); + + // Illegal proposed stop position. + try (Reader.ReaderIterator iterator = inMemoryReader.iterator()) { + Assert.assertNull( + iterator.updateStopPosition(cloudProgressToReaderProgress(new ApproximateProgress()))); + Assert.assertNull(iterator.updateStopPosition(cloudProgressToReaderProgress( + new ApproximateProgress().setPosition(makeIndexPosition(null))))); + } + + // Successful update. + try (InMemoryReader.InMemoryReaderIterator iterator = + (InMemoryReader.InMemoryReaderIterator) inMemoryReader.iterator()) { + Assert.assertEquals( + makeIndexPosition(stop), + sourcePositionToCloudPosition(iterator.updateStopPosition(cloudProgressToReaderProgress( + new ApproximateProgress().setPosition(makeIndexPosition(stop)))))); + Assert.assertEquals(stop, iterator.endPosition); + Assert.assertEquals(44, iterator.next().intValue()); + Assert.assertEquals(55, iterator.next().intValue()); + Assert.assertFalse(iterator.hasNext()); + } + + // Proposed stop position is before the current position, no update. + try (InMemoryReader.InMemoryReaderIterator iterator = + (InMemoryReader.InMemoryReaderIterator) inMemoryReader.iterator()) { + Assert.assertEquals(44, iterator.next().intValue()); + Assert.assertEquals(55, iterator.next().intValue()); + Assert.assertNull(iterator.updateStopPosition(cloudProgressToReaderProgress( + new ApproximateProgress().setPosition(makeIndexPosition(stop))))); + Assert.assertEquals((int) end, iterator.endPosition); + Assert.assertTrue(iterator.hasNext()); + } + + // Proposed stop position is after the current stop (end) position, no update. + try (InMemoryReader.InMemoryReaderIterator iterator = + (InMemoryReader.InMemoryReaderIterator) inMemoryReader.iterator()) { + Assert.assertNull(iterator.updateStopPosition(cloudProgressToReaderProgress( + new ApproximateProgress().setPosition(makeIndexPosition(end + 1))))); + Assert.assertEquals((int) end, iterator.endPosition); + } + } + + private Position makeIndexPosition(Long index) { + Position position = new Position(); + if (index != null) { + position.setRecordIndex(index); + } + return position; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceTest.java deleted file mode 100644 index d7574c517b4e8..0000000000000 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemorySourceTest.java +++ /dev/null @@ -1,236 +0,0 @@ -/* - * Copyright (C) 2014 Google Inc. - * - * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; - -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; -import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray; -import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString; - -import com.google.api.services.dataflow.model.ApproximateProgress; -import com.google.api.services.dataflow.model.Position; -import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; -import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; - -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -/** - * Tests for InMemorySource. - */ -@RunWith(JUnit4.class) -public class InMemorySourceTest { - static List encodedElements(List elements, Coder coder) - throws Exception { - List encodedElements = new ArrayList<>(); - for (T element : elements) { - byte[] encodedElement = encodeToByteArray(coder, element); - String encodedElementString = byteArrayToJsonString(encodedElement); - encodedElements.add(encodedElementString); - } - return encodedElements; - } - - void runTestReadInMemorySource(List elements, - Long startIndex, - Long endIndex, - List expectedElements, - List expectedSizes, - Coder coder) - throws Exception { - InMemorySource inMemorySource = new InMemorySource<>( - encodedElements(elements, coder), startIndex, endIndex, coder); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(inMemorySource); - List actualElements = new ArrayList<>(); - try (Source.SourceIterator iterator = inMemorySource.iterator()) { - for (long i = inMemorySource.startIndex; iterator.hasNext(); i++) { - Assert.assertEquals( - new ApproximateProgress().setPosition(makeIndexPosition(i)), - sourceProgressToCloudProgress(iterator.getProgress())); - actualElements.add(iterator.next()); - } - } - Assert.assertEquals(expectedElements, actualElements); - Assert.assertEquals(expectedSizes, observer.getActualSizes()); - } - - @Test - public void testReadAllElements() throws Exception { - runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), - null, - null, - Arrays.asList(33, 44, 55, 66, 77, 88), - Arrays.asList(4, 4, 4, 4, 4, 4), - BigEndianIntegerCoder.of()); - } - - @Test - public void testReadElementsFromStart() throws Exception { - runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), - 2L, - null, - Arrays.asList(55, 66, 77, 88), - Arrays.asList(4, 4, 4, 4), - BigEndianIntegerCoder.of()); - } - - @Test - public void testReadElementsToEnd() throws Exception { - runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), - null, - 3L, - Arrays.asList(33, 44, 55), - Arrays.asList(4, 4, 4), - BigEndianIntegerCoder.of()); - } - - @Test - public void testReadElementsFromStartToEnd() throws Exception { - runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), - 2L, - 5L, - Arrays.asList(55, 66, 77), - Arrays.asList(4, 4, 4), - BigEndianIntegerCoder.of()); - } - - @Test - public void testReadElementsOffEnd() throws Exception { - runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), - null, - 30L, - Arrays.asList(33, 44, 55, 66, 77, 88), - Arrays.asList(4, 4, 4, 4, 4, 4), - BigEndianIntegerCoder.of()); - } - - @Test - public void testReadElementsFromStartPastEnd() throws Exception { - runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), - 20L, - null, - Arrays.asList(), - Arrays.asList(), - BigEndianIntegerCoder.of()); - } - - @Test - public void testReadElementsFromStartToEndEmptyRange() throws Exception { - runTestReadInMemorySource(Arrays.asList(33, 44, 55, 66, 77, 88), - 2L, - 2L, - Arrays.asList(), - Arrays.asList(), - BigEndianIntegerCoder.of()); - } - - @Test - public void testReadNoElements() throws Exception { - runTestReadInMemorySource(Arrays.asList(), - null, - null, - Arrays.asList(), - Arrays.asList(), - BigEndianIntegerCoder.of()); - } - - @Test - public void testReadNoElementsFromStartToEndEmptyRange() throws Exception { - runTestReadInMemorySource(Arrays.asList(), - 0L, - 0L, - Arrays.asList(), - Arrays.asList(), - BigEndianIntegerCoder.of()); - } - - @Test - public void testUpdatePosition() throws Exception { - List elements = Arrays.asList(33, 44, 55, 66, 77, 88); - final long start = 1L; - final long stop = 3L; - final long end = 4L; - - Coder coder = BigEndianIntegerCoder.of(); - InMemorySource inMemorySource = new InMemorySource<>( - encodedElements(elements, coder), start, end, coder); - - // Illegal proposed stop position. - try (Source.SourceIterator iterator = inMemorySource.iterator()) { - Assert.assertNull(iterator.updateStopPosition( - cloudProgressToSourceProgress(new ApproximateProgress()))); - Assert.assertNull(iterator.updateStopPosition( - cloudProgressToSourceProgress( - new ApproximateProgress().setPosition(makeIndexPosition(null))))); - } - - // Successful update. - try (InMemorySource.InMemorySourceIterator iterator = - (InMemorySource.InMemorySourceIterator) inMemorySource.iterator()) { - Assert.assertEquals( - makeIndexPosition(stop), - sourcePositionToCloudPosition( - iterator.updateStopPosition( - cloudProgressToSourceProgress( - new ApproximateProgress().setPosition(makeIndexPosition(stop)))))); - Assert.assertEquals(stop, iterator.endPosition); - Assert.assertEquals(44, iterator.next().intValue()); - Assert.assertEquals(55, iterator.next().intValue()); - Assert.assertFalse(iterator.hasNext()); - } - - // Proposed stop position is before the current position, no update. - try (InMemorySource.InMemorySourceIterator iterator = - (InMemorySource.InMemorySourceIterator) inMemorySource.iterator()) { - Assert.assertEquals(44, iterator.next().intValue()); - Assert.assertEquals(55, iterator.next().intValue()); - Assert.assertNull(iterator.updateStopPosition( - cloudProgressToSourceProgress( - new ApproximateProgress().setPosition(makeIndexPosition(stop))))); - Assert.assertEquals((int) end, iterator.endPosition); - Assert.assertTrue(iterator.hasNext()); - } - - // Proposed stop position is after the current stop (end) position, no update. - try (InMemorySource.InMemorySourceIterator iterator = - (InMemorySource.InMemorySourceIterator) inMemorySource.iterator()) { - Assert.assertNull( - iterator.updateStopPosition( - cloudProgressToSourceProgress( - new ApproximateProgress().setPosition(makeIndexPosition(end + 1))))); - Assert.assertEquals((int) end, iterator.endPosition); - } - } - - private Position makeIndexPosition(Long index) { - Position position = new Position(); - if (index != null) { - position.setRecordIndex(index); - } - return position; - } -} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java index 2060865759331..445755130db5d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java @@ -33,13 +33,14 @@ 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.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.worker.ReaderFactoryTest.TestReader; +import com.google.cloud.dataflow.sdk.runners.worker.ReaderFactoryTest.TestReaderFactory; import com.google.cloud.dataflow.sdk.runners.worker.SinkFactoryTest.TestSink; import com.google.cloud.dataflow.sdk.runners.worker.SinkFactoryTest.TestSinkFactory; -import com.google.cloud.dataflow.sdk.runners.worker.SourceFactoryTest.TestSource; -import com.google.cloud.dataflow.sdk.runners.worker.SourceFactoryTest.TestSourceFactory; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; @@ -80,93 +81,67 @@ public class MapTaskExecutorFactoryTest { @Test public void testCreateMapTaskExecutor() throws Exception { - List instructions = - Arrays.asList( - createReadInstruction("Read"), - createParDoInstruction(0, 0, "DoFn1"), - createParDoInstruction(0, 0, "DoFn2"), - createFlattenInstruction(1, 0, 2, 0, "Flatten"), - createWriteInstruction(3, 0, "Write")); + List instructions = Arrays.asList(createReadInstruction("Read"), + createParDoInstruction(0, 0, "DoFn1"), createParDoInstruction(0, 0, "DoFn2"), + createFlattenInstruction(1, 0, 2, 0, "Flatten"), createWriteInstruction(3, 0, "Write")); MapTask mapTask = new MapTask(); mapTask.setStageName("test"); mapTask.setInstructions(instructions); CounterSet counterSet = null; - try (MapTaskExecutor executor = - MapTaskExecutorFactory.create( - PipelineOptionsFactory.create(), - mapTask, - new BatchModeExecutionContext())) { - + try ( + MapTaskExecutor executor = MapTaskExecutorFactory.create( + PipelineOptionsFactory.create(), mapTask, new BatchModeExecutionContext())) { @SuppressWarnings("unchecked") List operations = (List) executor.operations; assertThat( operations, - CoreMatchers.hasItems( - new IsInstanceOf(ReadOperation.class), - new IsInstanceOf(ParDoOperation.class), - new IsInstanceOf(ParDoOperation.class), - new IsInstanceOf(FlattenOperation.class), - new IsInstanceOf(WriteOperation.class))); + CoreMatchers.hasItems(new IsInstanceOf(ReadOperation.class), + new IsInstanceOf(ParDoOperation.class), new IsInstanceOf(ParDoOperation.class), + new IsInstanceOf(FlattenOperation.class), new IsInstanceOf(WriteOperation.class))); counterSet = executor.getOutputCounters(); } assertEquals( - new CounterSet( - Counter.longs("read_output_name-ElementCount", SUM) - .resetToValue(0L), - Counter.longs("read_output_name-MeanByteCount", MEAN) - .resetToValue(0, 0L), + new CounterSet(Counter.longs("read_output_name-ElementCount", SUM).resetToValue(0L), + Counter.longs("read_output_name-MeanByteCount", MEAN).resetToValue(0, 0L), Counter.longs("Read-ByteCount", SUM).resetToValue(0L), - Counter.longs("test-Read-start-msecs", SUM) - .resetToValue(0L), - Counter.longs("test-Read-read-msecs", SUM) - .resetToValue(0L), - Counter.longs("test-Read-process-msecs", SUM) - .resetToValue(0L), - Counter.longs("test-Read-finish-msecs", SUM) - .resetToValue(0L), - Counter.longs("DoFn1_output-ElementCount", SUM) - .resetToValue(0L), - Counter.longs("DoFn1_output-MeanByteCount", MEAN) - .resetToValue(0, 0L), + Counter.longs("test-Read-start-msecs", SUM).resetToValue(0L), + Counter.longs("test-Read-read-msecs", SUM).resetToValue(0L), + Counter.longs("test-Read-process-msecs", SUM).resetToValue(0L), + Counter.longs("test-Read-finish-msecs", SUM).resetToValue(0L), + Counter.longs("DoFn1_output-ElementCount", SUM).resetToValue(0L), + Counter.longs("DoFn1_output-MeanByteCount", MEAN).resetToValue(0, 0L), Counter.longs("test-DoFn1-start-msecs", SUM).resetToValue(0L), Counter.longs("test-DoFn1-process-msecs", SUM).resetToValue(0L), Counter.longs("test-DoFn1-finish-msecs", SUM).resetToValue(0L), - Counter.longs("DoFn2_output-ElementCount", SUM) - .resetToValue(0L), - Counter.longs("DoFn2_output-MeanByteCount", MEAN) - .resetToValue(0, 0L), + Counter.longs("DoFn2_output-ElementCount", SUM).resetToValue(0L), + Counter.longs("DoFn2_output-MeanByteCount", MEAN).resetToValue(0, 0L), Counter.longs("test-DoFn2-start-msecs", SUM).resetToValue(0L), Counter.longs("test-DoFn2-process-msecs", SUM).resetToValue(0L), Counter.longs("test-DoFn2-finish-msecs", SUM).resetToValue(0L), - Counter.longs("flatten_output_name-ElementCount", SUM) - .resetToValue(0L), - Counter.longs("flatten_output_name-MeanByteCount", MEAN) - .resetToValue(0, 0L), + Counter.longs("flatten_output_name-ElementCount", SUM).resetToValue(0L), + Counter.longs("flatten_output_name-MeanByteCount", MEAN).resetToValue(0, 0L), Counter.longs("test-Flatten-start-msecs", SUM).resetToValue(0L), Counter.longs("test-Flatten-process-msecs", SUM).resetToValue(0L), Counter.longs("test-Flatten-finish-msecs", SUM).resetToValue(0L), - Counter.longs("Write-ByteCount", SUM) - .resetToValue(0L), + Counter.longs("Write-ByteCount", SUM).resetToValue(0L), Counter.longs("test-Write-start-msecs", SUM).resetToValue(0L), Counter.longs("test-Write-process-msecs", SUM).resetToValue(0L), Counter.longs("test-Write-finish-msecs", SUM).resetToValue(0L), Counter.longs("test-other-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-other-msecs")).getAggregate(false))), + .resetToValue( + ((Counter) + counterSet.getExistingCounter("test-other-msecs")).getAggregate(false))), counterSet); } @Test public void testExecutionContextPlumbing() throws Exception { List instructions = - Arrays.asList( - createReadInstruction("Read"), - createParDoInstruction(0, 0, "DoFn1"), - createParDoInstruction(1, 0, "DoFn2"), - createWriteInstruction(2, 0, "Write")); + Arrays.asList(createReadInstruction("Read"), createParDoInstruction(0, 0, "DoFn1"), + createParDoInstruction(1, 0, "DoFn2"), createWriteInstruction(2, 0, "Write")); MapTask mapTask = new MapTask(); mapTask.setInstructions(instructions); @@ -174,24 +149,21 @@ public void testExecutionContextPlumbing() throws Exception { BatchModeExecutionContext context = new BatchModeExecutionContext(); try (MapTaskExecutor executor = - MapTaskExecutorFactory.create( - PipelineOptionsFactory.create(), mapTask, context)) { + MapTaskExecutorFactory.create(PipelineOptionsFactory.create(), mapTask, context)) { executor.execute(); } List stepNames = new ArrayList<>(); - for (ExecutionContext.StepContext stepContext - : context.getAllStepContexts()) { + for (ExecutionContext.StepContext stepContext : context.getAllStepContexts()) { stepNames.add(stepContext.getStepName()); } assertThat(stepNames, CoreMatchers.hasItems("DoFn1", "DoFn2")); } static ParallelInstruction createReadInstruction(String name) { - CloudObject spec = CloudObject.forClass(TestSourceFactory.class); + CloudObject spec = CloudObject.forClass(TestReaderFactory.class); - com.google.api.services.dataflow.model.Source cloudSource = - new com.google.api.services.dataflow.model.Source(); + Source cloudSource = new Source(); cloudSource.setSpec(spec); cloudSource.setCodec(CloudObject.forClass(StringUtf8Coder.class)); @@ -214,34 +186,25 @@ static ParallelInstruction createReadInstruction(String name) { public void testCreateReadOperation() throws Exception { CounterSet counterSet = new CounterSet(); String counterPrefix = "test-"; - StateSampler stateSampler = new StateSampler(counterPrefix, - counterSet.getAddCounterMutator()); - Operation operation = MapTaskExecutorFactory.createOperation( - PipelineOptionsFactory.create(), - createReadInstruction("Read"), - new BatchModeExecutionContext(), - Collections.emptyList(), - counterPrefix, - counterSet.getAddCounterMutator(), + StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); + Operation operation = MapTaskExecutorFactory.createOperation(PipelineOptionsFactory.create(), + createReadInstruction("Read"), new BatchModeExecutionContext(), + Collections.emptyList(), counterPrefix, counterSet.getAddCounterMutator(), stateSampler); assertThat(operation, new IsInstanceOf(ReadOperation.class)); ReadOperation readOperation = (ReadOperation) operation; assertEquals(readOperation.receivers.length, 1); assertEquals(readOperation.receivers[0].getReceiverCount(), 0); - assertEquals(readOperation.initializationState, - Operation.InitializationState.UNSTARTED); - assertThat(readOperation.source, new IsInstanceOf(TestSource.class)); + assertEquals(readOperation.initializationState, Operation.InitializationState.UNSTARTED); + assertThat(readOperation.reader, new IsInstanceOf(TestReader.class)); assertEquals( new CounterSet( - Counter.longs("test-Read-start-msecs", SUM) - .resetToValue(0L), - Counter.longs("read_output_name-MeanByteCount", MEAN) - .resetToValue(0, 0L), + Counter.longs("test-Read-start-msecs", SUM).resetToValue(0L), + Counter.longs("read_output_name-MeanByteCount", MEAN).resetToValue(0, 0L), Counter.longs("Read-ByteCount", SUM).resetToValue(0L), - Counter.longs("test-Read-finish-msecs", SUM) - .resetToValue(0L), + Counter.longs("test-Read-finish-msecs", SUM).resetToValue(0L), Counter.longs("test-Read-read-msecs", SUM), Counter.longs("test-Read-process-msecs", SUM), Counter.longs("read_output_name-ElementCount", SUM).resetToValue(0L)), @@ -249,9 +212,7 @@ public void testCreateReadOperation() throws Exception { } static ParallelInstruction createWriteInstruction( - int producerIndex, - int producerOutputNum, - String systemName) { + int producerIndex, int producerOutputNum, String systemName) { InstructionInput cloudInput = new InstructionInput(); cloudInput.setProducerInstructionIndex(producerIndex); cloudInput.setOutputNum(producerOutputNum); @@ -276,10 +237,8 @@ static ParallelInstruction createWriteInstruction( @Test public void testCreateWriteOperation() throws Exception { - List priorOperations = Arrays.asList(new Operation[]{ - new TestOperation(3), - new TestOperation(5), - new TestOperation(1) }); + List priorOperations = Arrays.asList( + new Operation[] {new TestOperation(3), new TestOperation(5), new TestOperation(1)}); int producerIndex = 1; int producerOutputNum = 2; @@ -289,55 +248,42 @@ public void testCreateWriteOperation() throws Exception { CounterSet counterSet = new CounterSet(); String counterPrefix = "test-"; - StateSampler stateSampler = new StateSampler(counterPrefix, - counterSet.getAddCounterMutator()); - Operation operation = MapTaskExecutorFactory.createOperation( - PipelineOptionsFactory.create(), - instruction, - new BatchModeExecutionContext(), - priorOperations, - counterPrefix, - counterSet.getAddCounterMutator(), - stateSampler); + StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); + Operation operation = MapTaskExecutorFactory.createOperation(PipelineOptionsFactory.create(), + instruction, new BatchModeExecutionContext(), priorOperations, counterPrefix, + counterSet.getAddCounterMutator(), stateSampler); assertThat(operation, new IsInstanceOf(WriteOperation.class)); WriteOperation writeOperation = (WriteOperation) operation; assertEquals(writeOperation.receivers.length, 0); - assertEquals(writeOperation.initializationState, - Operation.InitializationState.UNSTARTED); - assertThat(writeOperation.sink, - new IsInstanceOf(TestSink.class)); + assertEquals(writeOperation.initializationState, Operation.InitializationState.UNSTARTED); + assertThat(writeOperation.sink, new IsInstanceOf(TestSink.class)); assertSame( writeOperation, - priorOperations.get(producerIndex).receivers[producerOutputNum] - .getOnlyReceiver()); + priorOperations.get(producerIndex).receivers[producerOutputNum].getOnlyReceiver()); assertEquals( - new CounterSet( - Counter.longs("WriteOperation-ByteCount", SUM) - .resetToValue(0L), + new CounterSet(Counter.longs("WriteOperation-ByteCount", SUM).resetToValue(0L), Counter.longs("test-WriteOperation-start-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-WriteOperation-start-msecs")).getAggregate(false)), + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-WriteOperation-start-msecs")).getAggregate(false)), Counter.longs("test-WriteOperation-process-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-WriteOperation-process-msecs")).getAggregate(false)), + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-WriteOperation-process-msecs")).getAggregate(false)), Counter.longs("test-WriteOperation-finish-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-WriteOperation-finish-msecs")).getAggregate(false))), + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-WriteOperation-finish-msecs")).getAggregate(false))), counterSet); } static class TestDoFn extends DoFn { @Override - public void processElement(ProcessContext c) { } + public void processElement(ProcessContext c) {} } static ParallelInstruction createParDoInstruction( - int producerIndex, - int producerOutputNum, - String systemName) { + int producerIndex, int producerOutputNum, String systemName) { InstructionInput cloudInput = new InstructionInput(); cloudInput.setProducerInstructionIndex(producerIndex); cloudInput.setOutputNum(producerOutputNum); @@ -345,8 +291,7 @@ static ParallelInstruction createParDoInstruction( TestDoFn fn = new TestDoFn(); String serializedFn = - StringUtils.byteArrayToJsonString( - SerializableUtils.serializeToByteArray(fn)); + StringUtils.byteArrayToJsonString(SerializableUtils.serializeToByteArray(fn)); CloudObject cloudUserFn = CloudObject.forClassName("DoFn"); addString(cloudUserFn, PropertyNames.SERIALIZED_FN, serializedFn); @@ -369,10 +314,8 @@ static ParallelInstruction createParDoInstruction( @Test public void testCreateParDoOperation() throws Exception { - List priorOperations = Arrays.asList(new Operation[]{ - new TestOperation(3), - new TestOperation(5), - new TestOperation(1) }); + List priorOperations = Arrays.asList( + new Operation[] {new TestOperation(3), new TestOperation(5), new TestOperation(1)}); int producerIndex = 1; int producerOutputNum = 2; @@ -383,62 +326,46 @@ public void testCreateParDoOperation() throws Exception { BatchModeExecutionContext context = new BatchModeExecutionContext(); CounterSet counterSet = new CounterSet(); String counterPrefix = "test-"; - StateSampler stateSampler = new StateSampler(counterPrefix, - counterSet.getAddCounterMutator()); - Operation operation = MapTaskExecutorFactory.createOperation( - PipelineOptionsFactory.create(), - instruction, - context, - priorOperations, - counterPrefix, - counterSet.getAddCounterMutator(), stateSampler); + StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); + Operation operation = MapTaskExecutorFactory.createOperation(PipelineOptionsFactory.create(), + instruction, context, priorOperations, counterPrefix, counterSet.getAddCounterMutator(), + stateSampler); assertThat(operation, new IsInstanceOf(ParDoOperation.class)); ParDoOperation parDoOperation = (ParDoOperation) operation; assertEquals(parDoOperation.receivers.length, 1); assertEquals(parDoOperation.receivers[0].getReceiverCount(), 0); - assertEquals(parDoOperation.initializationState, - Operation.InitializationState.UNSTARTED); - assertThat(parDoOperation.fn, - new IsInstanceOf(NormalParDoFn.class)); + assertEquals(parDoOperation.initializationState, Operation.InitializationState.UNSTARTED); + assertThat(parDoOperation.fn, new IsInstanceOf(NormalParDoFn.class)); NormalParDoFn normalParDoFn = (NormalParDoFn) parDoOperation.fn; - assertThat(normalParDoFn.fn, - new IsInstanceOf(TestDoFn.class)); + assertThat(normalParDoFn.fn, new IsInstanceOf(TestDoFn.class)); assertSame( parDoOperation, - priorOperations.get(producerIndex).receivers[producerOutputNum] - .getOnlyReceiver()); + priorOperations.get(producerIndex).receivers[producerOutputNum].getOnlyReceiver()); assertEquals(context, normalParDoFn.executionContext); } static ParallelInstruction createPartialGroupByKeyInstruction( - int producerIndex, - int producerOutputNum) { + int producerIndex, int producerOutputNum) { InstructionInput cloudInput = new InstructionInput(); cloudInput.setProducerInstructionIndex(producerIndex); cloudInput.setOutputNum(producerOutputNum); - PartialGroupByKeyInstruction pgbkInstruction = - new PartialGroupByKeyInstruction(); + PartialGroupByKeyInstruction pgbkInstruction = new PartialGroupByKeyInstruction(); pgbkInstruction.setInput(cloudInput); - pgbkInstruction.setInputElementCodec( - makeCloudEncoding(FullWindowedValueCoder.class.getName(), - makeCloudEncoding("KvCoder", - makeCloudEncoding("StringUtf8Coder"), - makeCloudEncoding("BigEndianIntegerCoder")), - IntervalWindow.getCoder().asCloudObject())); + pgbkInstruction.setInputElementCodec(makeCloudEncoding( + FullWindowedValueCoder.class.getName(), + makeCloudEncoding("KvCoder", makeCloudEncoding("StringUtf8Coder"), + makeCloudEncoding("BigEndianIntegerCoder")), + IntervalWindow.getCoder().asCloudObject())); InstructionOutput output = new InstructionOutput(); output.setName("pgbk_output_name"); - output.setCodec(makeCloudEncoding( - "KvCoder", - makeCloudEncoding("StringUtf8Coder"), - makeCloudEncoding( - "IterableCoder", - makeCloudEncoding("BigEndianIntegerCoder")))); + output.setCodec(makeCloudEncoding("KvCoder", makeCloudEncoding("StringUtf8Coder"), + makeCloudEncoding("IterableCoder", makeCloudEncoding("BigEndianIntegerCoder")))); ParallelInstruction instruction = new ParallelInstruction(); instruction.setPartialGroupByKey(pgbkInstruction); @@ -449,10 +376,8 @@ static ParallelInstruction createPartialGroupByKeyInstruction( @Test public void testCreatePartialGroupByKeyOperation() throws Exception { - List priorOperations = Arrays.asList(new Operation[]{ - new TestOperation(3), - new TestOperation(5), - new TestOperation(1) }); + List priorOperations = Arrays.asList( + new Operation[] {new TestOperation(3), new TestOperation(5), new TestOperation(1)}); int producerIndex = 1; int producerOutputNum = 2; @@ -462,37 +387,24 @@ public void testCreatePartialGroupByKeyOperation() throws Exception { CounterSet counterSet = new CounterSet(); String counterPrefix = "test-"; - StateSampler stateSampler = new StateSampler(counterPrefix, - counterSet.getAddCounterMutator()); - Operation operation = MapTaskExecutorFactory.createOperation( - PipelineOptionsFactory.create(), - instruction, - new BatchModeExecutionContext(), - priorOperations, - counterPrefix, - counterSet.getAddCounterMutator(), - stateSampler); + StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); + Operation operation = MapTaskExecutorFactory.createOperation(PipelineOptionsFactory.create(), + instruction, new BatchModeExecutionContext(), priorOperations, counterPrefix, + counterSet.getAddCounterMutator(), stateSampler); assertThat(operation, instanceOf(PartialGroupByKeyOperation.class)); - PartialGroupByKeyOperation pgbkOperation = - (PartialGroupByKeyOperation) operation; + PartialGroupByKeyOperation pgbkOperation = (PartialGroupByKeyOperation) operation; assertEquals(pgbkOperation.receivers.length, 1); assertEquals(pgbkOperation.receivers[0].getReceiverCount(), 0); - assertEquals(pgbkOperation.initializationState, - Operation.InitializationState.UNSTARTED); + assertEquals(pgbkOperation.initializationState, Operation.InitializationState.UNSTARTED); assertSame( pgbkOperation, - priorOperations.get(producerIndex).receivers[producerOutputNum] - .getOnlyReceiver()); + priorOperations.get(producerIndex).receivers[producerOutputNum].getOnlyReceiver()); } - static ParallelInstruction createFlattenInstruction( - int producerIndex1, - int producerOutputNum1, - int producerIndex2, - int producerOutputNum2, - String systemName) { + static ParallelInstruction createFlattenInstruction(int producerIndex1, int producerOutputNum1, + int producerIndex2, int producerOutputNum2, String systemName) { List cloudInputs = new ArrayList<>(); InstructionInput cloudInput1 = new InstructionInput(); @@ -522,47 +434,35 @@ static ParallelInstruction createFlattenInstruction( @Test public void testCreateFlattenOperation() throws Exception { - List priorOperations = Arrays.asList(new Operation[]{ - new TestOperation(3), - new TestOperation(5), - new TestOperation(1) }); + List priorOperations = Arrays.asList( + new Operation[] {new TestOperation(3), new TestOperation(5), new TestOperation(1)}); int producerIndex1 = 1; int producerOutputNum1 = 2; int producerIndex2 = 0; int producerOutputNum2 = 1; - ParallelInstruction instruction = - createFlattenInstruction(producerIndex1, producerOutputNum1, - producerIndex2, producerOutputNum2, "Flatten"); + ParallelInstruction instruction = createFlattenInstruction( + producerIndex1, producerOutputNum1, producerIndex2, producerOutputNum2, "Flatten"); CounterSet counterSet = new CounterSet(); String counterPrefix = "test-"; - StateSampler stateSampler = new StateSampler(counterPrefix, - counterSet.getAddCounterMutator()); - Operation operation = MapTaskExecutorFactory.createOperation( - PipelineOptionsFactory.create(), - instruction, - new BatchModeExecutionContext(), - priorOperations, - counterPrefix, - counterSet.getAddCounterMutator(), - stateSampler); + StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); + Operation operation = MapTaskExecutorFactory.createOperation(PipelineOptionsFactory.create(), + instruction, new BatchModeExecutionContext(), priorOperations, counterPrefix, + counterSet.getAddCounterMutator(), stateSampler); assertThat(operation, new IsInstanceOf(FlattenOperation.class)); FlattenOperation flattenOperation = (FlattenOperation) operation; assertEquals(flattenOperation.receivers.length, 1); assertEquals(flattenOperation.receivers[0].getReceiverCount(), 0); - assertEquals(flattenOperation.initializationState, - Operation.InitializationState.UNSTARTED); + assertEquals(flattenOperation.initializationState, Operation.InitializationState.UNSTARTED); assertSame( flattenOperation, - priorOperations.get(producerIndex1).receivers[producerOutputNum1] - .getOnlyReceiver()); + priorOperations.get(producerIndex1).receivers[producerOutputNum1].getOnlyReceiver()); assertSame( flattenOperation, - priorOperations.get(producerIndex2).receivers[producerOutputNum2] - .getOnlyReceiver()); + priorOperations.get(producerIndex2).receivers[producerOutputNum2].getOnlyReceiver()); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleReaderTest.java similarity index 78% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceTest.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleReaderTest.java index 2b88456011dc4..11d7c0616cf94 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/PartitioningShuffleReaderTest.java @@ -24,9 +24,9 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; import com.google.cloud.dataflow.sdk.util.common.worker.Sink; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; import com.google.cloud.dataflow.sdk.values.KV; import com.google.common.collect.Lists; @@ -43,10 +43,10 @@ import java.util.NoSuchElementException; /** - * Tests for PartitioningShuffleSource. + * Tests for PartitioningShuffleReader. */ @RunWith(JUnit4.class) -public class PartitioningShuffleSourceTest { +public class PartitioningShuffleReaderTest { private static final List>> NO_KVS = Collections.emptyList(); private static final Instant timestamp = new Instant(123000); @@ -64,23 +64,20 @@ public class PartitioningShuffleSourceTest { WindowedValue.of(KV.of(4, "in 4d"), timestamp, Lists.newArrayList(window)), WindowedValue.of(KV.of(5, "in 5"), timestamp, Lists.newArrayList(window))); - private void runTestReadShuffleSource(List>> expected) + private void runTestReadFromShuffle(List>> expected) throws Exception { Coder>> elemCoder = WindowedValue.getFullCoder( - KvCoder.of(BigEndianIntegerCoder.of(), StringUtf8Coder.of()), - IntervalWindow.getCoder()); + KvCoder.of(BigEndianIntegerCoder.of(), StringUtf8Coder.of()), IntervalWindow.getCoder()); // Write to shuffle with PARTITION_KEYS ShuffleSink. ShuffleSink> shuffleSink = new ShuffleSink<>( - PipelineOptionsFactory.create(), - null, ShuffleSink.ShuffleKind.PARTITION_KEYS, - elemCoder); + PipelineOptionsFactory.create(), null, ShuffleSink.ShuffleKind.PARTITION_KEYS, elemCoder); TestShuffleWriter shuffleWriter = new TestShuffleWriter(); List actualSizes = new ArrayList<>(); try (Sink.SinkWriter>> shuffleSinkWriter = - shuffleSink.writer(shuffleWriter)) { + shuffleSink.writer(shuffleWriter)) { for (WindowedValue> value : expected) { actualSizes.add(shuffleSinkWriter.add(value)); } @@ -89,14 +86,12 @@ private void runTestReadShuffleSource(List>> e Assert.assertEquals(expected.size(), records.size()); Assert.assertEquals(shuffleWriter.getSizes(), actualSizes); - // Read from shuffle with PartitioningShuffleSource. - PartitioningShuffleSource shuffleSource = - new PartitioningShuffleSource<>( - PipelineOptionsFactory.create(), - null, null, null, - elemCoder); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(shuffleSource); + // Read from shuffle with PartitioningShuffleReader. + PartitioningShuffleReader partitioningShuffleReader = + new PartitioningShuffleReader<>( + PipelineOptionsFactory.create(), null, null, null, elemCoder); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(partitioningShuffleReader); TestShuffleReader shuffleReader = new TestShuffleReader(); List expectedSizes = new ArrayList<>(); @@ -106,8 +101,8 @@ private void runTestReadShuffleSource(List>> e } List>> actual = new ArrayList<>(); - try (Source.SourceIterator>> iter = - shuffleSource.iterator(shuffleReader)) { + try (Reader.ReaderIterator>> iter = + partitioningShuffleReader.iterator(shuffleReader)) { while (iter.hasNext()) { Assert.assertTrue(iter.hasNext()); actual.add(iter.next()); @@ -126,12 +121,12 @@ private void runTestReadShuffleSource(List>> e } @Test - public void testReadEmptyShuffleSource() throws Exception { - runTestReadShuffleSource(NO_KVS); + public void testReadEmptyShuffleData() throws Exception { + runTestReadFromShuffle(NO_KVS); } @Test - public void testReadNonEmptyShuffleSource() throws Exception { - runTestReadShuffleSource(KVS); + public void testReadNonEmptyShuffleData() throws Exception { + runTestReadFromShuffle(KVS); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderFactoryTest.java similarity index 52% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactoryTest.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderFactoryTest.java index 4b4665b55869d..1a3ab6a8f6a2a 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderFactoryTest.java @@ -19,13 +19,14 @@ import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; import static com.google.cloud.dataflow.sdk.util.Structs.addString; +import com.google.api.services.dataflow.model.Source; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.ExecutionContext; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.hamcrest.CoreMatchers; import org.hamcrest.core.IsInstanceOf; @@ -37,29 +38,29 @@ import java.util.NoSuchElementException; /** - * Tests for SourceFactory. + * Tests for ReaderFactory. */ @RunWith(JUnit4.class) -public class SourceFactoryTest { - static class TestSourceFactory { - public static TestSource create(PipelineOptions options, - CloudObject o, - Coder coder, - ExecutionContext executionContext) { - return new TestSource(); +public class ReaderFactoryTest { + static class TestReaderFactory { + public static TestReader create(PipelineOptions options, CloudObject o, Coder coder, + ExecutionContext executionContext) { + return new TestReader(); } } - static class TestSource extends Source { + static class TestReader extends Reader { @Override - public SourceIterator iterator() { - return new TestSourceIterator(); + public ReaderIterator iterator() { + return new TestReaderIterator(); } /** A source iterator that produces no values, for testing. */ - class TestSourceIterator extends AbstractSourceIterator { + class TestReaderIterator extends AbstractReaderIterator { @Override - public boolean hasNext() { return false; } + public boolean hasNext() { + return false; + } @Override public Integer next() { @@ -67,58 +68,49 @@ public Integer next() { } @Override - public void close() { - } + public void close() {} } } @Test - public void testCreatePredefinedSource() throws Exception { + public void testCreatePredefinedReader() throws Exception { CloudObject spec = CloudObject.forClassName("TextSource"); addString(spec, "filename", "/path/to/file.txt"); - com.google.api.services.dataflow.model.Source cloudSource = - new com.google.api.services.dataflow.model.Source(); + Source cloudSource = new Source(); cloudSource.setSpec(spec); cloudSource.setCodec(makeCloudEncoding("StringUtf8Coder")); - Source source = SourceFactory.create(PipelineOptionsFactory.create(), - cloudSource, - new BatchModeExecutionContext()); - Assert.assertThat(source, new IsInstanceOf(TextSource.class)); + Reader reader = ReaderFactory.create( + PipelineOptionsFactory.create(), cloudSource, new BatchModeExecutionContext()); + Assert.assertThat(reader, new IsInstanceOf(TextReader.class)); } @Test - public void testCreateUserDefinedSource() throws Exception { - CloudObject spec = CloudObject.forClass(TestSourceFactory.class); + public void testCreateUserDefinedReader() throws Exception { + CloudObject spec = CloudObject.forClass(TestReaderFactory.class); - com.google.api.services.dataflow.model.Source cloudSource = - new com.google.api.services.dataflow.model.Source(); + Source cloudSource = new Source(); cloudSource.setSpec(spec); cloudSource.setCodec(makeCloudEncoding("BigEndianIntegerCoder")); - Source source = SourceFactory.create(PipelineOptionsFactory.create(), - cloudSource, - new BatchModeExecutionContext()); - Assert.assertThat(source, new IsInstanceOf(TestSource.class)); + Reader reader = ReaderFactory.create( + PipelineOptionsFactory.create(), cloudSource, new BatchModeExecutionContext()); + Assert.assertThat(reader, new IsInstanceOf(TestReader.class)); } @Test - public void testCreateUnknownSource() throws Exception { + public void testCreateUnknownReader() throws Exception { CloudObject spec = CloudObject.forClassName("UnknownSource"); - com.google.api.services.dataflow.model.Source cloudSource = - new com.google.api.services.dataflow.model.Source(); + Source cloudSource = new Source(); cloudSource.setSpec(spec); cloudSource.setCodec(makeCloudEncoding("StringUtf8Coder")); try { - SourceFactory.create(PipelineOptionsFactory.create(), - cloudSource, - new BatchModeExecutionContext()); + ReaderFactory.create( + PipelineOptionsFactory.create(), cloudSource, new BatchModeExecutionContext()); Assert.fail("should have thrown an exception"); } catch (Exception exn) { - Assert.assertThat(exn.toString(), - CoreMatchers.containsString( - "unable to create a source")); + Assert.assertThat(exn.toString(), CoreMatchers.containsString("unable to create a source")); } } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleReaderFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleReaderFactoryTest.java new file mode 100644 index 0000000000000..ab1fa9145b3cd --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleReaderFactoryTest.java @@ -0,0 +1,185 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.client.util.Base64.encodeBase64String; +import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.api.services.dataflow.model.Source; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; + +import org.hamcrest.core.IsInstanceOf; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import javax.annotation.Nullable; + +/** + * Tests for UngroupedShuffleReaderFactory, GroupingShuffleReaderFactory, + * and PartitioningShuffleReaderFactory. + */ +@RunWith(JUnit4.class) +@SuppressWarnings({"rawtypes", "unchecked"}) +public class ShuffleReaderFactoryTest { + T runTestCreateShuffleReader(byte[] shuffleReaderConfig, + @Nullable String start, @Nullable String end, CloudObject encoding, + BatchModeExecutionContext context, Class shuffleReaderClass, String shuffleSourceAlias) + throws Exception { + CloudObject spec = CloudObject.forClassName(shuffleSourceAlias); + addString(spec, "shuffle_reader_config", encodeBase64String(shuffleReaderConfig)); + if (start != null) { + addString(spec, "start_shuffle_position", start); + } + if (end != null) { + addString(spec, "end_shuffle_position", end); + } + + Source cloudSource = new Source(); + cloudSource.setSpec(spec); + cloudSource.setCodec(encoding); + + Reader reader = ReaderFactory.create(PipelineOptionsFactory.create(), cloudSource, context); + Assert.assertThat(reader, new IsInstanceOf(shuffleReaderClass)); + T shuffleSource = (T) reader; + return shuffleSource; + } + + void runTestCreateUngroupedShuffleReader(byte[] shuffleReaderConfig, @Nullable String start, + @Nullable String end, CloudObject encoding, Coder coder) throws Exception { + UngroupedShuffleReader ungroupedShuffleReader = runTestCreateShuffleReader(shuffleReaderConfig, + start, end, encoding, new BatchModeExecutionContext(), UngroupedShuffleReader.class, + "UngroupedShuffleSource"); + Assert.assertArrayEquals(shuffleReaderConfig, ungroupedShuffleReader.shuffleReaderConfig); + Assert.assertEquals(start, ungroupedShuffleReader.startShufflePosition); + Assert.assertEquals(end, ungroupedShuffleReader.stopShufflePosition); + + Assert.assertEquals(coder, ungroupedShuffleReader.coder); + } + + void runTestCreateGroupingShuffleReader(byte[] shuffleReaderConfig, @Nullable String start, + @Nullable String end, CloudObject encoding, Coder keyCoder, Coder valueCoder) + throws Exception { + BatchModeExecutionContext context = new BatchModeExecutionContext(); + GroupingShuffleReader groupingShuffleReader = runTestCreateShuffleReader( + shuffleReaderConfig, start, end, encoding, context, GroupingShuffleReader.class, + "GroupingShuffleSource"); + Assert.assertArrayEquals(shuffleReaderConfig, groupingShuffleReader.shuffleReaderConfig); + Assert.assertEquals(start, groupingShuffleReader.startShufflePosition); + Assert.assertEquals(end, groupingShuffleReader.stopShufflePosition); + + Assert.assertEquals(keyCoder, groupingShuffleReader.keyCoder); + Assert.assertEquals(valueCoder, groupingShuffleReader.valueCoder); + Assert.assertEquals(context, groupingShuffleReader.executionContext); + } + + void runTestCreatePartitioningShuffleReader(byte[] shuffleReaderConfig, @Nullable String start, + @Nullable String end, CloudObject encoding, Coder keyCoder, Coder windowedValueCoder) + throws Exception { + PartitioningShuffleReader partitioningShuffleReader = + runTestCreateShuffleReader(shuffleReaderConfig, start, end, encoding, + new BatchModeExecutionContext(), PartitioningShuffleReader.class, + "PartitioningShuffleSource"); + Assert.assertArrayEquals(shuffleReaderConfig, partitioningShuffleReader.shuffleReaderConfig); + Assert.assertEquals(start, partitioningShuffleReader.startShufflePosition); + Assert.assertEquals(end, partitioningShuffleReader.stopShufflePosition); + + Assert.assertEquals(keyCoder, partitioningShuffleReader.keyCoder); + Assert.assertEquals(windowedValueCoder, partitioningShuffleReader.windowedValueCoder); + } + + @Test + public void testCreatePlainUngroupedShuffleReader() throws Exception { + runTestCreateUngroupedShuffleReader(new byte[] {(byte) 0xE1}, null, null, + makeCloudEncoding("StringUtf8Coder"), StringUtf8Coder.of()); + } + + @Test + public void testCreateRichUngroupedShuffleReader() throws Exception { + runTestCreateUngroupedShuffleReader(new byte[] {(byte) 0xE2}, "aaa", "zzz", + makeCloudEncoding("BigEndianIntegerCoder"), BigEndianIntegerCoder.of()); + } + + @Test + public void testCreatePlainGroupingShuffleReader() throws Exception { + runTestCreateGroupingShuffleReader( + new byte[] {(byte) 0xE1}, null, null, + makeCloudEncoding( + FullWindowedValueCoder.class.getName(), + makeCloudEncoding("KvCoder", makeCloudEncoding("BigEndianIntegerCoder"), + makeCloudEncoding("IterableCoder", makeCloudEncoding("StringUtf8Coder"))), + IntervalWindow.getCoder().asCloudObject()), + BigEndianIntegerCoder.of(), StringUtf8Coder.of()); + } + + @Test + public void testCreateRichGroupingShuffleReader() throws Exception { + runTestCreateGroupingShuffleReader( + new byte[] {(byte) 0xE2}, "aaa", "zzz", + makeCloudEncoding( + FullWindowedValueCoder.class.getName(), + makeCloudEncoding( + "KvCoder", makeCloudEncoding("BigEndianIntegerCoder"), + makeCloudEncoding( + "IterableCoder", + makeCloudEncoding("KvCoder", makeCloudEncoding("StringUtf8Coder"), + makeCloudEncoding("VoidCoder")))), + IntervalWindow.getCoder().asCloudObject()), + BigEndianIntegerCoder.of(), KvCoder.of(StringUtf8Coder.of(), VoidCoder.of())); + } + + @Test + public void testCreatePlainPartitioningShuffleReader() throws Exception { + runTestCreatePartitioningShuffleReader( + new byte[] {(byte) 0xE1}, null, null, + makeCloudEncoding( + FullWindowedValueCoder.class.getName(), + makeCloudEncoding("KvCoder", makeCloudEncoding("BigEndianIntegerCoder"), + makeCloudEncoding("StringUtf8Coder")), + IntervalWindow.getCoder().asCloudObject()), + BigEndianIntegerCoder.of(), + FullWindowedValueCoder.of(StringUtf8Coder.of(), IntervalWindow.getCoder())); + } + + @Test + public void testCreateRichPartitioningShuffleReader() throws Exception { + runTestCreatePartitioningShuffleReader( + new byte[] {(byte) 0xE2}, "aaa", "zzz", + makeCloudEncoding( + FullWindowedValueCoder.class.getName(), + makeCloudEncoding( + "KvCoder", makeCloudEncoding("BigEndianIntegerCoder"), + makeCloudEncoding("KvCoder", makeCloudEncoding("StringUtf8Coder"), + makeCloudEncoding("VoidCoder"))), + IntervalWindow.getCoder().asCloudObject()), + BigEndianIntegerCoder.of(), + FullWindowedValueCoder.of( + KvCoder.of(StringUtf8Coder.of(), VoidCoder.of()), IntervalWindow.getCoder())); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSourceFactoryTest.java deleted file mode 100644 index 1803b06d7fd1e..0000000000000 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSourceFactoryTest.java +++ /dev/null @@ -1,231 +0,0 @@ -/* - * Copyright (C) 2014 Google Inc. - * - * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; - -import static com.google.api.client.util.Base64.encodeBase64String; -import static com.google.cloud.dataflow.sdk.util.CoderUtils.makeCloudEncoding; -import static com.google.cloud.dataflow.sdk.util.Structs.addString; - -import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; -import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.coders.KvCoder; -import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; -import com.google.cloud.dataflow.sdk.coders.VoidCoder; -import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; -import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; -import com.google.cloud.dataflow.sdk.util.CloudObject; -import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; - -import org.hamcrest.core.IsInstanceOf; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -import javax.annotation.Nullable; - -/** - * Tests for UngroupedShuffleSourceFactory, GroupingShuffleSourceFactory, - * and PartitioningShuffleSourceFactory. - */ -@RunWith(JUnit4.class) -@SuppressWarnings({"rawtypes", "unchecked"}) -public class ShuffleSourceFactoryTest { - - T runTestCreateShuffleSource(byte[] shuffleReaderConfig, - @Nullable String start, - @Nullable String end, - CloudObject encoding, - BatchModeExecutionContext context, - Class shuffleSourceClass) - throws Exception { - CloudObject spec = CloudObject.forClassName(shuffleSourceClass.getSimpleName()); - addString(spec, "shuffle_reader_config", encodeBase64String(shuffleReaderConfig)); - if (start != null) { - addString(spec, "start_shuffle_position", start); - } - if (end != null) { - addString(spec, "end_shuffle_position", end); - } - - com.google.api.services.dataflow.model.Source cloudSource = - new com.google.api.services.dataflow.model.Source(); - cloudSource.setSpec(spec); - cloudSource.setCodec(encoding); - - Source source = SourceFactory.create( - PipelineOptionsFactory.create(), cloudSource, context); - Assert.assertThat(source, new IsInstanceOf(shuffleSourceClass)); - T shuffleSource = (T) source; - return shuffleSource; - } - - void runTestCreateUngroupedShuffleSource(byte[] shuffleReaderConfig, - @Nullable String start, - @Nullable String end, - CloudObject encoding, - Coder coder) throws Exception { - UngroupedShuffleSource shuffleSource = - runTestCreateShuffleSource(shuffleReaderConfig, - start, - end, - encoding, - new BatchModeExecutionContext(), - UngroupedShuffleSource.class); - Assert.assertArrayEquals(shuffleReaderConfig, - shuffleSource.shuffleReaderConfig); - Assert.assertEquals(start, shuffleSource.startShufflePosition); - Assert.assertEquals(end, shuffleSource.stopShufflePosition); - - Assert.assertEquals(coder, shuffleSource.coder); - } - - void runTestCreateGroupingShuffleSource(byte[] shuffleReaderConfig, - @Nullable String start, - @Nullable String end, - CloudObject encoding, - Coder keyCoder, - Coder valueCoder) throws Exception { - BatchModeExecutionContext context = new BatchModeExecutionContext(); - GroupingShuffleSource shuffleSource = - runTestCreateShuffleSource(shuffleReaderConfig, - start, - end, - encoding, - context, - GroupingShuffleSource.class); - Assert.assertArrayEquals(shuffleReaderConfig, - shuffleSource.shuffleReaderConfig); - Assert.assertEquals(start, shuffleSource.startShufflePosition); - Assert.assertEquals(end, shuffleSource.stopShufflePosition); - - Assert.assertEquals(keyCoder, shuffleSource.keyCoder); - Assert.assertEquals(valueCoder, shuffleSource.valueCoder); - Assert.assertEquals(context, shuffleSource.executionContext); - } - - void runTestCreatePartitioningShuffleSource(byte[] shuffleReaderConfig, - @Nullable String start, - @Nullable String end, - CloudObject encoding, - Coder keyCoder, - Coder windowedValueCoder) throws Exception { - PartitioningShuffleSource shuffleSource = - runTestCreateShuffleSource(shuffleReaderConfig, - start, - end, - encoding, - new BatchModeExecutionContext(), - PartitioningShuffleSource.class); - Assert.assertArrayEquals(shuffleReaderConfig, - shuffleSource.shuffleReaderConfig); - Assert.assertEquals(start, shuffleSource.startShufflePosition); - Assert.assertEquals(end, shuffleSource.stopShufflePosition); - - Assert.assertEquals(keyCoder, shuffleSource.keyCoder); - Assert.assertEquals(windowedValueCoder, shuffleSource.windowedValueCoder); - } - - @Test - public void testCreatePlainUngroupedShuffleSource() throws Exception { - runTestCreateUngroupedShuffleSource( - new byte[]{(byte) 0xE1}, null, null, - makeCloudEncoding("StringUtf8Coder"), - StringUtf8Coder.of()); - } - - @Test - public void testCreateRichUngroupedShuffleSource() throws Exception { - runTestCreateUngroupedShuffleSource( - new byte[]{(byte) 0xE2}, "aaa", "zzz", - makeCloudEncoding("BigEndianIntegerCoder"), - BigEndianIntegerCoder.of()); - } - - @Test - public void testCreatePlainGroupingShuffleSource() throws Exception { - runTestCreateGroupingShuffleSource( - new byte[]{(byte) 0xE1}, null, null, - makeCloudEncoding( - FullWindowedValueCoder.class.getName(), - makeCloudEncoding( - "KvCoder", - makeCloudEncoding("BigEndianIntegerCoder"), - makeCloudEncoding( - "IterableCoder", - makeCloudEncoding("StringUtf8Coder"))), - IntervalWindow.getCoder().asCloudObject()), - BigEndianIntegerCoder.of(), - StringUtf8Coder.of()); - } - - @Test - public void testCreateRichGroupingShuffleSource() throws Exception { - runTestCreateGroupingShuffleSource( - new byte[]{(byte) 0xE2}, "aaa", "zzz", - makeCloudEncoding( - FullWindowedValueCoder.class.getName(), - makeCloudEncoding( - "KvCoder", - makeCloudEncoding("BigEndianIntegerCoder"), - makeCloudEncoding( - "IterableCoder", - makeCloudEncoding( - "KvCoder", - makeCloudEncoding("StringUtf8Coder"), - makeCloudEncoding("VoidCoder")))), - IntervalWindow.getCoder().asCloudObject()), - BigEndianIntegerCoder.of(), - KvCoder.of(StringUtf8Coder.of(), VoidCoder.of())); - } - - @Test - public void testCreatePlainPartitioningShuffleSource() throws Exception { - runTestCreatePartitioningShuffleSource( - new byte[]{(byte) 0xE1}, null, null, - makeCloudEncoding( - FullWindowedValueCoder.class.getName(), - makeCloudEncoding( - "KvCoder", - makeCloudEncoding("BigEndianIntegerCoder"), - makeCloudEncoding("StringUtf8Coder")), - IntervalWindow.getCoder().asCloudObject()), - BigEndianIntegerCoder.of(), - FullWindowedValueCoder.of(StringUtf8Coder.of(), IntervalWindow.getCoder())); - } - - @Test - public void testCreateRichPartitioningShuffleSource() throws Exception { - runTestCreatePartitioningShuffleSource( - new byte[]{(byte) 0xE2}, "aaa", "zzz", - makeCloudEncoding( - FullWindowedValueCoder.class.getName(), - makeCloudEncoding( - "KvCoder", - makeCloudEncoding("BigEndianIntegerCoder"), - makeCloudEncoding( - "KvCoder", - makeCloudEncoding("StringUtf8Coder"), - makeCloudEncoding("VoidCoder"))), - IntervalWindow.getCoder().asCloudObject()), - BigEndianIntegerCoder.of(), - FullWindowedValueCoder.of(KvCoder.of(StringUtf8Coder.of(), VoidCoder.of()), - IntervalWindow.getCoder())); - } -} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtilsTest.java index ea879335ec02a..343e172715ad3 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtilsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SideInputUtilsTest.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertThat; import com.google.api.services.dataflow.model.SideInputInfo; +import com.google.api.services.dataflow.model.Source; import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; @@ -41,28 +42,23 @@ */ @RunWith(JUnit4.class) public class SideInputUtilsTest { - SideInputInfo createSingletonSideInputInfo( - com.google.api.services.dataflow.model.Source sideInputSource) { + SideInputInfo createSingletonSideInputInfo(Source sideInputSource) { SideInputInfo sideInputInfo = new SideInputInfo(); sideInputInfo.setSources(Arrays.asList(sideInputSource)); sideInputInfo.setKind(CloudObject.forClassName("singleton")); return sideInputInfo; } - SideInputInfo createCollectionSideInputInfo( - com.google.api.services.dataflow.model.Source... sideInputSources) { + SideInputInfo createCollectionSideInputInfo(Source... sideInputSources) { SideInputInfo sideInputInfo = new SideInputInfo(); sideInputInfo.setSources(Arrays.asList(sideInputSources)); sideInputInfo.setKind(CloudObject.forClassName("collection")); return sideInputInfo; } - com.google.api.services.dataflow.model.Source createSideInputSource(Integer... ints) - throws Exception { - return InMemorySourceFactoryTest.createInMemoryCloudSource( - Arrays.asList(ints), - null, null, - BigEndianIntegerCoder.of()); + Source createSideInputSource(Integer... ints) throws Exception { + return InMemoryReaderFactoryTest.createInMemoryCloudSource( + Arrays.asList(ints), null, null, BigEndianIntegerCoder.of()); } void assertThatContains(Object actual, Object... expected) { @@ -77,52 +73,41 @@ void assertThatContains(Object actual, Object... expected) { @Test public void testReadSingletonSideInput() throws Exception { - SideInputInfo sideInputInfo = - createSingletonSideInputInfo(createSideInputSource(42)); + SideInputInfo sideInputInfo = createSingletonSideInputInfo(createSideInputSource(42)); - assertEquals(42, - SideInputUtils.readSideInput(PipelineOptionsFactory.create(), - sideInputInfo, - new BatchModeExecutionContext())); + assertEquals( + 42, + SideInputUtils.readSideInput( + PipelineOptionsFactory.create(), sideInputInfo, new BatchModeExecutionContext())); } @Test public void testReadEmptyCollectionSideInput() throws Exception { - SideInputInfo sideInputInfo = - createCollectionSideInputInfo(createSideInputSource()); + SideInputInfo sideInputInfo = createCollectionSideInputInfo(createSideInputSource()); - assertThatContains( - SideInputUtils.readSideInput(PipelineOptionsFactory.create(), - sideInputInfo, - new BatchModeExecutionContext())); + assertThatContains(SideInputUtils.readSideInput( + PipelineOptionsFactory.create(), sideInputInfo, new BatchModeExecutionContext())); } @Test public void testReadCollectionSideInput() throws Exception { - SideInputInfo sideInputInfo = - createCollectionSideInputInfo(createSideInputSource(3, 4, 5, 6)); + SideInputInfo sideInputInfo = createCollectionSideInputInfo(createSideInputSource(3, 4, 5, 6)); assertThatContains( - SideInputUtils.readSideInput(PipelineOptionsFactory.create(), - sideInputInfo, - new BatchModeExecutionContext()), + SideInputUtils.readSideInput( + PipelineOptionsFactory.create(), sideInputInfo, new BatchModeExecutionContext()), 3, 4, 5, 6); } @Test public void testReadCollectionShardedSideInput() throws Exception { SideInputInfo sideInputInfo = - createCollectionSideInputInfo( - createSideInputSource(3), - createSideInputSource(), - createSideInputSource(4, 5), - createSideInputSource(6), - createSideInputSource()); + createCollectionSideInputInfo(createSideInputSource(3), createSideInputSource(), + createSideInputSource(4, 5), createSideInputSource(6), createSideInputSource()); assertThatContains( - SideInputUtils.readSideInput(PipelineOptionsFactory.create(), - sideInputInfo, - new BatchModeExecutionContext()), + SideInputUtils.readSideInput( + PipelineOptionsFactory.create(), sideInputInfo, new BatchModeExecutionContext()), 3, 4, 5, 6); } @@ -131,15 +116,13 @@ public void testReadSingletonSideInputValue() throws Exception { CloudObject sideInputKind = CloudObject.forClassName("singleton"); Object elem = "hi"; List elems = Arrays.asList(elem); - assertEquals(elem, - SideInputUtils.readSideInputValue(sideInputKind, elems)); + assertEquals(elem, SideInputUtils.readSideInputValue(sideInputKind, elems)); } @Test public void testReadCollectionSideInputValue() throws Exception { CloudObject sideInputKind = CloudObject.forClassName("collection"); List elems = Arrays.asList("hi", "there", "bob"); - assertEquals(elems, - SideInputUtils.readSideInputValue(sideInputKind, elems)); + assertEquals(elems, SideInputUtils.readSideInputValue(sideInputKind, elems)); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactoryTest.java similarity index 56% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactoryTest.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactoryTest.java index 05d8a721fefb3..160eace961a72 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactoryTest.java @@ -21,13 +21,14 @@ import static com.google.cloud.dataflow.sdk.util.Structs.addLong; import static com.google.cloud.dataflow.sdk.util.Structs.addString; +import com.google.api.services.dataflow.model.Source; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.TextualIntegerCoder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CloudObject; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.hamcrest.core.IsInstanceOf; import org.junit.Assert; @@ -38,16 +39,12 @@ import javax.annotation.Nullable; /** - * Tests for TextSourceFactory. + * Tests for TextReaderFactory. */ @RunWith(JUnit4.class) -public class TextSourceFactoryTest { - void runTestCreateTextSource(String filename, - @Nullable Boolean stripTrailingNewlines, - @Nullable Long start, - @Nullable Long end, - CloudObject encoding, - Coder coder) +public class TextReaderFactoryTest { + void runTestCreateTextReader(String filename, @Nullable Boolean stripTrailingNewlines, + @Nullable Long start, @Nullable Long end, CloudObject encoding, Coder coder) throws Exception { CloudObject spec = CloudObject.forClassName("TextSource"); addString(spec, "filename", filename); @@ -61,38 +58,32 @@ void runTestCreateTextSource(String filename, addLong(spec, "end_offset", end); } - com.google.api.services.dataflow.model.Source cloudSource = - new com.google.api.services.dataflow.model.Source(); + Source cloudSource = new Source(); cloudSource.setSpec(spec); cloudSource.setCodec(encoding); - Source source = SourceFactory.create(PipelineOptionsFactory.create(), - cloudSource, - new BatchModeExecutionContext()); - Assert.assertThat(source, new IsInstanceOf(TextSource.class)); - TextSource textSource = (TextSource) source; - Assert.assertEquals(filename, textSource.filename); + Reader reader = ReaderFactory.create( + PipelineOptionsFactory.create(), cloudSource, new BatchModeExecutionContext()); + Assert.assertThat(reader, new IsInstanceOf(TextReader.class)); + TextReader textReader = (TextReader) reader; + Assert.assertEquals(filename, textReader.filename); Assert.assertEquals( stripTrailingNewlines == null ? true : stripTrailingNewlines, - textSource.stripTrailingNewlines); - Assert.assertEquals(start, textSource.startPosition); - Assert.assertEquals(end, textSource.endPosition); - Assert.assertEquals(coder, textSource.coder); + textReader.stripTrailingNewlines); + Assert.assertEquals(start, textReader.startPosition); + Assert.assertEquals(end, textReader.endPosition); + Assert.assertEquals(coder, textReader.coder); } @Test - public void testCreatePlainTextSource() throws Exception { - runTestCreateTextSource( - "/path/to/file.txt", null, null, null, - makeCloudEncoding("StringUtf8Coder"), - StringUtf8Coder.of()); + public void testCreatePlainTextReader() throws Exception { + runTestCreateTextReader("/path/to/file.txt", null, null, null, + makeCloudEncoding("StringUtf8Coder"), StringUtf8Coder.of()); } @Test - public void testCreateRichTextSource() throws Exception { - runTestCreateTextSource( - "gs://bucket/path/to/file2.txt", false, 200L, 500L, - makeCloudEncoding("TextualIntegerCoder"), - TextualIntegerCoder.of()); + public void testCreateRichTextReader() throws Exception { + runTestCreateTextReader("gs://bucket/path/to/file2.txt", false, 200L, 500L, + makeCloudEncoding("TextualIntegerCoder"), TextualIntegerCoder.of()); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java similarity index 57% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceTest.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java index 87f8379479ad8..4ebd671361eda 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java @@ -16,7 +16,7 @@ package com.google.cloud.dataflow.sdk.runners.worker; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; import static org.hamcrest.Matchers.greaterThan; @@ -27,7 +27,7 @@ import com.google.cloud.dataflow.sdk.coders.TextualIntegerCoder; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.junit.Assert; import org.junit.Rule; @@ -46,13 +46,11 @@ import java.util.List; /** - * Tests for TextSource. + * Tests for TextReader. */ @RunWith(JUnit4.class) -public class TextSourceTest { - private static final String[] fileContent = {"First line\n", - "Second line\r\n", - "Third line"}; +public class TextReaderTest { + private static final String[] fileContent = {"First line\n", "Second line\r\n", "Third line"}; private static final long TOTAL_BYTES_COUNT; static { @@ -63,7 +61,8 @@ public class TextSourceTest { TOTAL_BYTES_COUNT = sumLen; } - @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); private File initTestFile() throws IOException { File tmpFile = tmpFolder.newFile(); @@ -78,9 +77,9 @@ private File initTestFile() throws IOException { @Test public void testReadEmptyFile() throws Exception { - TextSource textSource = new TextSource<>( - "/dev/null", true, null, null, StringUtf8Coder.of()); - try (Source.SourceIterator iterator = textSource.iterator()) { + TextReader textReader = + new TextReader<>("/dev/null", true, null, null, StringUtf8Coder.of()); + try (Reader.ReaderIterator iterator = textReader.iterator()) { Assert.assertFalse(iterator.hasNext()); } } @@ -96,13 +95,13 @@ public void testStrippedNewlines() throws Exception { public void testStrippedNewlinesAtEndOfReadBuffer() throws Exception { boolean stripNewLines = true; StringBuilder payload = new StringBuilder(); - for (int i = 0; i < TextSource.BUF_SIZE - 2; ++i) { + for (int i = 0; i < TextReader.BUF_SIZE - 2; ++i) { payload.append('a'); } String[] lines = {payload.toString(), payload.toString()}; - testStringPayload(lines , "\r", stripNewLines); - testStringPayload(lines , "\r\n", stripNewLines); - testStringPayload(lines , "\n", stripNewLines); + testStringPayload(lines, "\r", stripNewLines); + testStringPayload(lines, "\r\n", stripNewLines); + testStringPayload(lines, "\n", stripNewLines); } @Test @@ -116,13 +115,13 @@ public void testUnstrippedNewlines() throws Exception { public void testUnstrippedNewlinesAtEndOfReadBuffer() throws Exception { boolean stripNewLines = false; StringBuilder payload = new StringBuilder(); - for (int i = 0; i < TextSource.BUF_SIZE - 2; ++i) { + for (int i = 0; i < TextReader.BUF_SIZE - 2; ++i) { payload.append('a'); } String[] lines = {payload.toString(), payload.toString()}; - testStringPayload(lines , "\r", stripNewLines); - testStringPayload(lines , "\r\n", stripNewLines); - testStringPayload(lines , "\n", stripNewLines); + testStringPayload(lines, "\r", stripNewLines); + testStringPayload(lines, "\r\n", stripNewLines); + testStringPayload(lines, "\n", stripNewLines); } @Test @@ -130,12 +129,12 @@ public void testStartPosition() throws Exception { File tmpFile = initTestFile(); { - TextSource textSource = new TextSource<>( - tmpFile.getPath(), false, 11L, null, StringUtf8Coder.of()); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(textSource); + TextReader textReader = + new TextReader<>(tmpFile.getPath(), false, 11L, null, StringUtf8Coder.of()); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(textReader); - try (Source.SourceIterator iterator = textSource.iterator()) { + try (Reader.ReaderIterator iterator = textReader.iterator()) { Assert.assertEquals("Second line\r\n", iterator.next()); Assert.assertEquals("Third line", iterator.next()); Assert.assertFalse(iterator.hasNext()); @@ -146,12 +145,12 @@ public void testStartPosition() throws Exception { } { - TextSource textSource = new TextSource<>( - tmpFile.getPath(), false, 20L, null, StringUtf8Coder.of()); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(textSource); + TextReader textReader = + new TextReader<>(tmpFile.getPath(), false, 20L, null, StringUtf8Coder.of()); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(textReader); - try (Source.SourceIterator iterator = textSource.iterator()) { + try (Reader.ReaderIterator iterator = textReader.iterator()) { Assert.assertEquals("Third line", iterator.next()); Assert.assertFalse(iterator.hasNext()); // The first '5' in the array represents the reading of a portion of the second @@ -161,12 +160,12 @@ public void testStartPosition() throws Exception { } { - TextSource textSource = new TextSource<>( - tmpFile.getPath(), true, 0L, 20L, StringUtf8Coder.of()); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(textSource); + TextReader textReader = + new TextReader<>(tmpFile.getPath(), true, 0L, 20L, StringUtf8Coder.of()); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(textReader); - try (Source.SourceIterator iterator = textSource.iterator()) { + try (Reader.ReaderIterator iterator = textReader.iterator()) { Assert.assertEquals("First line", iterator.next()); Assert.assertEquals("Second line", iterator.next()); Assert.assertFalse(iterator.hasNext()); @@ -175,12 +174,12 @@ public void testStartPosition() throws Exception { } { - TextSource textSource = new TextSource<>( - tmpFile.getPath(), true, 1L, 20L, StringUtf8Coder.of()); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(textSource); + TextReader textReader = + new TextReader<>(tmpFile.getPath(), true, 1L, 20L, StringUtf8Coder.of()); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(textReader); - try (Source.SourceIterator iterator = textSource.iterator()) { + try (Reader.ReaderIterator iterator = textReader.iterator()) { Assert.assertEquals("Second line", iterator.next()); Assert.assertFalse(iterator.hasNext()); // The first '11' in the array represents the reading of the entire first @@ -196,22 +195,21 @@ public void testUtf8Handling() throws Exception { FileOutputStream output = new FileOutputStream(tmpFile); // first line: €\n // second line: ¢\n - output.write(new byte[]{(byte) 0xE2, (byte) 0x82, (byte) 0xAC, '\n', - (byte) 0xC2, (byte) 0xA2, '\n'}); + output.write( + new byte[] {(byte) 0xE2, (byte) 0x82, (byte) 0xAC, '\n', (byte) 0xC2, (byte) 0xA2, '\n'}); output.close(); { // 3L is after the first line if counting codepoints, but within // the first line if counting chars. So correct behavior is to return // just one line, since offsets are in chars, not codepoints. - TextSource textSource = new TextSource<>( - tmpFile.getPath(), true, 0L, 3L, StringUtf8Coder.of()); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(textSource); - - try (Source.SourceIterator iterator = textSource.iterator()) { - Assert.assertArrayEquals("€".getBytes("UTF-8"), - iterator.next().getBytes("UTF-8")); + TextReader textReader = + new TextReader<>(tmpFile.getPath(), true, 0L, 3L, StringUtf8Coder.of()); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(textReader); + + try (Reader.ReaderIterator iterator = textReader.iterator()) { + Assert.assertArrayEquals("€".getBytes("UTF-8"), iterator.next().getBytes("UTF-8")); Assert.assertFalse(iterator.hasNext()); Assert.assertEquals(Arrays.asList(4), observer.getActualSizes()); } @@ -220,14 +218,13 @@ public void testUtf8Handling() throws Exception { { // Starting location is mid-way into a codepoint. // Ensures we don't fail when skipping over an incomplete codepoint. - TextSource textSource = new TextSource<>( - tmpFile.getPath(), true, 2L, null, StringUtf8Coder.of()); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(textSource); - - try (Source.SourceIterator iterator = textSource.iterator()) { - Assert.assertArrayEquals("¢".getBytes("UTF-8"), - iterator.next().getBytes("UTF-8")); + TextReader textReader = + new TextReader<>(tmpFile.getPath(), true, 2L, null, StringUtf8Coder.of()); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(textReader); + + try (Reader.ReaderIterator iterator = textReader.iterator()) { + Assert.assertArrayEquals("¢".getBytes("UTF-8"), iterator.next().getBytes("UTF-8")); Assert.assertFalse(iterator.hasNext()); // The first '3' in the array represents the reading of a portion of the first // line, which had to be read to find the beginning of the second line. @@ -236,20 +233,10 @@ public void testUtf8Handling() throws Exception { } } - private void testNewlineHandling(String separator, boolean stripNewlines) - throws Exception { + private void testNewlineHandling(String separator, boolean stripNewlines) throws Exception { File tmpFile = tmpFolder.newFile(); - PrintStream writer = - new PrintStream( - new FileOutputStream(tmpFile)); - List expected = Arrays.asList( - "", - " hi there ", - "bob", - "", - " ", - "--zowie!--", - ""); + PrintStream writer = new PrintStream(new FileOutputStream(tmpFile)); + List expected = Arrays.asList("", " hi there ", "bob", "", " ", "--zowie!--", ""); List expectedSizes = new ArrayList<>(); for (String line : expected) { writer.print(line); @@ -258,13 +245,13 @@ private void testNewlineHandling(String separator, boolean stripNewlines) } writer.close(); - TextSource textSource = new TextSource<>( - tmpFile.getPath(), stripNewlines, null, null, StringUtf8Coder.of()); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(textSource); + TextReader textReader = + new TextReader<>(tmpFile.getPath(), stripNewlines, null, null, StringUtf8Coder.of()); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(textReader); List actual = new ArrayList<>(); - try (Source.SourceIterator iterator = textSource.iterator()) { + try (Reader.ReaderIterator iterator = textReader.iterator()) { while (iterator.hasNext()) { actual.add(iterator.next()); } @@ -283,14 +270,11 @@ private void testNewlineHandling(String separator, boolean stripNewlines) Assert.assertEquals(expectedSizes, observer.getActualSizes()); } - private void testStringPayload( - String[] lines, String separator, boolean stripNewlines) + private void testStringPayload(String[] lines, String separator, boolean stripNewlines) throws Exception { File tmpFile = tmpFolder.newFile(); List expected = new ArrayList<>(); - PrintStream writer = - new PrintStream( - new FileOutputStream(tmpFile)); + PrintStream writer = new PrintStream(new FileOutputStream(tmpFile)); for (String line : lines) { writer.print(line); writer.print(separator); @@ -298,31 +282,29 @@ private void testStringPayload( } writer.close(); - TextSource textSource = new TextSource<>( - tmpFile.getPath(), stripNewlines, null, null, StringUtf8Coder.of()); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(textSource); + TextReader textReader = + new TextReader<>(tmpFile.getPath(), stripNewlines, null, null, StringUtf8Coder.of()); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(textReader); List actual = new ArrayList<>(); - try (Source.SourceIterator iterator = textSource.iterator()) { - while (iterator.hasNext()) { - actual.add(iterator.next()); - } + try (Reader.ReaderIterator iterator = textReader.iterator()) { + while (iterator.hasNext()) { + actual.add(iterator.next()); } + } Assert.assertEquals(expected, actual); } @Test - public void testCloneIteratorWithEndPositionAndFinalBytesInBuffer() - throws Exception { + public void testCloneIteratorWithEndPositionAndFinalBytesInBuffer() throws Exception { String line = "a\n"; boolean stripNewlines = false; File tmpFile = tmpFolder.newFile(); List expected = new ArrayList<>(); PrintStream writer = new PrintStream(new FileOutputStream(tmpFile)); // Write 5x the size of the buffer and 10 extra trailing bytes - for (long bytesWritten = 0; - bytesWritten < TextSource.BUF_SIZE * 3 + 10; ) { + for (long bytesWritten = 0; bytesWritten < TextReader.BUF_SIZE * 3 + 10;) { writer.print(line); expected.add(line); bytesWritten += line.length(); @@ -330,12 +312,11 @@ public void testCloneIteratorWithEndPositionAndFinalBytesInBuffer() writer.close(); Long fileSize = tmpFile.length(); - TextSource textSource = new TextSource<>( - tmpFile.getPath(), stripNewlines, - null, fileSize, StringUtf8Coder.of()); + TextReader textReader = + new TextReader<>(tmpFile.getPath(), stripNewlines, null, fileSize, StringUtf8Coder.of()); List actual = new ArrayList<>(); - Source.SourceIterator iterator = textSource.iterator(); + Reader.ReaderIterator iterator = textReader.iterator(); while (iterator.hasNext()) { actual.add(iterator.next()); iterator = iterator.copy(); @@ -346,27 +327,24 @@ public void testCloneIteratorWithEndPositionAndFinalBytesInBuffer() @Test public void testNonStringCoders() throws Exception { File tmpFile = tmpFolder.newFile(); - PrintStream writer = - new PrintStream( - new FileOutputStream(tmpFile)); + PrintStream writer = new PrintStream(new FileOutputStream(tmpFile)); List expected = TestUtils.INTS; List expectedSizes = new ArrayList<>(); for (Integer elem : expected) { - byte[] encodedElem = - CoderUtils.encodeToByteArray(TextualIntegerCoder.of(), elem); + byte[] encodedElem = CoderUtils.encodeToByteArray(TextualIntegerCoder.of(), elem); writer.print(elem); writer.print("\n"); expectedSizes.add(1 + encodedElem.length); } writer.close(); - TextSource textSource = new TextSource<>( - tmpFile.getPath(), true, null, null, TextualIntegerCoder.of()); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(textSource); + TextReader textReader = + new TextReader<>(tmpFile.getPath(), true, null, null, TextualIntegerCoder.of()); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(textReader); List actual = new ArrayList<>(); - try (Source.SourceIterator iterator = textSource.iterator()) { + try (Reader.ReaderIterator iterator = textReader.iterator()) { while (iterator.hasNext()) { actual.add(iterator.next()); } @@ -379,34 +357,29 @@ public void testNonStringCoders() throws Exception { @Test public void testGetApproximatePosition() throws Exception { File tmpFile = initTestFile(); - TextSource textSource = new TextSource<>( - tmpFile.getPath(), false, 0L, null, StringUtf8Coder.of()); - - try (Source.SourceIterator iterator = textSource.iterator()) { - ApproximateProgress progress = - sourceProgressToCloudProgress(iterator.getProgress()); - Assert.assertEquals(0L, - progress.getPosition().getByteOffset().longValue()); + TextReader textReader = + new TextReader<>(tmpFile.getPath(), false, 0L, null, StringUtf8Coder.of()); + + try (Reader.ReaderIterator iterator = textReader.iterator()) { + ApproximateProgress progress = sourceProgressToCloudProgress(iterator.getProgress()); + Assert.assertEquals(0L, progress.getPosition().getByteOffset().longValue()); iterator.next(); progress = sourceProgressToCloudProgress(iterator.getProgress()); - Assert.assertEquals(11L, - progress.getPosition().getByteOffset().longValue()); + Assert.assertEquals(11L, progress.getPosition().getByteOffset().longValue()); iterator.next(); progress = sourceProgressToCloudProgress(iterator.getProgress()); - Assert.assertEquals(24L, - progress.getPosition().getByteOffset().longValue()); + Assert.assertEquals(24L, progress.getPosition().getByteOffset().longValue()); iterator.next(); progress = sourceProgressToCloudProgress(iterator.getProgress()); - Assert.assertEquals(34L, - progress.getPosition().getByteOffset().longValue()); + Assert.assertEquals(34L, progress.getPosition().getByteOffset().longValue()); Assert.assertFalse(iterator.hasNext()); } } @Test public void testUpdateStopPosition() throws Exception { - final long end = 10L; // in the first line - final long stop = 14L; // in the middle of the second line + final long end = 10L; // in the first line + final long stop = 14L; // in the middle of the second line File tmpFile = initTestFile(); com.google.api.services.dataflow.model.Position proposedStopPosition = @@ -414,91 +387,88 @@ public void testUpdateStopPosition() throws Exception { // Illegal proposed stop position, no update. { - TextSource textSource = new TextSource<>( - tmpFile.getPath(), false, null, null, - StringUtf8Coder.of()); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(textSource); - - try (TextSource.TextFileIterator iterator = - (TextSource.TextFileIterator) textSource.iterator()) { - Assert.assertNull(iterator.updateStopPosition( - cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); - } + TextReader textReader = + new TextReader<>(tmpFile.getPath(), false, null, null, StringUtf8Coder.of()); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(textReader); + + try (TextReader.TextFileIterator iterator = + (TextReader.TextFileIterator) textReader.iterator()) { + Assert.assertNull(iterator.updateStopPosition( + cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); + } } proposedStopPosition.setByteOffset(stop); // Successful update. { - TextSource textSource = new TextSource<>( - tmpFile.getPath(), false, null, null, - StringUtf8Coder.of()); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(textSource); - - try (TextSource.TextFileIterator iterator = - (TextSource.TextFileIterator) textSource.iterator()) { + TextReader textReader = + new TextReader<>(tmpFile.getPath(), false, null, null, StringUtf8Coder.of()); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(textReader); + + try (TextReader.TextFileIterator iterator = + (TextReader.TextFileIterator) textReader.iterator()) { Assert.assertNull(iterator.getEndOffset()); Assert.assertEquals( stop, sourcePositionToCloudPosition( iterator.updateStopPosition( - cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))) - .getByteOffset().longValue()); + cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))) + .getByteOffset() + .longValue()); Assert.assertEquals(stop, iterator.getEndOffset().longValue()); Assert.assertEquals(fileContent[0], iterator.next()); Assert.assertEquals(fileContent[1], iterator.next()); Assert.assertFalse(iterator.hasNext()); - Assert.assertEquals(Arrays.asList(fileContent[0].length(), - fileContent[1].length()), - observer.getActualSizes()); + Assert.assertEquals( + Arrays.asList(fileContent[0].length(), fileContent[1].length()), + observer.getActualSizes()); } } // Proposed stop position is before the current position, no update. { - TextSource textSource = new TextSource<>( - tmpFile.getPath(), false, null, null, - StringUtf8Coder.of()); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(textSource); - - try (TextSource.TextFileIterator iterator = - (TextSource.TextFileIterator) textSource.iterator()) { + TextReader textReader = + new TextReader<>(tmpFile.getPath(), false, null, null, StringUtf8Coder.of()); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(textReader); + + try (TextReader.TextFileIterator iterator = + (TextReader.TextFileIterator) textReader.iterator()) { Assert.assertEquals(fileContent[0], iterator.next()); Assert.assertEquals(fileContent[1], iterator.next()); - Assert.assertThat(sourceProgressToCloudProgress(iterator.getProgress()) - .getPosition().getByteOffset(), + Assert.assertThat( + sourceProgressToCloudProgress(iterator.getProgress()).getPosition().getByteOffset(), greaterThan(stop)); Assert.assertNull(iterator.updateStopPosition( - cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); Assert.assertNull(iterator.getEndOffset()); Assert.assertTrue(iterator.hasNext()); Assert.assertEquals(fileContent[2], iterator.next()); - Assert.assertEquals(Arrays.asList(fileContent[0].length(), - fileContent[1].length(), - fileContent[2].length()), - observer.getActualSizes()); + Assert.assertEquals( + Arrays.asList( + fileContent[0].length(), fileContent[1].length(), fileContent[2].length()), + observer.getActualSizes()); } } // Proposed stop position is after the current stop (end) position, no update. { - TextSource textSource = new TextSource<>( - tmpFile.getPath(), false, null, end, StringUtf8Coder.of()); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(textSource); + TextReader textReader = + new TextReader<>(tmpFile.getPath(), false, null, end, StringUtf8Coder.of()); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(textReader); - try (TextSource.TextFileIterator iterator = - (TextSource.TextFileIterator) textSource.iterator()) { + try (TextReader.TextFileIterator iterator = + (TextReader.TextFileIterator) textReader.iterator()) { Assert.assertEquals(fileContent[0], iterator.next()); Assert.assertNull(iterator.updateStopPosition( - cloudProgressToSourceProgress(createApproximateProgress(proposedStopPosition)))); + cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); Assert.assertEquals(end, iterator.getEndOffset().longValue()); Assert.assertFalse(iterator.hasNext()); - Assert.assertEquals(Arrays.asList(fileContent[0].length()), - observer.getActualSizes()); + Assert.assertEquals(Arrays.asList(fileContent[0].length()), observer.getActualSizes()); } } } @@ -513,8 +483,7 @@ public void testUpdateStopPositionExhaustive() throws Exception { for (long start = 0; start < TOTAL_BYTES_COUNT - 1; start++) { for (long end = start + 1; end < TOTAL_BYTES_COUNT; end++) { for (long stop = start; stop <= end; stop++) { - stopPositionTestInternal(start, end, - stop, tmpFile); + stopPositionTestInternal(start, end, stop, tmpFile); } } } @@ -525,21 +494,18 @@ public void testUpdateStopPositionExhaustive() throws Exception { } } - private void stopPositionTestInternal(Long startOffset, - Long endOffset, - Long stopOffset, - File tmpFile) throws Exception { + private void stopPositionTestInternal( + Long startOffset, Long endOffset, Long stopOffset, File tmpFile) throws Exception { String readWithoutSplit; String readWithSplit1, readWithSplit2; StringBuilder accumulatedRead = new StringBuilder(); // Read from source without split attempts. - TextSource textSource = new TextSource<>( - tmpFile.getPath(), false, startOffset, endOffset, - StringUtf8Coder.of()); + TextReader textReader = + new TextReader<>(tmpFile.getPath(), false, startOffset, endOffset, StringUtf8Coder.of()); - try (TextSource.TextFileIterator iterator = - (TextSource.TextFileIterator) textSource.iterator()) { + try (TextReader.TextFileIterator iterator = + (TextReader.TextFileIterator) textReader.iterator()) { while (iterator.hasNext()) { accumulatedRead.append((String) iterator.next()); } @@ -547,13 +513,12 @@ private void stopPositionTestInternal(Long startOffset, } // Read the first half of the split. - textSource = new TextSource<>( - tmpFile.getPath(), false, startOffset, stopOffset, - StringUtf8Coder.of()); + textReader = + new TextReader<>(tmpFile.getPath(), false, startOffset, stopOffset, StringUtf8Coder.of()); accumulatedRead = new StringBuilder(); - try (TextSource.TextFileIterator iterator = - (TextSource.TextFileIterator) textSource.iterator()) { + try (TextReader.TextFileIterator iterator = + (TextReader.TextFileIterator) textReader.iterator()) { while (iterator.hasNext()) { accumulatedRead.append((String) iterator.next()); } @@ -561,13 +526,12 @@ private void stopPositionTestInternal(Long startOffset, } // Read the second half of the split. - textSource = new TextSource<>( - tmpFile.getPath(), false, stopOffset, endOffset, - StringUtf8Coder.of()); + textReader = + new TextReader<>(tmpFile.getPath(), false, stopOffset, endOffset, StringUtf8Coder.of()); accumulatedRead = new StringBuilder(); - try (TextSource.TextFileIterator iterator = - (TextSource.TextFileIterator) textSource.iterator()) { + try (TextReader.TextFileIterator iterator = + (TextReader.TextFileIterator) textReader.iterator()) { while (iterator.hasNext()) { accumulatedRead.append((String) iterator.next()); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleReaderTest.java similarity index 81% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleReaderTest.java index 75d9803ed7eed..5406f4156a140 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedShuffleReaderTest.java @@ -23,9 +23,9 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.cloud.dataflow.sdk.util.common.worker.ShuffleEntry; import com.google.cloud.dataflow.sdk.util.common.worker.Sink; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; import com.google.common.collect.Lists; import org.joda.time.Instant; @@ -39,14 +39,14 @@ import java.util.NoSuchElementException; /** - * Tests for UngroupedShuffleSource. + * Tests for UngroupedShuffleReader. */ @RunWith(JUnit4.class) -public class UngroupedShuffleSourceTest { +public class UngroupedShuffleReaderTest { private static final Instant timestamp = new Instant(123000); private static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000)); - void runTestReadShuffleSource(List expected) throws Exception { + void runTestReadFromShuffle(List expected) throws Exception { Coder> elemCoder = WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), IntervalWindow.getCoder()); @@ -70,14 +70,14 @@ void runTestReadShuffleSource(List expected) throws Exception { Assert.assertEquals(expected.size(), records.size()); Assert.assertEquals(shuffleWriter.getSizes(), actualSizes); - // Read from shuffle with UngroupedShuffleSource. - UngroupedShuffleSource> shuffleSource = - new UngroupedShuffleSource<>( + // Read from shuffle with UngroupedShuffleReader. + UngroupedShuffleReader> ungroupedShuffleReader = + new UngroupedShuffleReader<>( PipelineOptionsFactory.create(), null, null, null, elemCoder); - ExecutorTestUtils.TestSourceObserver observer = - new ExecutorTestUtils.TestSourceObserver(shuffleSource); + ExecutorTestUtils.TestReaderObserver observer = + new ExecutorTestUtils.TestReaderObserver(ungroupedShuffleReader); TestShuffleReader shuffleReader = new TestShuffleReader(); List expectedSizes = new ArrayList<>(); @@ -87,8 +87,8 @@ void runTestReadShuffleSource(List expected) throws Exception { } List actual = new ArrayList<>(); - try (Source.SourceIterator> iter = - shuffleSource.iterator(shuffleReader)) { + try (Reader.ReaderIterator> iter = + ungroupedShuffleReader.iterator(shuffleReader)) { while (iter.hasNext()) { Assert.assertTrue(iter.hasNext()); Assert.assertTrue(iter.hasNext()); @@ -112,12 +112,12 @@ void runTestReadShuffleSource(List expected) throws Exception { } @Test - public void testReadEmptyShuffleSource() throws Exception { - runTestReadShuffleSource(TestUtils.NO_INTS); + public void testReadEmptyShuffleData() throws Exception { + runTestReadFromShuffle(TestUtils.NO_INTS); } @Test - public void testReadNonEmptyShuffleSource() throws Exception { - runTestReadShuffleSource(TestUtils.INTS); + public void testReadNonEmptyShuffleData() throws Exception { + runTestReadFromShuffle(TestUtils.INTS); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java index 89ca01ef724d2..b3cebad3e21e5 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java @@ -17,8 +17,8 @@ package com.google.cloud.dataflow.sdk.util; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; -import com.google.cloud.dataflow.sdk.runners.worker.TextSource; -import com.google.cloud.dataflow.sdk.util.common.worker.Source; +import com.google.cloud.dataflow.sdk.runners.worker.TextReader; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.junit.Assert; import org.junit.Rule; @@ -38,7 +38,6 @@ */ @RunWith(JUnit4.class) public class IOFactoryTest { - @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); @@ -77,13 +76,11 @@ public void testMultiFileRead() throws Exception { output.close(); - TextSource source = new TextSource<>( - tmpFolder.getRoot() + "/file*", - true /* strip newlines */, - null, null, StringUtf8Coder.of()); + TextReader reader = new TextReader<>( + tmpFolder.getRoot() + "/file*", true/* strip newlines */, null, null, StringUtf8Coder.of()); Set records = new TreeSet<>(); - try (Source.SourceIterator iterator = source.iterator()) { + try (Reader.ReaderIterator iterator = reader.iterator()) { while (iterator.hasNext()) { records.add(iterator.next()); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ExecutorTestUtils.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ExecutorTestUtils.java index 07b3bdde5dc26..c38bb56ac9304 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ExecutorTestUtils.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ExecutorTestUtils.java @@ -37,7 +37,7 @@ @SuppressWarnings({"rawtypes", "unchecked"}) public class ExecutorTestUtils { // Do not instantiate. - private ExecutorTestUtils() { } + private ExecutorTestUtils() {} /** An Operation with a specified number of outputs. */ public static class TestOperation extends Operation { @@ -51,33 +51,23 @@ public TestOperation(int numOutputs) { TestOperation(int numOutputs, CounterSet counters, String counterPrefix) { this(numOutputs, counterPrefix, counters.getAddCounterMutator(), - new StateSampler(counterPrefix, counters.getAddCounterMutator())); + new StateSampler(counterPrefix, counters.getAddCounterMutator())); } - TestOperation(int numOutputs, - String counterPrefix, - CounterSet.AddCounterMutator addCounterMutator, - StateSampler stateSampler) { + TestOperation(int numOutputs, String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) { super("TestOperation", - createOutputReceivers(numOutputs, counterPrefix, - addCounterMutator, stateSampler), - counterPrefix, - addCounterMutator, - stateSampler); - } - - private static OutputReceiver[] createOutputReceivers( - int numOutputs, - String counterPrefix, - CounterSet.AddCounterMutator addCounterMutator, - StateSampler stateSampler) { + createOutputReceivers(numOutputs, counterPrefix, addCounterMutator, stateSampler), + counterPrefix, addCounterMutator, stateSampler); + } + + private static OutputReceiver[] createOutputReceivers(int numOutputs, String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) { OutputReceiver[] receivers = new OutputReceiver[numOutputs]; for (int i = 0; i < numOutputs; i++) { - receivers[i] = new OutputReceiver( - "out_" + i, - new ElementByteSizeObservableCoder(StringUtf8Coder.of()), - counterPrefix, - addCounterMutator); + receivers[i] = + new OutputReceiver("out_" + i, new ElementByteSizeObservableCoder(StringUtf8Coder.of()), + counterPrefix, addCounterMutator); } return receivers; } @@ -96,10 +86,7 @@ public TestReceiver(Coder coder) { } public TestReceiver(Coder coder, CounterSet counterSet) { - this("test_receiver_out", - new ElementByteSizeObservableCoder(coder), - counterSet, - "test-"); + this("test_receiver_out", new ElementByteSizeObservableCoder(coder), counterSet, "test-"); } public TestReceiver(CounterSet counterSet, String counterPrefix) { @@ -110,27 +97,20 @@ public TestReceiver(String outputName, CounterSet counterSet) { this(outputName, counterSet, "test-"); } - public TestReceiver(String outputName, - CounterSet counterSet, String counterPrefix) { - this(outputName, - new ElementByteSizeObservableCoder(StringUtf8Coder.of()), - counterSet, - counterPrefix); + public TestReceiver(String outputName, CounterSet counterSet, String counterPrefix) { + this(outputName, new ElementByteSizeObservableCoder(StringUtf8Coder.of()), counterSet, + counterPrefix); } - public TestReceiver(ElementByteSizeObservable elementByteSizeObservable, - CounterSet counterSet, String counterPrefix) { - this("test_receiver_out", elementByteSizeObservable, - counterSet, counterPrefix); + public TestReceiver(ElementByteSizeObservable elementByteSizeObservable, CounterSet counterSet, + String counterPrefix) { + this("test_receiver_out", elementByteSizeObservable, counterSet, counterPrefix); } - public TestReceiver(String outputName, - ElementByteSizeObservable elementByteSizeObservable, - CounterSet counterSet, String counterPrefix) { - super(outputName, - elementByteSizeObservable, - counterPrefix, - counterSet.getAddCounterMutator()); + public TestReceiver(String outputName, ElementByteSizeObservable elementByteSizeObservable, + CounterSet counterSet, String counterPrefix) { + super( + outputName, elementByteSizeObservable, counterPrefix, counterSet.getAddCounterMutator()); } @Override @@ -145,8 +125,8 @@ protected boolean sampleElement() { } } - /** A {@code Source} that yields a specified set of values. */ - public static class TestSource extends Source { + /** A {@code Reader} that yields a specified set of values. */ + public static class TestReader extends Reader { List inputs = new ArrayList<>(); public void addInput(String... inputs) { @@ -154,20 +134,22 @@ public void addInput(String... inputs) { } @Override - public SourceIterator iterator() { - return new TestSourceIterator(inputs); + public ReaderIterator iterator() { + return new TestReaderIterator(inputs); } - class TestSourceIterator extends AbstractSourceIterator { + class TestReaderIterator extends AbstractReaderIterator { Iterator iter; boolean closed = false; - public TestSourceIterator(List inputs) { + public TestReaderIterator(List inputs) { iter = inputs.iterator(); } @Override - public boolean hasNext() { return iter.hasNext(); } + public boolean hasNext() { + return iter.hasNext(); + } @Override public String next() { @@ -188,18 +170,18 @@ public void close() { * An Observer that stores all sizes into an ArrayList, to compare * against the gold standard during testing. */ - public static class TestSourceObserver implements Observer { - private final Source source; + public static class TestReaderObserver implements Observer { + private final Reader reader; private final List sizes; - public TestSourceObserver(Source source) { - this(source, new ArrayList()); + public TestReaderObserver(Reader reader) { + this(reader, new ArrayList()); } - public TestSourceObserver(Source source, List sizes) { - this.source = source; + public TestReaderObserver(Reader reader, List sizes) { + this.reader = reader; this.sizes = sizes; - source.addObserver(this); + reader.addObserver(this); } @Override diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java index fa46e2f235b71..07944893ffebd 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java @@ -16,8 +16,8 @@ package com.google.cloud.dataflow.sdk.util.common.worker; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToSourcePosition; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; @@ -26,8 +26,8 @@ import com.google.cloud.dataflow.sdk.util.common.Counter; import com.google.cloud.dataflow.sdk.util.common.CounterSet; import com.google.cloud.dataflow.sdk.util.common.CounterSet.AddCounterMutator; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils.TestReader; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils.TestReceiver; -import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils.TestSource; import org.hamcrest.CoreMatchers; import org.junit.Assert; @@ -50,29 +50,22 @@ static class TestOperation extends Operation { private static CounterSet counterSet = new CounterSet(); private static String counterPrefix = "test-"; - private static StateSampler stateSampler = new StateSampler( - counterPrefix, counterSet.getAddCounterMutator()); + private static StateSampler stateSampler = + new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); TestOperation(String label, List log) { - super(label, - new OutputReceiver[]{}, - counterPrefix, - counterSet.getAddCounterMutator(), - stateSampler); + super(label, new OutputReceiver[] {}, counterPrefix, counterSet.getAddCounterMutator(), + stateSampler); this.label = label; this.log = log; } - TestOperation(String outputName, - String counterPrefix, - CounterSet.AddCounterMutator addCounterMutator, - StateSampler stateSampler, - long outputCount) { - super(outputName, new OutputReceiver[]{}, - counterPrefix, addCounterMutator, stateSampler); + TestOperation(String outputName, String counterPrefix, + CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler, + long outputCount) { + super(outputName, new OutputReceiver[] {}, counterPrefix, addCounterMutator, stateSampler); addCounterMutator.addCounter( - Counter.longs(outputName + "-ElementCount", SUM) - .resetToValue(outputCount)); + Counter.longs(outputName + "-ElementCount", SUM).resetToValue(outputCount)); } @Override @@ -92,26 +85,21 @@ public void finish() throws Exception { static class TestReadOperation extends ReadOperation { private ApproximateProgress progress = null; - TestReadOperation(OutputReceiver outputReceiver, - String counterPrefix, - AddCounterMutator addCounterMutator, - StateSampler stateSampler) { - super(new TestSource(), outputReceiver, - counterPrefix, addCounterMutator, stateSampler); + TestReadOperation(OutputReceiver outputReceiver, String counterPrefix, + AddCounterMutator addCounterMutator, StateSampler stateSampler) { + super(new TestReader(), outputReceiver, counterPrefix, addCounterMutator, stateSampler); } @Override - public Source.Progress getProgress() { - return cloudProgressToSourceProgress(progress); + public Reader.Progress getProgress() { + return cloudProgressToReaderProgress(progress); } @Override - public Source.Position proposeStopPosition( - Source.Progress proposedStopPosition) { + public Reader.Position proposeStopPosition(Reader.Progress proposedStopPosition) { // Fakes the return with the same position as proposed. - return cloudPositionToSourcePosition( - sourceProgressToCloudProgress(proposedStopPosition) - .getPosition()); + return cloudPositionToReaderPosition( + sourceProgressToCloudProgress(proposedStopPosition).getPosition()); } public void setProgress(ApproximateProgress progress) { @@ -123,27 +111,20 @@ public void setProgress(ApproximateProgress progress) { public void testExecuteMapTaskExecutor() throws Exception { List log = new ArrayList<>(); - List operations = Arrays.asList(new Operation[]{ - new TestOperation("o1", log), - new TestOperation("o2", log), - new TestOperation("o3", log)}); + List operations = Arrays.asList(new Operation[] { + new TestOperation("o1", log), new TestOperation("o2", log), new TestOperation("o3", log)}); CounterSet counters = new CounterSet(); String counterPrefix = "test-"; - StateSampler stateSampler = new StateSampler( - counterPrefix, counters.getAddCounterMutator()); - MapTaskExecutor executor = - new MapTaskExecutor(operations, counters, stateSampler); + StateSampler stateSampler = new StateSampler(counterPrefix, counters.getAddCounterMutator()); + MapTaskExecutor executor = new MapTaskExecutor(operations, counters, stateSampler); executor.execute(); - Assert.assertThat(log, CoreMatchers.hasItems( - "o3 started", - "o2 started", - "o1 started", - "o1 finished", - "o2 finished", - "o3 finished")); + Assert.assertThat( + log, + CoreMatchers.hasItems( + "o3 started", "o2 started", "o1 started", "o1 finished", "o2 finished", "o3 finished")); executor.close(); } @@ -153,55 +134,51 @@ public void testExecuteMapTaskExecutor() throws Exception { public void testGetOutputCounters() throws Exception { CounterSet counters = new CounterSet(); String counterPrefix = "test-"; - StateSampler stateSampler = new StateSampler( - counterPrefix, counters.getAddCounterMutator()); - List operations = Arrays.asList(new Operation[]{ - new TestOperation( - "o1", counterPrefix, counters.getAddCounterMutator(), - stateSampler, 1), - new TestOperation( - "o2", counterPrefix, counters.getAddCounterMutator(), - stateSampler, 2), - new TestOperation( - "o3", counterPrefix, counters.getAddCounterMutator(), - stateSampler, 3)}); + StateSampler stateSampler = new StateSampler(counterPrefix, counters.getAddCounterMutator()); + List operations = Arrays.asList(new Operation[] { + new TestOperation("o1", counterPrefix, counters.getAddCounterMutator(), stateSampler, 1), + new TestOperation("o2", counterPrefix, counters.getAddCounterMutator(), stateSampler, 2), + new TestOperation("o3", counterPrefix, counters.getAddCounterMutator(), stateSampler, 3)}); - MapTaskExecutor executor = - new MapTaskExecutor(operations, counters, stateSampler); + MapTaskExecutor executor = new MapTaskExecutor(operations, counters, stateSampler); CounterSet counterSet = executor.getOutputCounters(); Assert.assertEquals( - new CounterSet( - Counter.longs("o1-ElementCount", SUM).resetToValue(1L), + new CounterSet(Counter.longs("o1-ElementCount", SUM).resetToValue(1L), Counter.longs("test-o1-start-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-o1-start-msecs")).getAggregate(false)), + .resetToValue( + ((Counter) + counterSet.getExistingCounter("test-o1-start-msecs")).getAggregate(false)), Counter.longs("test-o1-process-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-o1-process-msecs")).getAggregate(false)), + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-o1-process-msecs")).getAggregate(false)), Counter.longs("test-o1-finish-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-o1-finish-msecs")).getAggregate(false)), + .resetToValue( + ((Counter) + counterSet.getExistingCounter("test-o1-finish-msecs")).getAggregate(false)), Counter.longs("o2-ElementCount", SUM).resetToValue(2L), Counter.longs("test-o2-start-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-o2-start-msecs")).getAggregate(false)), + .resetToValue( + ((Counter) + counterSet.getExistingCounter("test-o2-start-msecs")).getAggregate(false)), Counter.longs("test-o2-process-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-o2-process-msecs")).getAggregate(false)), + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-o2-process-msecs")).getAggregate(false)), Counter.longs("test-o2-finish-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-o2-finish-msecs")).getAggregate(false)), + .resetToValue( + ((Counter) + counterSet.getExistingCounter("test-o2-finish-msecs")).getAggregate(false)), Counter.longs("o3-ElementCount", SUM).resetToValue(3L), Counter.longs("test-o3-start-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-o3-start-msecs")).getAggregate(false)), + .resetToValue( + ((Counter) + counterSet.getExistingCounter("test-o3-start-msecs")).getAggregate(false)), Counter.longs("test-o3-process-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-o3-process-msecs")).getAggregate(false)), + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-o3-process-msecs")).getAggregate(false)), Counter.longs("test-o3-finish-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-o3-finish-msecs")).getAggregate(false))), + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-o3-finish-msecs")).getAggregate(false))), counterSet); executor.close(); @@ -211,12 +188,10 @@ public void testGetOutputCounters() throws Exception { public void testGetReadOperation() throws Exception { CounterSet counterSet = new CounterSet(); String counterPrefix = "test-"; - StateSampler stateSampler = new StateSampler( - counterPrefix, counterSet.getAddCounterMutator()); + StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); // Test MapTaskExecutor without a single operation. MapTaskExecutor executor = - new MapTaskExecutor(new ArrayList(), - counterSet, stateSampler); + new MapTaskExecutor(new ArrayList(), counterSet, stateSampler); try { ReadOperation readOperation = executor.getReadOperation(); @@ -225,13 +200,10 @@ public void testGetReadOperation() throws Exception { // Exception expected } - List operations = Arrays.asList(new Operation[]{ - new TestOperation("o1", - counterPrefix, counterSet.getAddCounterMutator(), - stateSampler, 1), - new TestOperation("o2", - counterPrefix, counterSet.getAddCounterMutator(), - stateSampler, 2)}); + List operations = Arrays.asList(new Operation[] { + new TestOperation("o1", counterPrefix, counterSet.getAddCounterMutator(), stateSampler, 1), + new TestOperation( + "o2", counterPrefix, counterSet.getAddCounterMutator(), stateSampler, 2)}); // Test MapTaskExecutor without ReadOperation. executor = new MapTaskExecutor(operations, counterSet, stateSampler); @@ -245,10 +217,8 @@ public void testGetReadOperation() throws Exception { executor.close(); TestReceiver receiver = new TestReceiver(counterSet, counterPrefix); - operations = Arrays.asList(new Operation[]{ - new TestReadOperation( - receiver, counterPrefix, counterSet.getAddCounterMutator(), - stateSampler)}); + operations = Arrays.asList(new Operation[] {new TestReadOperation( + receiver, counterPrefix, counterSet.getAddCounterMutator(), stateSampler)}); executor = new MapTaskExecutor(operations, counterSet, stateSampler); Assert.assertEquals(operations.get(0), executor.getReadOperation()); executor.close(); @@ -258,15 +228,12 @@ public void testGetReadOperation() throws Exception { public void testGetProgressAndRequestSplit() throws Exception { CounterSet counterSet = new CounterSet(); String counterPrefix = "test-"; - StateSampler stateSampler = new StateSampler( - counterPrefix, counterSet.getAddCounterMutator()); + StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); TestReceiver receiver = new TestReceiver(counterSet, counterPrefix); - TestReadOperation operation = - new TestReadOperation(receiver, - counterPrefix, counterSet.getAddCounterMutator(), - stateSampler); - MapTaskExecutor executor = new MapTaskExecutor( - Arrays.asList(new Operation[]{operation}), counterSet, stateSampler); + TestReadOperation operation = new TestReadOperation( + receiver, counterPrefix, counterSet.getAddCounterMutator(), stateSampler); + MapTaskExecutor executor = + new MapTaskExecutor(Arrays.asList(new Operation[] {operation}), counterSet, stateSampler); operation.setProgress(new ApproximateProgress().setPosition(makePosition(1L))); Assert.assertEquals( @@ -274,10 +241,8 @@ public void testGetProgressAndRequestSplit() throws Exception { sourceProgressToCloudProgress(executor.getWorkerProgress()).getPosition()); Assert.assertEquals( makePosition(1L), - sourcePositionToCloudPosition( - executor.proposeStopPosition( - cloudProgressToSourceProgress( - new ApproximateProgress().setPosition(makePosition(1L)))))); + sourcePositionToCloudPosition(executor.proposeStopPosition(cloudProgressToReaderProgress( + new ApproximateProgress().setPosition(makePosition(1L)))))); executor.close(); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java index c324a0bfe0010..922e32f1887af 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java @@ -16,7 +16,7 @@ package com.google.cloud.dataflow.sdk.util.common.worker; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToSourceProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN; @@ -29,8 +29,8 @@ import com.google.api.services.dataflow.model.Position; import com.google.cloud.dataflow.sdk.util.common.Counter; import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils.TestReader; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils.TestReceiver; -import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils.TestSource; import org.hamcrest.CoreMatchers; import org.junit.Assert; @@ -53,19 +53,18 @@ public class ReadOperationTest { private static final long ITERATIONS = 3L; /** - * The test Source for testing updating stop position and progress report. + * The test Reader for testing updating stop position and progress report. * The number of read iterations is controlled by ITERATIONS. */ - static class TestTextSource extends Source { + static class TestTextReader extends Reader { @Override - public SourceIterator iterator() { - return new TestTextSourceIterator(); + public ReaderIterator iterator() { + return new TestTextReaderIterator(); } - class TestTextSourceIterator extends AbstractSourceIterator { + class TestTextReaderIterator extends AbstractReaderIterator { long offset = 0L; - List proposedPositions = - new ArrayList<>(); + List proposedPositions = new ArrayList<>(); @Override public boolean hasNext() { @@ -91,7 +90,7 @@ public Progress getProgress() { ApproximateProgress progress = new ApproximateProgress(); progress.setPosition(currentPosition); - return cloudProgressToSourceProgress(progress); + return cloudProgressToReaderProgress(progress); } @Override @@ -105,7 +104,7 @@ public Position updateStopPosition(Progress proposedStopPosition) { /** * The OutputReceiver for testing updating stop position and progress report. - * The offset of the Source (iterator) will be advanced each time this + * The offset of the Reader (iterator) will be advanced each time this * Receiver processes a record. */ static class TestTextReceiver extends OutputReceiver { @@ -131,18 +130,17 @@ public void process(Object outputElem) throws Exception { progresses.add(sourceProgressToCloudProgress(readOperation.getProgress())); // We expect that call to proposeStopPosition is a no-op that does not // update the stop position for every iteration. We will verify it is - // delegated to SourceIterator after ReadOperation finishes. - Assert.assertNull( - readOperation.proposeStopPosition( - cloudProgressToSourceProgress(makeApproximateProgress(proposedStopPosition)))); + // delegated to ReaderIterator after ReadOperation finishes. + Assert.assertNull(readOperation.proposeStopPosition( + cloudProgressToReaderProgress(makeApproximateProgress(proposedStopPosition)))); } } @Test @SuppressWarnings("unchecked") public void testRunReadOperation() throws Exception { - TestSource source = new TestSource(); - source.addInput("hi", "there", "", "bob"); + TestReader reader = new TestReader(); + reader.addInput("hi", "there", "", "bob"); CounterSet counterSet = new CounterSet(); String counterPrefix = "test-"; @@ -150,7 +148,7 @@ public void testRunReadOperation() throws Exception { TestReceiver receiver = new TestReceiver(counterSet, counterPrefix); ReadOperation readOperation = new ReadOperation( - source, receiver, counterPrefix, counterSet.getAddCounterMutator(), stateSampler); + reader, receiver, counterPrefix, counterSet.getAddCounterMutator(), stateSampler); readOperation.start(); readOperation.finish(); @@ -158,35 +156,36 @@ public void testRunReadOperation() throws Exception { Assert.assertThat( receiver.outputElems, CoreMatchers.hasItems("hi", "there", "", "bob")); - Assert.assertEquals( - new CounterSet( - Counter.longs("ReadOperation-ByteCount", SUM).resetToValue(2L + 5 + 0 + 3), - Counter.longs("test_receiver_out-ElementCount", SUM).resetToValue(4L), - Counter.longs("test_receiver_out-MeanByteCount", MEAN).resetToValue(4, 10L), - Counter.longs("test-ReadOperation-start-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-ReadOperation-start-msecs")).getAggregate(false)), - Counter.longs("test-ReadOperation-read-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-ReadOperation-read-msecs")).getAggregate(false)), - Counter.longs("test-ReadOperation-process-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-ReadOperation-process-msecs")).getAggregate(false)), - Counter.longs("test-ReadOperation-finish-msecs", SUM) - .resetToValue(((Counter) counterSet.getExistingCounter( - "test-ReadOperation-finish-msecs")).getAggregate(false))), - counterSet); + Assert + .assertEquals( + new CounterSet( + Counter.longs("ReadOperation-ByteCount", SUM).resetToValue(2L + 5 + 0 + 3), + Counter.longs("test_receiver_out-ElementCount", SUM).resetToValue(4L), + Counter.longs("test_receiver_out-MeanByteCount", MEAN).resetToValue(4, 10L), + Counter.longs("test-ReadOperation-start-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-ReadOperation-start-msecs")).getAggregate(false)), + Counter.longs("test-ReadOperation-read-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-ReadOperation-read-msecs")).getAggregate(false)), + Counter.longs("test-ReadOperation-process-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-ReadOperation-process-msecs")).getAggregate(false)), + Counter.longs("test-ReadOperation-finish-msecs", SUM) + .resetToValue(((Counter) counterSet.getExistingCounter( + "test-ReadOperation-finish-msecs")).getAggregate(false))), + counterSet); } @Test public void testGetProgressAndProposeStopPosition() throws Exception { - TestTextSource testSource = new TestTextSource(); + TestTextReader testTextReader = new TestTextReader(); CounterSet counterSet = new CounterSet(); String counterPrefix = "test-"; StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); TestTextReceiver receiver = new TestTextReceiver(counterSet, counterPrefix); ReadOperation readOperation = new ReadOperation( - testSource, receiver, counterPrefix, counterSet.getAddCounterMutator(), stateSampler); + testTextReader, receiver, counterPrefix, counterSet.getAddCounterMutator(), stateSampler); readOperation.setProgressUpdatePeriodMs(0); receiver.setReadOperation(readOperation); @@ -195,37 +194,36 @@ public void testGetProgressAndProposeStopPosition() throws Exception { Assert.assertNull(readOperation.getProgress()); Assert.assertNull(readOperation.proposeStopPosition( - cloudProgressToSourceProgress( - makeApproximateProgress(proposedStopPosition)))); + cloudProgressToReaderProgress(makeApproximateProgress(proposedStopPosition)))); readOperation.start(); readOperation.finish(); - TestTextSource.TestTextSourceIterator testIterator = - (TestTextSource.TestTextSourceIterator) readOperation.sourceIterator; + TestTextReader.TestTextReaderIterator testIterator = + (TestTextReader.TestTextReaderIterator) readOperation.readerIterator; - Assert.assertEquals(sourceProgressToCloudProgress(testIterator.getProgress()), - sourceProgressToCloudProgress(readOperation.getProgress())); - Assert.assertEquals(sourcePositionToCloudPosition(testIterator.updateStopPosition( - cloudProgressToSourceProgress( - makeApproximateProgress(proposedStopPosition)))), - sourcePositionToCloudPosition(readOperation.proposeStopPosition( - cloudProgressToSourceProgress( - makeApproximateProgress(proposedStopPosition))))); + Assert.assertEquals( + sourceProgressToCloudProgress(testIterator.getProgress()), + sourceProgressToCloudProgress(readOperation.getProgress())); + Assert.assertEquals( + sourcePositionToCloudPosition(testIterator.updateStopPosition( + cloudProgressToReaderProgress(makeApproximateProgress(proposedStopPosition)))), + sourcePositionToCloudPosition(readOperation.proposeStopPosition( + cloudProgressToReaderProgress(makeApproximateProgress(proposedStopPosition))))); // Verifies progress report and stop position updates. Assert.assertEquals(testIterator.proposedPositions.size(), ITERATIONS + 2); + Assert.assertThat(testIterator.proposedPositions, everyItem(equalTo(makePosition(3L)))); Assert.assertThat( - testIterator.proposedPositions, everyItem(equalTo(makePosition(3L)))); - Assert.assertThat( - receiver.progresses, contains(makeApproximateProgress(1L), makeApproximateProgress(2L), - makeApproximateProgress(3L))); + receiver.progresses, + contains( + makeApproximateProgress(1L), makeApproximateProgress(2L), makeApproximateProgress(3L))); } @Test public void testGetProgressDoesNotBlock() throws Exception { final BlockingQueue queue = new LinkedBlockingQueue<>(); - final Source.SourceIterator iterator = new Source.AbstractSourceIterator() { + final Reader.ReaderIterator iterator = new Reader.AbstractReaderIterator() { private int itemsReturned = 0; @Override @@ -244,15 +242,15 @@ public Integer next() throws IOException { } @Override - public Source.Progress getProgress() { - return cloudProgressToSourceProgress(new ApproximateProgress().setPosition( + public Reader.Progress getProgress() { + return cloudProgressToReaderProgress(new ApproximateProgress().setPosition( new Position().setRecordIndex((long) itemsReturned))); } }; - Source source = new Source() { + Reader reader = new Reader() { @Override - public SourceIterator iterator() throws IOException { + public ReaderIterator iterator() throws IOException { return iterator; } }; @@ -262,7 +260,7 @@ public SourceIterator iterator() throws IOException { StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); TestTextReceiver receiver = new TestTextReceiver(counterSet, counterPrefix); final ReadOperation readOperation = new ReadOperation( - source, receiver, counterPrefix, counterSet.getAddCounterMutator(), stateSampler); + reader, receiver, counterPrefix, counterSet.getAddCounterMutator(), stateSampler); // Update progress not continuously, but so that it's never more than 1 record stale. readOperation.setProgressUpdatePeriodMs(150); receiver.setReadOperation(readOperation); From 239a98328ad84109c4ad2e43c37f1d499208da19 Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 22 Dec 2014 18:55:02 -0800 Subject: [PATCH 0054/1541] Add basic encode/decode tests for most coders. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82692960 --- .../dataflow/sdk/coders/CollectionCoder.java | 2 +- .../dataflow/sdk/coders/IterableCoder.java | 2 +- .../sdk/coders/IterableLikeCoder.java | 22 +-- .../cloud/dataflow/sdk/coders/SetCoder.java | 82 +++--------- .../dataflow/sdk/coders/StringUtf8Coder.java | 2 +- .../sdk/coders/TableRowJsonCoder.java | 3 +- .../sdk/coders/TextualIntegerCoder.java | 2 +- .../dataflow/sdk/coders/VarIntCoder.java | 2 +- .../cloud/dataflow/sdk/coders/VoidCoder.java | 2 +- .../dataflow/sdk/coders/AvroCoderTest.java | 13 +- .../sdk/coders/BigEndianIntegerCoderTest.java | 44 ++++++ .../sdk/coders/BigEndianLongCoderTest.java | 46 +++++++ .../sdk/coders/ByteArrayCoderTest.java | 41 ++---- .../dataflow/sdk/coders/CoderProperties.java | 126 ++++++++++++++++-- .../sdk/coders/CollectionCoderTest.java | 51 +++++++ .../dataflow/sdk/coders/CustomCoderTest.java | 10 +- .../dataflow/sdk/coders/EntityCoderTest.java | 67 ++++++++++ .../dataflow/sdk/coders/InstantCoderTest.java | 5 +- .../sdk/coders/IterableCoderTest.java | 18 +++ .../dataflow/sdk/coders/KvCoderTest.java | 67 ++++++++++ .../dataflow/sdk/coders/ListCoderTest.java | 28 +++- .../dataflow/sdk/coders/MapCoderTest.java | 16 +++ .../dataflow/sdk/coders/SetCoderTest.java | 47 +++++++ .../sdk/coders/StringUtf8CoderTest.java | 45 +++++++ .../sdk/coders/TableRowJsonCoderTest.java | 61 +++++++++ .../sdk/coders/TextualIntegerCoderTest.java | 44 ++++++ .../dataflow/sdk/coders/URICoderTest.java | 19 +-- .../dataflow/sdk/coders/VarIntCoderTest.java | 45 +++++++ .../dataflow/sdk/coders/VarLongCoderTest.java | 46 +++++++ 29 files changed, 801 insertions(+), 157 deletions(-) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CollectionCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/EntityCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/KvCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/SetCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/StringUtf8CoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/VarIntCoderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/VarLongCoderTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java index c75f645d344ef..b78c2674176e0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CollectionCoder.java @@ -63,7 +63,7 @@ public static List getInstanceComponents( return getInstanceComponentsHelper(exampleValue); } - CollectionCoder(Coder elemCoder) { + protected CollectionCoder(Coder elemCoder) { super(elemCoder); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java index 79d6173742a88..61737e298463d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableCoder.java @@ -65,7 +65,7 @@ public static List getInstanceComponents( return getInstanceComponentsHelper(exampleValue); } - IterableCoder(Coder elemCoder) { + protected IterableCoder(Coder elemCoder) { super(elemCoder); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java index 128d23bf21a2c..c9332853e267f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java @@ -40,7 +40,7 @@ public abstract class IterableLikeCoder> extends StandardCoder { - public Coder getElemCoder() { return elemCoder; } + public Coder getElemCoder() { return elementCoder; } /** * Builds an instance of the coder's associated {@code Iterable} from a list @@ -53,7 +53,7 @@ public abstract class IterableLikeCoder> ///////////////////////////////////////////////////////////////////////////// // Internal operations below here. - final Coder elemCoder; + private final Coder elementCoder; /** * Returns the first element in this iterable-like if it is non-empty, @@ -68,8 +68,8 @@ List getInstanceComponentsHelper( return null; } - protected IterableLikeCoder(Coder elemCoder) { - this.elemCoder = elemCoder; + protected IterableLikeCoder(Coder elementCoder) { + this.elementCoder = elementCoder; } @Override @@ -86,7 +86,7 @@ public void encode(IT iterable, OutputStream outStream, Context context) Collection collection = (Collection) iterable; dataOutStream.writeInt(collection.size()); for (T elem : collection) { - elemCoder.encode(elem, dataOutStream, nestedContext); + elementCoder.encode(elem, dataOutStream, nestedContext); } } else { // We don't know the size without traversing it. So use a @@ -95,7 +95,7 @@ public void encode(IT iterable, OutputStream outStream, Context context) dataOutStream.writeInt(-1); for (T elem : iterable) { dataOutStream.writeBoolean(true); - elemCoder.encode(elem, dataOutStream, nestedContext); + elementCoder.encode(elem, dataOutStream, nestedContext); } dataOutStream.writeBoolean(false); } @@ -112,7 +112,7 @@ public IT decode(InputStream inStream, Context context) if (size >= 0) { List elements = new ArrayList<>(size); for (int i = 0; i < size; i++) { - elements.add(elemCoder.decode(dataInStream, nestedContext)); + elements.add(elementCoder.decode(dataInStream, nestedContext)); } return decodeToIterable(elements); } else { @@ -120,7 +120,7 @@ public IT decode(InputStream inStream, Context context) // each element. List elements = new ArrayList<>(); while (dataInStream.readBoolean()) { - elements.add(elemCoder.decode(dataInStream, nestedContext)); + elements.add(elementCoder.decode(dataInStream, nestedContext)); } return decodeToIterable(elements); } @@ -128,7 +128,7 @@ public IT decode(InputStream inStream, Context context) @Override public List> getCoderArguments() { - return Arrays.asList(elemCoder); + return Arrays.asList(elementCoder); } /** @@ -176,7 +176,7 @@ public void registerByteSizeObserver( Collection collection = (Collection) iterable; observer.update(4L); for (T elem : collection) { - elemCoder.registerByteSizeObserver(elem, observer, nestedContext); + elementCoder.registerByteSizeObserver(elem, observer, nestedContext); } } else { // We don't know the size without traversing it. So use a @@ -185,7 +185,7 @@ public void registerByteSizeObserver( observer.update(4L); for (T elem : iterable) { observer.update(1L); - elemCoder.registerByteSizeObserver(elem, observer, nestedContext); + elementCoder.registerByteSizeObserver(elem, observer, nestedContext); } observer.update(1L); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java index 8b5fca7638a5c..9a65d39a0e403 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java @@ -18,17 +18,10 @@ import com.google.api.client.util.Preconditions; import com.google.cloud.dataflow.sdk.util.PropertyNames; -import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -39,7 +32,7 @@ * @param the type of the elements of the set */ @SuppressWarnings("serial") -public class SetCoder extends StandardCoder> { +public class SetCoder extends IterableLikeCoder> { /** * Produces a SetCoder with the given elementCoder. @@ -48,6 +41,9 @@ public static SetCoder of(Coder elementCoder) { return new SetCoder<>(elementCoder); } + /** + * Dynamically typed constructor for JSON deserialization. + */ @JsonCreator public static SetCoder of( @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) @@ -57,53 +53,11 @@ public static SetCoder of( return of((Coder) components.get(0)); } - public Coder getElementCoder() { return elementCoder; } - - ///////////////////////////////////////////////////////////////////////////// - - Coder elementCoder; - - SetCoder(Coder elementCoder) { - this.elementCoder = elementCoder; - } - - @Override - public void encode( - Set set, - OutputStream outStream, - Context context) - throws IOException, CoderException { - DataOutputStream dataOutStream = new DataOutputStream(outStream); - dataOutStream.writeInt(set.size()); - for (T element : set) { - elementCoder.encode(element, outStream, context.nested()); - } - dataOutStream.flush(); - } - - @Override - public Set decode(InputStream inStream, Context context) - throws IOException, CoderException { - DataInputStream dataInStream = new DataInputStream(inStream); - int size = dataInStream.readInt(); - Set retval = new HashSet(); - for (int i = 0; i < size; ++i) { - T element = elementCoder.decode(inStream, context.nested()); - retval.add(element); - } - return retval; - } - - @Override - public List> getCoderArguments() { - return Arrays.>asList(elementCoder); - } - /** * Not all sets have a deterministic encoding. * - *

For example, HashSet comparison does not depend on element order, so - * two HashSet instances may be equal but produce different encodings. + *

For example, {@code HashSet} comparison does not depend on element order, so + * two {@code HashSet} instances may be equal but produce different encodings. */ @Override public boolean isDeterministic() { @@ -111,15 +65,23 @@ public boolean isDeterministic() { } /** - * Notifies ElementByteSizeObserver about the byte size of the encoded value using this coder. + * Returns the first element in this set if it is non-empty, + * otherwise returns {@code null}. */ + public static List getInstanceComponents( + Set exampleValue) { + return getInstanceComponentsHelper(exampleValue); + } + + ///////////////////////////////////////////////////////////////////////////// + // Internal operations below here. + @Override - public void registerByteSizeObserver( - Set set, ElementByteSizeObserver observer, Context context) - throws Exception { - observer.update(4L); - for (T element : set) { - elementCoder.registerByteSizeObserver(element, observer, context.nested()); - } + protected final Set decodeToIterable(List decodedElements) { + return new HashSet(decodedElements); + } + + protected SetCoder(Coder elemCoder) { + super(elemCoder); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java index 94db9e4ed3899..48b807b6fcf8c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java @@ -37,12 +37,12 @@ */ @SuppressWarnings("serial") public class StringUtf8Coder extends AtomicCoder { + @JsonCreator public static StringUtf8Coder of() { return INSTANCE; } - ///////////////////////////////////////////////////////////////////////////// private static final StringUtf8Coder INSTANCE = new StringUtf8Coder(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java index f21aaa513eb2f..349ff323c1519 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java @@ -67,8 +67,7 @@ protected long getEncodedElementByteSize(TableRow value, Context context) private static final TableRowJsonCoder INSTANCE = new TableRowJsonCoder(); - private TableRowJsonCoder() { - } + private TableRowJsonCoder() { } /** * TableCell can hold arbitrary Object instances, which makes the encoding diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java index bd01ecbcca7b7..92f343bd4d7e9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java @@ -27,12 +27,12 @@ */ @SuppressWarnings("serial") public class TextualIntegerCoder extends AtomicCoder { + @JsonCreator public static TextualIntegerCoder of() { return new TextualIntegerCoder(); } - ///////////////////////////////////////////////////////////////////////////// private TextualIntegerCoder() {} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java index 2ce9ffd632e0f..f357fce890284 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java @@ -33,12 +33,12 @@ */ @SuppressWarnings("serial") public class VarIntCoder extends AtomicCoder { + @JsonCreator public static VarIntCoder of() { return INSTANCE; } - ///////////////////////////////////////////////////////////////////////////// private static final VarIntCoder INSTANCE = diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java index 24d5e061cfc73..e949fdfd03484 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java @@ -26,12 +26,12 @@ */ @SuppressWarnings("serial") public class VoidCoder extends AtomicCoder { + @JsonCreator public static VoidCoder of() { return INSTANCE; } - ///////////////////////////////////////////////////////////////////////////// private static final VoidCoder INSTANCE = new VoidCoder(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java index fffbb5839e7ca..e3da71bcdff08 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java @@ -24,7 +24,6 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.util.CloudObject; -import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.values.PCollection; import org.apache.avro.Schema; @@ -111,13 +110,10 @@ public void testAvroCoderEncoding() throws Exception { @Test public void testPojoEncoding() throws Exception { - Pojo before = new Pojo("Hello", 42); - + Pojo value = new Pojo("Hello", 42); AvroCoder coder = AvroCoder.of(Pojo.class); - byte[] bytes = CoderUtils.encodeToByteArray(coder, before); - Pojo after = CoderUtils.decodeFromByteArray(coder, bytes); - Assert.assertEquals(before, after); + CoderProperties.coderDecodeEncodeEqual(coder, value); } @Test @@ -140,11 +136,8 @@ public void testGenericRecordEncoding() throws Exception { // Leave favorite_color null AvroCoder coder = AvroCoder.of(GenericRecord.class, schema); - byte[] bytes = CoderUtils.encodeToByteArray(coder, before); - GenericRecord after = CoderUtils.decodeFromByteArray(coder, bytes); - - Assert.assertEquals(before, after); + CoderProperties.coderDecodeEncodeEqual(coder, before); Assert.assertEquals(schema, coder.getSchema()); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoderTest.java new file mode 100644 index 0000000000000..14c4a72f8d7ff --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/BigEndianIntegerCoderTest.java @@ -0,0 +1,44 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Test case for {@link BigEndianIntegerCoder}. + */ +@RunWith(JUnit4.class) +public class BigEndianIntegerCoderTest { + + private static final List TEST_VALUES = Arrays.asList( + -11, -3, -1, 0, 1, 5, 13, 29, + Integer.MAX_VALUE, + Integer.MIN_VALUE); + + @Test + public void testDecodeEncodeEqual() throws Exception { + Coder coder = BigEndianIntegerCoder.of(); + for (Integer value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeEqual(coder, value); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoderTest.java new file mode 100644 index 0000000000000..4e94e6bca45a0 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/BigEndianLongCoderTest.java @@ -0,0 +1,46 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Test case for {@link BigEndianLongCoder}. + */ +@RunWith(JUnit4.class) +public class BigEndianLongCoderTest { + + private static final List TEST_VALUES = Arrays.asList( + -11L, -3L, -1L, 0L, 1L, 5L, 13L, 29L, + Integer.MAX_VALUE + 131L, + Integer.MIN_VALUE - 29L, + Long.MAX_VALUE, + Long.MIN_VALUE); + + @Test + public void testDecodeEncodeEqual() throws Exception { + Coder coder = BigEndianLongCoder.of(); + for (Long value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeEqual(coder, value); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoderTest.java index 8f500a1655994..b8715a66c7057 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoderTest.java @@ -16,49 +16,26 @@ package com.google.cloud.dataflow.sdk.coders; -import static org.hamcrest.Matchers.equalTo; -import static org.junit.Assert.assertThat; - import com.google.cloud.dataflow.sdk.util.common.CounterTestUtils; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; - -/** Unit tests for {@link ByteArrayCoder}. */ +/** + * Unit tests for {@link ByteArrayCoder}. + */ @RunWith(JUnit4.class) public class ByteArrayCoderTest { - @Test - public void testOuterContext() throws CoderException, IOException { - byte[] buffer = {0xa, 0xb, 0xc}; - ByteArrayOutputStream os = new ByteArrayOutputStream(); - ByteArrayCoder.of().encode(buffer, os, Coder.Context.OUTER); - byte[] encoded = os.toByteArray(); - - ByteArrayInputStream is = new ByteArrayInputStream(encoded); - byte[] decoded = ByteArrayCoder.of().decode(is, Coder.Context.OUTER); - assertThat(decoded, equalTo(buffer)); - } + private static final byte[][] TEST_VALUES = { + {0xa, 0xb, 0xc}, {}, {}, {0xd, 0xe}, {}}; @Test - public void testNestedContext() throws CoderException, IOException { - byte[][] buffers = {{0xa, 0xb, 0xc}, {}, {}, {0xd, 0xe}, {}}; - - ByteArrayOutputStream os = new ByteArrayOutputStream(); - for (byte[] buffer : buffers) { - ByteArrayCoder.of().encode(buffer, os, Coder.Context.NESTED); - } - byte[] encoded = os.toByteArray(); - - ByteArrayInputStream is = new ByteArrayInputStream(encoded); - for (byte[] buffer : buffers) { - byte[] decoded = ByteArrayCoder.of().decode(is, Coder.Context.NESTED); - assertThat(decoded, equalTo(buffer)); + public void testDecodeEncodeEquals() throws Exception { + ByteArrayCoder coder = ByteArrayCoder.of(); + for (byte[] value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeEqual(coder, value); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java index ef096eb01c999..89754789cc17f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java @@ -16,14 +16,21 @@ package com.google.cloud.dataflow.sdk.coders; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assume.assumeThat; +import com.google.common.collect.Iterables; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; /** * Properties for use in {@link Coder} tests. These are implemented with junit assertions @@ -31,28 +38,125 @@ */ public class CoderProperties { + /** + * All the contexts, for use in test cases. + */ + public static final List ALL_CONTEXTS = Arrays.asList( + Coder.Context.OUTER, Coder.Context.NESTED); + + /** + * Verifies that for the given {@link Coder}, and values of + * type {@code T}, if the values are equal then the encoded bytes are equal, + * in any {@link Coder.Context}. + */ + public static void coderDeterministic( + Coder coder, T value1, T value2) + throws Exception { + for (Coder.Context context : ALL_CONTEXTS) { + coderDeterministicInContext(coder, context, value1, value2); + } + } + /** * Verifies that for the given {@link Coder}, {@link Coder.Context}, and values of * type {@code T}, if the values are equal then the encoded bytes are equal. */ - public static void coderDeterministic( + public static void coderDeterministicInContext( Coder coder, Coder.Context context, T value1, T value2) throws Exception { assumeThat(value1, equalTo(value2)); - assertArrayEquals(encode(coder, context, value1), encode(coder, context, value2)); + assertThat( + encode(coder, context, value1), + equalTo(encode(coder, context, value2))); } /** - * Verifies that for the given {@link Coder}, {@link Coder.Context}, + * Verifies that for the given {@link Coder}, * and value of type {@code T}, encoding followed by decoding yields an - * equal of type {@code T}. + * equal value of type {@code T}, in any {@link Coder.Context}. */ public static void coderDecodeEncodeEqual( + Coder coder, T value) + throws Exception { + for (Coder.Context context : ALL_CONTEXTS) { + coderDecodeEncodeEqualInContext(coder, context, value); + } + } + + /** + * Verifies that for the given {@link Coder}, {@link Coder.Context}, + * and value of type {@code T}, encoding followed by decoding yields an + * equal value of type {@code T}. + */ + public static void coderDecodeEncodeEqualInContext( Coder coder, Coder.Context context, T value) throws Exception { - assertEquals( - decode(coder, context, encode(coder, context, value)), - value); + assertThat(decodeEncode(coder, context, value), equalTo(value)); + } + + /** + * Verifies that for the given {@link Coder>}, + * and value of type {@code Collection}, encoding followed by decoding yields an + * equal value of type {@code Collection}, in any {@link Coder.Context}. + */ + public static > void coderDecodeEncodeContentsEqual( + Coder coder, IT value) + throws Exception { + for (Coder.Context context : ALL_CONTEXTS) { + coderDecodeEncodeContentsEqualInContext(coder, context, value); + } + } + + /** + * Verifies that for the given {@link Coder>}, + * and value of type {@code Collection}, encoding followed by decoding yields an + * equal value of type {@code Collection}, in the given {@link Coder.Context}. + */ + @SuppressWarnings("unchecked") + public static > void coderDecodeEncodeContentsEqualInContext( + Coder coder, Coder.Context context, CT value) + throws Exception { + // Matchers.containsInAnyOrder() requires at least one element + Collection result = decodeEncode(coder, context, value); + if (value.isEmpty()) { + assertThat(result, emptyIterable()); + } else { + // This is the only Matchers.containInAnyOrder() overload that takes literal values + assertThat(result, containsInAnyOrder((T[]) value.toArray())); + } + } + + /** + * Verifies that for the given {@link Coder>}, + * and value of type {@code Collection}, encoding followed by decoding yields an + * equal value of type {@code Collection}, in any {@link Coder.Context}. + */ + public static > void coderDecodeEncodeContentsInSameOrder( + Coder coder, IT value) + throws Exception { + for (Coder.Context context : ALL_CONTEXTS) { + CoderProperties.coderDecodeEncodeContentsInSameOrderInContext( + coder, context, value); + } + } + + /** + * Verifies that for the given {@link Coder>}, + * and value of type {@code Iterable}, encoding followed by decoding yields an + * equal value of type {@code Collection}, in the given {@link Coder.Context}. + */ + @SuppressWarnings("unchecked") + public static > void coderDecodeEncodeContentsInSameOrderInContext( + Coder coder, Coder.Context context, IT value) + throws Exception { + Iterable result = decodeEncode(coder, context, value); + // Matchers.contains() requires at least one element + if (Iterables.isEmpty(value)) { + assertThat(result, emptyIterable()); + } else { + // This is the only Matchers.contains() overload that takes literal values + assertThat(result, contains((T[]) Iterables.toArray(value, Object.class))); + } } ////////////////////////////////////////////////////////////////////////// @@ -70,4 +174,8 @@ private static T decode( return coder.decode(is, context); } + private static T decodeEncode(Coder coder, Coder.Context context, T value) + throws CoderException, IOException { + return decode(coder, context, encode(coder, context, value)); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CollectionCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CollectionCoderTest.java new file mode 100644 index 0000000000000..ffacbd768a4e4 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CollectionCoderTest.java @@ -0,0 +1,51 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; + +/** + * Test case for {@link CollectionCoder}. + */ +@RunWith(JUnit4.class) +public class CollectionCoderTest { + + private static final List> TEST_VALUES = Arrays.>asList( + Collections.emptyList(), + Collections.emptySet(), + Collections.singletonList(13), + Arrays.asList(1, 2, 3, 4), + new LinkedList<>(Arrays.asList(7, 6, 5)), + new HashSet<>(Arrays.asList(31, -5, 83))); + + @Test + public void testDecodeEncodeContentsEqual() throws Exception { + Coder> coder = CollectionCoder.of(VarIntCoder.of()); + for (Collection value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeContentsEqual(coder, value); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CustomCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CustomCoderTest.java index c71a50843fd02..229bc7d5ae9d7 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CustomCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CustomCoderTest.java @@ -67,18 +67,18 @@ public int hashCode() { } } - @Test public void testEncodeDecode() throws Exception { + @Test + public void testEncodeDecode() throws Exception { MyCustomCoder coder = new MyCustomCoder("key"); - byte[] encoded = CoderUtils.encodeToByteArray(coder, KV.of("key", 3L)); - Assert.assertEquals( - KV.of("key", 3L), CoderUtils.decodeFromByteArray(coder, encoded)); + CoderProperties.coderDecodeEncodeEqual(coder, KV.of("key", 3L)); byte[] encoded2 = CoderUtils.encodeToByteArray(coder, KV.of("ignored", 3L)); Assert.assertEquals( KV.of("key", 3L), CoderUtils.decodeFromByteArray(coder, encoded2)); } - @Test public void testEncodable() throws Exception { + @Test + public void testEncodable() throws Exception { SerializableUtils.ensureSerializable(new MyCustomCoder("key")); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/EntityCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/EntityCoderTest.java new file mode 100644 index 0000000000000..bd388a3343d1e --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/EntityCoderTest.java @@ -0,0 +1,67 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static com.google.api.services.datastore.client.DatastoreHelper.makeKey; +import static com.google.api.services.datastore.client.DatastoreHelper.makeProperty; +import static com.google.api.services.datastore.client.DatastoreHelper.makeValue; + +import com.google.api.services.datastore.DatastoreV1.Entity; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Test case for {@link EntityCoder}. + */ +@RunWith(JUnit4.class) +public class EntityCoderTest { + + // Presumably if anything works, everything works, + // as actual serialization is fully delegated to + // autogenerated code from a well-tested library. + private static final List TEST_VALUES = Arrays.asList( + Entity.newBuilder() + .setKey(makeKey("TestKind", "emptyEntity")) + .build(), + Entity.newBuilder() + .setKey(makeKey("TestKind", "testSimpleProperties")) + .addProperty(makeProperty("trueProperty", makeValue(true))) + .addProperty(makeProperty("falseProperty", makeValue(false))) + .addProperty(makeProperty("stringProperty", makeValue("hello"))) + .addProperty(makeProperty("integerProperty", makeValue(3))) + .addProperty(makeProperty("doubleProperty", makeValue(-1.583257))) + .build(), + Entity.newBuilder() + .setKey(makeKey("TestKind", "testNestedEntity")) + .addProperty(makeProperty("entityProperty", + makeValue(Entity.newBuilder() + .addProperty(makeProperty("stringProperty", makeValue("goodbye")))))) + .build()); + + @Test + public void testDecodeEncodeEqual() throws Exception { + Coder coder = EntityCoder.of(); + for (Entity value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeEqual(coder, value); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/InstantCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/InstantCoderTest.java index dd719004eab1c..31f80c92ae730 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/InstantCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/InstantCoderTest.java @@ -34,6 +34,7 @@ /** Unit tests for {@link InstantCoder}. */ @RunWith(JUnit4.class) public class InstantCoderTest { + private final InstantCoder coder = InstantCoder.of(); private final List timestamps = Arrays.asList(0L, 1L, -1L, -255L, 256L, Long.MIN_VALUE, Long.MAX_VALUE); @@ -41,9 +42,7 @@ public class InstantCoderTest { @Test public void testBasicEncoding() throws Exception { for (long timestamp : timestamps) { - Assert.assertEquals(new Instant(timestamp), - CoderUtils.decodeFromByteArray(coder, - CoderUtils.encodeToByteArray(coder, new Instant(timestamp)))); + CoderProperties.coderDecodeEncodeEqual(coder, new Instant(timestamp)); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/IterableCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/IterableCoderTest.java index 993c5d0a5e917..f44b53f828103 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/IterableCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/IterableCoderTest.java @@ -24,11 +24,29 @@ import org.junit.runners.JUnit4; import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; import java.util.List; /** Unit tests for {@link IterableCoder}. */ @RunWith(JUnit4.class) public class IterableCoderTest { + + private static final List> TEST_VALUES = Arrays.>asList( + Collections.emptyList(), + Collections.singletonList(13), + Arrays.asList(1, 2, 3, 4), + new LinkedList(Arrays.asList(7, 6, 5))); + + @Test + public void testDecodeEncodeContentsInSameOrder() throws Exception { + Coder> coder = IterableCoder.of(VarIntCoder.of()); + for (Iterable value : TEST_VALUES) { + CoderProperties.>coderDecodeEncodeContentsInSameOrder( + coder, value); + } + } + @Test public void testGetInstanceComponentsNonempty() { Iterable iterable = Arrays.asList(2, 58, 99, 5); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/KvCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/KvCoderTest.java new file mode 100644 index 0000000000000..924e960740d63 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/KvCoderTest.java @@ -0,0 +1,67 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.collect.ImmutableMap; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; + +/** + * Test case for {@link KvCoder}. + */ +@RunWith(JUnit4.class) +public class KvCoderTest { + + private static final Map, Iterable> TEST_DATA = + new ImmutableMap.Builder, Iterable>() + .put(VarIntCoder.of(), + Arrays.asList(-1, 0, 1, 13, Integer.MAX_VALUE, Integer.MIN_VALUE)) + .put(BigEndianLongCoder.of(), + Arrays.asList(-1L, 0L, 1L, 13L, Long.MAX_VALUE, Long.MIN_VALUE)) + .put(StringUtf8Coder.of(), + Arrays.asList("", "hello", "goodbye", "1")) + .put(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()), + Arrays.asList(KV.of("", -1), KV.of("hello", 0), KV.of("goodbye", Integer.MAX_VALUE))) + .put(ListCoder.of(VarLongCoder.of()), + Arrays.asList( + Arrays.asList(1L, 2L, 3L), + Collections.emptyList())) + .build(); + + @Test + public void testDecodeEncodeEqual() throws Exception { + for (Map.Entry, Iterable> entry : TEST_DATA.entrySet()) { + // The coder and corresponding values must be the same type. + // If someone messes this up in the above test data, the test + // will fail anyhow (unless the coder magically works on data + // it does not understand). + @SuppressWarnings("unchecked") + Coder coder = (Coder) entry.getKey(); + Iterable values = entry.getValue(); + for (Object value : values) { + CoderProperties.coderDecodeEncodeEqual(coder, value); + } + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ListCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ListCoderTest.java index c04d3e16745b8..595cd7811b7a1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ListCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ListCoderTest.java @@ -24,13 +24,30 @@ import org.junit.runners.JUnit4; import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; import java.util.List; /** Unit tests for {@link ListCoder}. */ @RunWith(JUnit4.class) public class ListCoderTest { + + private static final List> TEST_VALUES = Arrays.>asList( + Collections.emptyList(), + Collections.singletonList(43), + Arrays.asList(1, 2, 3, 4), + new LinkedList(Arrays.asList(7, 6, 5))); + + @Test + public void testDecodeEncodeContentsInSameOrder() throws Exception { + Coder> coder = ListCoder.of(VarIntCoder.of()); + for (List value : TEST_VALUES) { + CoderProperties.>coderDecodeEncodeContentsInSameOrder(coder, value); + } + } + @Test - public void testGetInstanceComponentsNonempty() { + public void testGetInstanceComponentsNonempty() throws Exception { List list = Arrays.asList(21, 5, 3, 5); List components = ListCoder.getInstanceComponents(list); assertEquals(1, components.size()); @@ -38,9 +55,16 @@ public void testGetInstanceComponentsNonempty() { } @Test - public void testGetInstanceComponentsEmpty() { + public void testGetInstanceComponentsEmpty() throws Exception { List list = Arrays.asList(); List components = ListCoder.getInstanceComponents(list); assertNull(components); } + + @Test + public void testEmptyList() throws Exception { + List list = Collections.emptyList(); + Coder> coder = ListCoder.of(VarIntCoder.of()); + CoderProperties.>coderDecodeEncodeEqual(coder, list); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/MapCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/MapCoderTest.java index 30cd0d8e8100b..4132776c8ad01 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/MapCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/MapCoderTest.java @@ -19,10 +19,14 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import com.google.common.collect.ImmutableMap; + import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -30,6 +34,18 @@ /** Unit tests for {@link MapCoder}. */ @RunWith(JUnit4.class) public class MapCoderTest { + private static final List> TEST_VALUES = Arrays.>asList( + Collections.emptyMap(), + new ImmutableMap.Builder().put(1, "hello").put(-1, "foo").build()); + + @Test + public void testDecodeEncodeContentsInSameOrder() throws Exception { + Coder> coder = MapCoder.of(VarIntCoder.of(), StringUtf8Coder.of()); + for (Map value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeEqual(coder, value); + } + } + @Test public void testGetInstanceComponentsNonempty() { Map map = new HashMap<>(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/SetCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/SetCoderTest.java new file mode 100644 index 0000000000000..db5b42aa4b1d2 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/SetCoderTest.java @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Test case for {@link SetCoder}. + */ +@RunWith(JUnit4.class) +public class SetCoderTest { + + private static final List> TEST_VALUES = Arrays.>asList( + Collections.emptySet(), + Collections.singleton(13), + new HashSet<>(Arrays.asList(31, -5, 83))); + + @Test + public void testDecodeEncodeContentsEqual() throws Exception { + Coder> coder = SetCoder.of(VarIntCoder.of()); + for (Set value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeContentsEqual(coder, value); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/StringUtf8CoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/StringUtf8CoderTest.java new file mode 100644 index 0000000000000..9c85836e94324 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/StringUtf8CoderTest.java @@ -0,0 +1,45 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Test case for {@link StringUtf8Coder}. + */ +@RunWith(JUnit4.class) +public class StringUtf8CoderTest { + + private static final List TEST_VALUES = Arrays.asList( + "", "a", "13", "hello", + "a longer string with spaces and all that", + "a string with a \n newline", + "スタリング"); + + @Test + public void testDecodeEncodeEqual() throws Exception { + Coder coder = StringUtf8Coder.of(); + for (String value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeEqual(coder, value); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoderTest.java new file mode 100644 index 0000000000000..cd185539442a3 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoderTest.java @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.api.services.bigquery.model.TableRow; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Test case for {@link TableRowJsonCoder}. + */ +@RunWith(JUnit4.class) +public class TableRowJsonCoderTest { + + private static class TableRowBuilder { + private TableRow row; + public TableRowBuilder() { + row = new TableRow(); + } + public TableRowBuilder set(String fieldName, Object value) { + row.set(fieldName, value); + return this; + } + public TableRow build() { + return row; + } + } + + private static final List TEST_VALUES = Arrays.asList( + new TableRowBuilder().build(), + new TableRowBuilder().set("a", "1").build(), + new TableRowBuilder().set("b", 3.14).build(), + new TableRowBuilder().set("a", "1").set("b", true).set("c", "hi").build()); + + @Test + public void testDecodeEncodeEqual() throws Exception { + Coder coder = TableRowJsonCoder.of(); + for (TableRow value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeEqual(coder, value); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoderTest.java new file mode 100644 index 0000000000000..a88c4764ef1bc --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoderTest.java @@ -0,0 +1,44 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Test case for {@link TextualIntegerCoder}. + */ +@RunWith(JUnit4.class) +public class TextualIntegerCoderTest { + + private static final List TEST_VALUES = Arrays.asList( + -11, -3, -1, 0, 1, 5, 13, 29, + Integer.MAX_VALUE, + Integer.MIN_VALUE); + + @Test + public void testDecodeEncodeEqual() throws Exception { + Coder coder = TextualIntegerCoder.of(); + for (Integer value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeEqual(coder, value); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/URICoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/URICoderTest.java index f464e813bf5d4..dd0d32ec419b9 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/URICoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/URICoderTest.java @@ -28,8 +28,6 @@ @RunWith(JUnit4.class) public class URICoderTest { - // Test data - private static final List TEST_URI_STRINGS = Arrays.asList( "http://www.example.com", "gs://myproject/mybucket/a/gcs/path", @@ -37,32 +35,19 @@ public class URICoderTest { "file:/path/with/no/authority", "file:///path/with/empty/authority"); - private static final List TEST_CONTEXTS = Arrays.asList( - Coder.Context.OUTER, - Coder.Context.NESTED); - - // Tests - @Test public void testDeterministic() throws Exception { Coder coder = URICoder.of(); - for (String uriString : TEST_URI_STRINGS) { - for (Coder.Context context : TEST_CONTEXTS) { - // Obviously equal, but distinct as objects - CoderProperties.coderDeterministic(coder, context, new URI(uriString), new URI(uriString)); - } + CoderProperties.coderDeterministic(coder, new URI(uriString), new URI(uriString)); } } @Test public void testDecodeEncodeEqual() throws Exception { Coder coder = URICoder.of(); - for (String uriString : TEST_URI_STRINGS) { - for (Coder.Context context : TEST_CONTEXTS) { - CoderProperties.coderDecodeEncodeEqual(coder, context, new URI(uriString)); - } + CoderProperties.coderDecodeEncodeEqual(coder, new URI(uriString)); } } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/VarIntCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/VarIntCoderTest.java new file mode 100644 index 0000000000000..e0ccd1801638a --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/VarIntCoderTest.java @@ -0,0 +1,45 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Test case for {@link VarIntCoder}. + */ +@RunWith(JUnit4.class) +public class VarIntCoderTest { + + private static final List TEST_VALUES = Arrays.asList( + -11, -3, -1, 0, 1, 5, 13, 29, + Integer.MAX_VALUE, + Integer.MIN_VALUE); + + @Test + public void testDecodeEncodeEqual() throws Exception { + Coder coder = VarIntCoder.of(); + for (Integer value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeEqual(coder, value); + } + } +} + diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/VarLongCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/VarLongCoderTest.java new file mode 100644 index 0000000000000..bc85dac79f689 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/VarLongCoderTest.java @@ -0,0 +1,46 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Test case for {@link VarLongCoder}. + */ +@RunWith(JUnit4.class) +public class VarLongCoderTest { + + private static final List TEST_VALUES = Arrays.asList( + -11L, -3L, -1L, 0L, 1L, 5L, 13L, 29L, + Integer.MAX_VALUE + 131L, + Integer.MIN_VALUE - 29L, + Long.MAX_VALUE, + Long.MIN_VALUE); + + @Test + public void testDecodeEncodeEqual() throws Exception { + Coder coder = VarLongCoder.of(); + for (Long value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeEqual(coder, value); + } + } +} From 0e4feec9be53e489a6a80427f872edffe2879a9f Mon Sep 17 00:00:00 2001 From: vanya Date: Tue, 23 Dec 2014 16:16:11 -0800 Subject: [PATCH 0055/1541] Add TextIO.Write.{,.Bound}.withoutValidation() method and plumb the validate_sink value. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82750082 --- .../google/cloud/dataflow/sdk/io/TextIO.java | 58 ++++++++++++++++--- .../runners/dataflow/TextIOTranslator.java | 1 + .../dataflow/sdk/util/PropertyNames.java | 1 + .../cloud/dataflow/sdk/io/TextIOTest.java | 9 ++- 4 files changed, 59 insertions(+), 10 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java index 86512be491927..14fa60cf42e5c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java @@ -355,6 +355,18 @@ public static Bound withCoder(Coder coder) { return new Bound<>(coder); } + /** + * Returns a TextIO.Write PTransform that has GCS path validation on + * pipeline creation disabled. + * + *

This can be useful in the case where the GCS output location does + * not exist at the pipeline creation time, but is expected to be available + * at execution time. + */ + public static Bound withoutValidation() { + return new Bound<>(DEFAULT_TEXT_CODER).withoutValidation(); + } + // TODO: appendingNewlines, gzipped, header, footer, etc. /** @@ -382,18 +394,22 @@ public static class Bound extends PTransform, PDone> { /** Shard template string. */ final String shardTemplate; + /** An option to indicate if output validation is desired. Default is true. */ + final boolean validate; + Bound(Coder coder) { - this(null, null, "", coder, 0, ShardNameTemplate.INDEX_OF_MAX); + this(null, null, "", coder, 0, ShardNameTemplate.INDEX_OF_MAX, true); } Bound(String name, String filenamePrefix, String filenameSuffix, Coder coder, - int numShards, String shardTemplate) { + int numShards, String shardTemplate, boolean validate) { super(name); this.coder = coder; this.filenamePrefix = filenamePrefix; this.filenameSuffix = filenameSuffix; this.numShards = numShards; this.shardTemplate = shardTemplate; + this.validate = validate; } /** @@ -401,7 +417,8 @@ public static class Bound extends PTransform, PDone> { * with the given step name. Does not modify this object. */ public Bound named(String name) { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate); + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate, + validate); } /** @@ -414,7 +431,8 @@ public Bound named(String name) { */ public Bound to(String filenamePrefix) { validateOutputComponent(filenamePrefix); - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate); + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate, + validate); } /** @@ -427,7 +445,8 @@ public Bound to(String filenamePrefix) { */ public Bound withSuffix(String nameExtension) { validateOutputComponent(nameExtension); - return new Bound<>(name, filenamePrefix, nameExtension, coder, numShards, shardTemplate); + return new Bound<>(name, filenamePrefix, nameExtension, coder, numShards, shardTemplate, + validate); } /** @@ -446,7 +465,8 @@ public Bound withSuffix(String nameExtension) { */ public Bound withNumShards(int numShards) { Preconditions.checkArgument(numShards >= 0); - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate); + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate, + validate); } /** @@ -458,7 +478,8 @@ public Bound withNumShards(int numShards) { * @see ShardNameTemplate */ public Bound withShardNameTemplate(String shardTemplate) { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate); + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate, + validate); } /** @@ -471,7 +492,7 @@ public Bound withShardNameTemplate(String shardTemplate) { *

Does not modify this object. */ public Bound withoutSharding() { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, 1, ""); + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, 1, "", validate); } /** @@ -483,7 +504,22 @@ public Bound withoutSharding() { * @param the type of the elements of the input PCollection */ public Bound withCoder(Coder coder) { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate); + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate, + validate); + } + + /** + * Returns a new TextIO.Write PTransform that's like this one but + * that has GCS output path validation on pipeline creation disabled. + * Does not modify this object. + * + *

This can be useful in the case where the GCS output location does + * not exist at the pipeline creation time, but is expected to be + * available at execution time. + */ + public Bound withoutValidation() { + return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, shardTemplate, + false); } @Override @@ -532,6 +568,10 @@ public Coder getCoder() { return coder; } + public boolean needsValidation() { + return validate; + } + static { DirectPipelineRunner.registerDefaultTransformEvaluator( Bound.class, new DirectPipelineRunner.TransformEvaluator() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java index ee70949d5954c..92aaf210e4ec8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java @@ -119,6 +119,7 @@ private void translateWriteHelper( context.addInput(PropertyNames.SHARD_NAME_TEMPLATE, transform.getShardNameTemplate()); context.addInput(PropertyNames.FILENAME_SUFFIX, transform.getFilenameSuffix()); + context.addInput(PropertyNames.VALIDATE_SINK, transform.needsValidation()); long numShards = transform.getNumShards(); if (numShards > 0) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java index c10f07fd98ed2..c7a5a307f1965 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java @@ -83,6 +83,7 @@ public class PropertyNames { public static final String USER_FN = "user_fn"; public static final String USER_NAME = "user_name"; public static final String USES_KEYED_STATE = "uses_keyed_state"; + public static final String VALIDATE_SINK = "validate_sink"; public static final String VALIDATE_SOURCE = "validate_source"; public static final String VALUE = "value"; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java index 46bdd5944a969..16748cf566d17 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java @@ -415,9 +415,16 @@ public void testBadWildcardBrackets() throws Exception { } @Test - public void testWithoutValidationFlag() throws Exception { + public void testReadWithoutValidationFlag() throws Exception { TextIO.Read.Bound read = TextIO.Read.from("gs://bucket/foo*/baz"); assertTrue(read.needsValidation()); assertFalse(read.withoutValidation().needsValidation()); } + + @Test + public void testWriteWithoutValidationFlag() throws Exception { + TextIO.Write.Bound write = TextIO.Write.to("gs://bucket/foo/baz"); + assertTrue(write.needsValidation()); + assertFalse(write.withoutValidation().needsValidation()); + } } From 7325a9a1e73d5583e6e8000fb99aa6eec7c7cc49 Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 29 Dec 2014 05:51:00 -0800 Subject: [PATCH 0056/1541] In DataflowAssert.IterableAssert, use appropriate Coder instead of Java serialization for expected results. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82947656 --- .../dataflow/sdk/testing/DataflowAssert.java | 576 ++++++++++++------ .../sdk/testing/DataflowAssertTest.java | 227 +++++++ 2 files changed, 629 insertions(+), 174 deletions(-) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/DataflowAssertTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java index 3debc9ff4f034..3773aab5ea93e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java @@ -16,7 +16,10 @@ package com.google.cloud.dataflow.sdk.testing; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; @@ -24,25 +27,26 @@ import com.google.cloud.dataflow.sdk.transforms.View; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.common.base.Optional; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import java.io.Serializable; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import java.util.Arrays; import java.util.Collection; +import java.util.List; +import java.util.NoSuchElementException; /** * An assertion on the contents of a {@link PCollection} * incorporated into the pipeline. Such an assertion - * can be checked no matter what kind of - * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} is - * used, so it's good for testing using the - * {@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner}, - * the - * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}, - * etc. + * can be checked no matter what kind of {@link PipelineRunner} is + * used. * *

Note that the {@code DataflowAssert} call must precede the call - * to {@link com.google.cloud.dataflow.sdk.Pipeline#run}. + * to {@link Pipeline#run}. * *

Examples of use: *

{@code
@@ -65,253 +69,368 @@
  * }
* *

JUnit and Hamcrest must be linked in by any code that uses DataflowAssert. - * - * @param The type of elements in the input collection. */ -public class DataflowAssert { +public class DataflowAssert { + + // Do not instantiate. + private DataflowAssert() {} + /** - * Constructs an IterableAssert for the elements of the provided - * {@code PCollection}. + * Constructs an {@link IterableAssert} for the elements of the provided + * {@link PCollection PCollection}. */ - public static IterableAssert that(PCollection futureResult) { - return new IterableAssert<>(futureResult.apply(View.asIterable())); + public static IterableAssert that(PCollection actual) { + return new IterableAssert<>(actual.apply(View.asIterable())) + .setCoder(actual.getCoder()); } /** - * Constructs an IterableAssert for the value of the provided - * {@code PCollection>}, which must be a singleton. + * Constructs an {@link IterableAssert} for the value of the provided + * {@link PCollection PCollection>}, which must be a singleton. */ - public static IterableAssert thatSingletonIterable( - PCollection> futureResult) { - return new IterableAssert<>(futureResult.apply(View.>asSingleton())); + public static IterableAssert thatSingletonIterable(PCollection> actual) { + List> maybeElementCoder = actual.getCoder().getCoderArguments(); + Coder tCoder; + try { + tCoder = (Coder) Iterables.getOnlyElement(maybeElementCoder); + } catch (NoSuchElementException | IllegalArgumentException exc) { + throw new IllegalArgumentException( + "DataflowAssert.thatSingltonIterable requires a PCollection>" + + " with a Coder> where getCoderArguments() yields a" + + " single Coder to apply to the elements."); + } + + return new IterableAssert<>(actual.apply(View.>asSingleton())) + .setCoder(tCoder); } /** - * Constructs an IterableAssert for the value of the provided - * {@code PCollectionView, ?>}. + * Constructs an {@link IterableAssert} for the value of the provided + * {@code PCollectionView PCollectionView, ?>}. */ - public static IterableAssert thatIterable( - PCollectionView, ?> futureResult) { - return new IterableAssert<>(futureResult); + public static IterableAssert thatIterable(PCollectionView, ?> actual) { + return new IterableAssert<>(actual); } /** - * An assertion about the contents of a {@link PCollectionView<, ?>}. + * Constructs a {@link SingletonAssert} for the value of the provided + * {@code PCollection PCollection}, which must be a singleton. + */ + public static SingletonAssert thatSingleton(PCollection actual) { + return new SingletonAssert<>(actual.apply(View.asSingleton())) + .setCoder(actual.getCoder()); + } + + //////////////////////////////////////////////////////////// + + /** + * An assertion about the contents of a + * {@link PCollectionView PCollectionView<, ?>}. */ @SuppressWarnings("serial") public static class IterableAssert implements Serializable { - private final PCollectionView, ?> actualResults; - private IterableAssert(PCollectionView, ?> futureResult) { - actualResults = futureResult; + private final PCollectionView, ?> actualView; + private Optional> coder; + + protected IterableAssert(PCollectionView, ?> actualView) { + this.actualView = actualView; + coder = Optional.absent(); } /** - * Applies a SerializableFunction to check the elements of the Iterable. - * - *

Returns this IterableAssert. + * Sets the coder to use for elements of type {@code T}, as needed + * for internal purposes. */ - public IterableAssert satisfies( - final SerializableFunction, Void> checkerFn) { - - actualResults.getPipeline() - .apply(Create.of((Void) null)) - .setCoder(VoidCoder.of()) - .apply(ParDo - .withSideInputs(actualResults) - .of(new DoFn() { - @Override - public void processElement(ProcessContext c) { - Iterable actualContents = c.sideInput(actualResults); - checkerFn.apply(actualContents); - } - })); - + public IterableAssert setCoder(Coder coder) { + this.coder = Optional.of(coder); return this; } /** - * Checks that the Iterable contains the expected elements, in any - * order. + * Sets the coder to use for elements of type {@code T}, as needed + * for internal purposes. * - *

Returns this IterableAssert. + *

Returns this {@code IterableAssert}. */ - public IterableAssert containsInAnyOrder(T... expectedElements) { - return this.satisfies(new AssertContainsInAnyOrder(expectedElements)); + public IterableAssert setCoder(Optional> coder) { + this.coder = coder; + return this; } /** - * Checks that the Iterable contains the expected elements, in any - * order. - * - *

Returns this IterableAssert. + * Gets the coder, which may yet be absent. */ - public IterableAssert containsInAnyOrder( - Collection expectedElements) { - return this.satisfies(new AssertContainsInAnyOrder(expectedElements)); + public Coder getCoder() { + if (coder.isPresent()) { + return coder.get(); + } else { + throw new IllegalStateException( + "Attempting to access the coder of an IterableAssert" + + " which has not been set yet."); + } } /** - * Checks that the Iterable contains the expected elements, in the - * specified order. + * Applies a {@link SerializableFunction} to check the elements of the {@code Iterable}. * - *

Returns this IterableAssert. + *

Returns this {@code IterableAssert}. */ - public IterableAssert containsInOrder(T... expectedElements) { - return this.satisfies(new AssertContainsInOrder(expectedElements)); + public IterableAssert satisfies(SerializableFunction, Void> checkerFn) { + new OneSideInputAssert>(actualView).satisfies(checkerFn); + return this; } /** - * Checks that the Iterable contains the expected elements, in the - * specified order. + * Applies a {@link SerializableFunction} to check the elements of the {@code Iterable}. * - *

Returns this IterableAssert. + *

Returns this {@code IterableAssert}. */ - public IterableAssert containsInOrder(Collection expectedElements) { - return this.satisfies(new AssertContainsInOrder(expectedElements)); + public IterableAssert satisfies( + AssertRelation, Iterable> relation, + Iterable expectedElements) { + new TwoSideInputAssert, Iterable>(actualView, + actualView.getPipeline() + .apply(Create.of(expectedElements)) + .setOrdered(true) + .setCoder(getCoder()) + .apply(View.asIterable())) + .satisfies(relation); + return this; } /** - * SerializableFunction that performs an {@code Assert.assertThat()} - * operation using a {@code Matcher} operation that takes an array - * of elements. + * Checks that the {@code Iterable} contains the expected elements, in any + * order. + * + *

Returns this {@code IterableAssert}. */ - @SuppressWarnings("serial") - static class AssertThatIterable extends AssertThat, T[]> { - AssertThatIterable(T[] expected, - String matcherClassName, - String matcherFactoryMethodName) { - super(expected, Object[].class, - matcherClassName, matcherFactoryMethodName); - } + public IterableAssert containsInAnyOrder(Iterable expectedElements) { + return satisfies(new AssertContainsInAnyOrderRelation(), expectedElements); } /** - * SerializableFunction that verifies that an Iterable contains - * expected items in any order. + * Checks that the {@code Iterable} contains the expected elements, in any + * order. + * + *

Returns this {@code IterableAssert}. */ - @SuppressWarnings("serial") - static class AssertContainsInAnyOrder extends AssertThatIterable { - AssertContainsInAnyOrder(T... expected) { - super(expected, - "org.hamcrest.collection.IsIterableContainingInAnyOrder", - "containsInAnyOrder"); - } - @SuppressWarnings("unchecked") - AssertContainsInAnyOrder(Collection expected) { - this((T[]) expected.toArray()); - } - } + public IterableAssert containsInAnyOrder(T... expectedElements) { + return satisfies( + new AssertContainsInAnyOrderRelation(), + Arrays.asList(expectedElements)); + }; + /** - * SerializableFunction that verifies that an Iterable contains - * expected items in the provided order. + * Checks that the {@code Iterable} contains the expected elements, in the + * specified order. + * + *

Returns this {@code IterableAssert}. */ - @SuppressWarnings("serial") - static class AssertContainsInOrder extends AssertThatIterable { - AssertContainsInOrder(T... expected) { - super(expected, - "org.hamcrest.collection.IsIterableContainingInOrder", - "contains"); - } - @SuppressWarnings("unchecked") - AssertContainsInOrder(Collection expected) { - this((T[]) expected.toArray()); - } + public IterableAssert containsInOrder(T... expectedElements) { + return this.satisfies( + new AssertContainsInOrderRelation(), + Arrays.asList(expectedElements)); } - } - - ///////////////////////////////////////////////////////////////////////////// - /** - * Constructs a SingletonAssert for the value of the provided - * {@code PCollection}, which must be a singleton. - */ - public static SingletonAssert thatSingleton(PCollection futureResult) { - return new SingletonAssert<>(futureResult.apply(View.asSingleton())); + /** + * Checks that the {@code Iterable} contains the expected elements, in the + * specified order. + * + *

Returns this {@code IterableAssert}. + */ + public IterableAssert containsInOrder(Iterable expectedElements) { + return this.satisfies(new AssertContainsInOrderRelation(), expectedElements); + } } /** - * An assertion about a single value. + * An assertion about the single value of type {@code T} + * associated with a {@link PCollectionView PCollectionView}. */ @SuppressWarnings("serial") public static class SingletonAssert implements Serializable { - private final PCollectionView actualResult; - private SingletonAssert(PCollectionView futureResult) { - actualResult = futureResult; + private final PCollectionView actualView; + private Optional> coder; + + protected SingletonAssert(PCollectionView actualView) { + this.actualView = actualView; + coder = Optional.absent(); } /** - * Applies a SerializableFunction to check the value of this - * SingletonAssert's view. - * - *

Returns this SingletonAssert. + * Sets the coder to use for elements of type {@code T}, as needed + * for internal purposes. */ - public SingletonAssert satisfies(final SerializableFunction checkerFn) { - actualResult.getPipeline() - .apply(Create.of((Void) null)) - .setCoder(VoidCoder.of()) - .apply(ParDo - .withSideInputs(actualResult) - .of(new DoFn() { - @Override - public void processElement(ProcessContext c) { - T actualContents = c.sideInput(actualResult); - checkerFn.apply(actualContents); - } - })); - + public SingletonAssert setCoder(Coder coder) { + this.coder = Optional.of(coder); return this; } /** - * Checks that the value of this SingletonAssert's view is equal - * to the expected value. + * Sets the coder to use for elements of type {@code T}, as needed + * for internal purposes. * - *

Returns this SingletonAssert. + *

Returns this {@code SingletonAssert}. */ - public SingletonAssert is(T expectedValue) { - return this.satisfies(new AssertIs(expectedValue)); + public SingletonAssert setCoder(Optional> coder) { + this.coder = coder; + return this; } /** - * SerializableFunction that performs an {@code Assert.assertThat()} - * operation using a {@code Matcher} operation that takes a single element. + * Gets the coder, which may yet be absent. */ - @SuppressWarnings("serial") - static class AssertThatValue extends AssertThat { - AssertThatValue(T expected, - String matcherClassName, - String matcherFactoryMethodName) { - super(expected, Object.class, - matcherClassName, matcherFactoryMethodName); + public Coder getCoder() { + if (coder.isPresent()) { + return coder.get(); + } else { + throw new IllegalStateException( + "Attempting to access the coder of a SingletonAssert" + + " which has not been set yet."); } } /** - * SerializableFunction that verifies that a value is equal to an - * expected value. + * Applies a {@link SerializableFunction} to check the value of this + * {@code SingletonAssert}'s view. + * + *

Returns this {@code SingletonAssert}. + */ + public SingletonAssert satisfies(final SerializableFunction checkerFn) { + new OneSideInputAssert(actualView).satisfies(checkerFn); + return this; + } + + /** + * Applies an {@link AssertRelation} to check the provided relation against the + * value of this assert and the provided expected value. + * + *

Returns this {@code SingletonAssert}. */ - @SuppressWarnings("serial") - public static class AssertIs extends AssertThatValue { - AssertIs(T expected) { - super(expected, "org.hamcrest.core.IsEqual", "equalTo"); - } + public SingletonAssert satisfies( + AssertRelation relation, + T expectedValue) { + new TwoSideInputAssert(actualView, + actualView.getPipeline() + .apply(Create.of(expectedValue)) + .setOrdered(true) + .setCoder(getCoder()) + .apply(View.asSingleton())) + .satisfies(relation); + return this; + } + + + /** + * Checks that the value of this {@code SingletonAssert}'s view is equal + * to the expected value. + * + *

Returns this {@code SingletonAssert}. + */ + public SingletonAssert isEqualTo(T expectedValue) { + return satisfies(new AssertIsEqualToRelation(), expectedValue); + } + + @Deprecated + public SingletonAssert is(T expectedValue) { + return isEqualTo(expectedValue); + } + + } + + //////////////////////////////////////////////////////////////////////// + + /** + * An assertion checker that takes a single {@link PCollectionView PCollectionView} + * and an assertion over {@code A}, and checks it within a dataflow pipeline. + * + *

Note that the entire assertion must be serializable. If + * you need to make assertions involving multiple inputs + * that are each not serializable, use TwoSideInputAssert. + * + *

This is generally useful for assertion functions that + * are serializable but whose underlying data may not have a coder. + */ + @SuppressWarnings("serial") + private static class OneSideInputAssert implements Serializable { + + private final PCollectionView actualView; + + public OneSideInputAssert(PCollectionView actualView) { + this.actualView = actualView; + } + + public OneSideInputAssert satisfies( + final SerializableFunction checkerFn) { + actualView.getPipeline() + .apply(Create.of((Void) null)) + .setCoder(VoidCoder.of()) + .apply(ParDo + .withSideInputs(actualView) + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + Actual actualContents = c.sideInput(actualView); + checkerFn.apply(actualContents); + } + })); + return this; } } - ///////////////////////////////////////////////////////////////////////////// + /** + * An assertion checker that takes a {@link PCollectionView PCollectionView}, + * a {@link PCollectionView PCollectionView}, a relation + * over {@code A} and {@code B}, and checks that the relation holds + * within a dataflow pipeline. + * + *

This is useful when either/both of {@code A} and {@code B} + * are not serializable, but have coders (provided + * by the underlying {@link PCollection}s). + */ + @SuppressWarnings("serial") + private static class TwoSideInputAssert implements Serializable { + private final PCollectionView actualView; + private final PCollectionView expectedView; - // Do not instantiate. - private DataflowAssert() {} + protected TwoSideInputAssert( + PCollectionView actualView, + PCollectionView expectedView) { + this.actualView = actualView; + this.expectedView = expectedView; + } + + public TwoSideInputAssert satisfies( + final AssertRelation relation) { + actualView.getPipeline() + .apply(Create.of((Void) null)) + .setCoder(VoidCoder.of()) + .apply(ParDo + .withSideInputs(actualView, expectedView) + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + Actual actualContents = c.sideInput(actualView); + Expected expectedContents = c.sideInput(expectedView); + relation.assertFor(expectedContents).apply(actualContents); + } + })); + return this; + } + } + + ///////////////////////////////////////////////////////////////////////////// /** - * SerializableFunction that performs an {@code Assert.assertThat()} - * operation using a {@code Matcher} operation. + * A {@link SerializableFunction} that performs an + * {@code Assert.assertThat()} operation using a + * {@code Matcher} operation. * - *

The MatcherFactory should take an {@code Expected} and + *

The {@code MatcherFactory} should take an {@code Expected} and * produce a Matcher to be used to check an {@code Actual} value * against. */ @@ -319,12 +438,12 @@ private DataflowAssert() {} public static class AssertThat implements SerializableFunction { final Expected expected; - final Class expectedClass; + final Class expectedClass; final String matcherClassName; final String matcherFactoryMethodName; AssertThat(Expected expected, - Class expectedClass, + Class expectedClass, String matcherClassName, String matcherFactoryMethodName) { this.expected = expected; @@ -358,11 +477,10 @@ public Void apply(Actual in) { } /** - * SerializableFunction that performs an {@code Assert.assertThat()} - * operation using a {@code Matcher} operation that takes a single element. + * An {@link AssertThat} taking a single element. */ @SuppressWarnings("serial") - static class AssertThatValue extends AssertThat { + private static class AssertThatValue extends AssertThat { AssertThatValue(T expected, String matcherClassName, String matcherFactoryMethodName) { @@ -372,13 +490,123 @@ static class AssertThatValue extends AssertThat { } /** - * SerializableFunction that verifies that a value is equal to an + * An {@link AssertThatValue} that verifies that an actual value is equal to an * expected value. */ @SuppressWarnings("serial") - public static class AssertIs extends AssertThatValue { - public AssertIs(T expected) { + private static class AssertIsEqualTo extends AssertThatValue { + public AssertIsEqualTo(T expected) { super(expected, "org.hamcrest.core.IsEqual", "equalTo"); } } + + /** + * An {@link AssertThat} that operates on an {@code Iterable}. The + * underlying matcher takes a {@code T[]} of expected values, for + * compatibility with the corresponding Hamcrest {@code Matcher}s. + */ + @SuppressWarnings("serial") + private static class AssertThatIterable extends AssertThat, T[]> { + AssertThatIterable(T[] expected, + String matcherClassName, + String matcherFactoryMethodName) { + super(expected, Object[].class, + matcherClassName, matcherFactoryMethodName); + } + } + + /** + * An {@link AssertThatIterable} that verifies that an {@code Iterable} contains + * expected items in any order. + */ + @SuppressWarnings("serial") + private static class AssertContainsInAnyOrder extends AssertThatIterable { + public AssertContainsInAnyOrder(T... expected) { + super(expected, + "org.hamcrest.collection.IsIterableContainingInAnyOrder", + "containsInAnyOrder"); + } + + @SuppressWarnings("unchecked") + public AssertContainsInAnyOrder(Collection expected) { + this((T[]) expected.toArray()); + } + + @SuppressWarnings("unchecked") + public AssertContainsInAnyOrder(Iterable expected) { + this(Lists.newArrayList(expected)); + } + } + + /** + * An {@link AssertThatIterable} that verifies that an {@code Iterable} contains + * the expected items in the provided order. + */ + @SuppressWarnings("serial") + private static class AssertContainsInOrder extends AssertThatIterable { + public AssertContainsInOrder(T... expected) { + super(expected, + "org.hamcrest.collection.IsIterableContainingInOrder", + "contains"); + } + + @SuppressWarnings("unchecked") + public AssertContainsInOrder(Collection expected) { + this((T[]) expected.toArray()); + } + + @SuppressWarnings("unchecked") + public AssertContainsInOrder(Iterable expected) { + this(Lists.newArrayList(expected)); + } + } + + //////////////////////////////////////////////////////////// + + /** + * A serializable function implementing a binary predicate + * between types {@code Actual} and {@code Expected}. + */ + public static interface AssertRelation extends Serializable { + public SerializableFunction assertFor(Expected input); + } + + /** + * An {@link AssertRelation} implementing the binary predicate + * that two objects are equal. + */ + private static class AssertIsEqualToRelation + implements AssertRelation { + + @Override + public AssertThat assertFor(T expected) { + return new AssertIsEqualTo(expected); + } + } + + /** + * An {@code AssertRelation} implementing the binary predicate + * that two collections are equal modulo reordering. + */ + private static class AssertContainsInAnyOrderRelation + implements AssertRelation, Iterable> { + + @Override + public SerializableFunction, Void> assertFor(Iterable expectedElements) { + return new AssertContainsInAnyOrder(expectedElements); + } + } + + /** + * A {@code AssertRelation} implementating the binary function + * that two iterables have equal contents, in the same order. + */ + private static class AssertContainsInOrderRelation + implements AssertRelation, Iterable> { + + @Override + public SerializableFunction, Void> assertFor(Iterable expectedElements) { + return new AssertContainsInOrder(expectedElements); + } + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/DataflowAssertTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/DataflowAssertTest.java new file mode 100644 index 0000000000000..11a6384245591 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/DataflowAssertTest.java @@ -0,0 +1,227 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * Test case for {@link DataflowAssert}. + */ +@RunWith(JUnit4.class) +public class DataflowAssertTest implements Serializable { + + @Rule + public transient ExpectedException thrown = ExpectedException.none(); + + private static class NotSerializableObject { + + @Override + public boolean equals(Object other) { + return (other instanceof NotSerializableObject); + } + + @Override + public int hashCode() { + return 73; + } + } + + private static class NotSerializableObjectCoder extends AtomicCoder { + private NotSerializableObjectCoder() { } + private static final NotSerializableObjectCoder INSTANCE = new NotSerializableObjectCoder(); + + @JsonCreator + public static NotSerializableObjectCoder of() { + return INSTANCE; + } + + @Override + public void encode(NotSerializableObject value, OutputStream outStream, Context context) + throws CoderException, IOException { + } + + @Override + public NotSerializableObject decode(InputStream inStream, Context context) + throws CoderException, IOException { + return new NotSerializableObject(); + } + + @Override + public boolean isDeterministic() { + return true; + } + + @Override + public boolean isRegisterByteSizeObserverCheap(NotSerializableObject value, Context context) { + return true; + } + + @Override + public void registerByteSizeObserver( + NotSerializableObject value, ElementByteSizeObserver observer, Context context) + throws Exception { + observer.update(0L); + } + } + + /** + * A {@link DataflowAssert} about the contents of a {@link PCollection} + * must not require the contents of the {@link PCollection} to be + * serializable. + */ + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testContainsInAnyOrderNotSerializable() throws Exception { + Pipeline pipeline = TestPipeline.create(); + + PCollection pcollection = pipeline + .apply(Create.of( + new NotSerializableObject(), + new NotSerializableObject())) + .setCoder(NotSerializableObjectCoder.of()); + + DataflowAssert.that(pcollection).containsInAnyOrder( + new NotSerializableObject(), + new NotSerializableObject()); + + pipeline.run(); + } + + /** + * A {@link DataflowAssert} about the contents of a {@link PCollection} + * is allows to be verified by an arbitrary {@link SerializableFunction}, + * though. + */ + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testSerializablePredicate() throws Exception { + Pipeline pipeline = TestPipeline.create(); + + PCollection pcollection = pipeline + .apply(Create.of( + new NotSerializableObject(), + new NotSerializableObject())) + .setCoder(NotSerializableObjectCoder.of()); + + DataflowAssert.that(pcollection).satisfies( + new SerializableFunction, Void>() { + @Override + public Void apply(Iterable contents) { + return (Void) null; // no problem! + } + }); + + pipeline.run(); + } + + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testIsEqualTo() throws Exception { + Pipeline pipeline = TestPipeline.create(); + + PCollection pcollection = pipeline + .apply(Create.of(43)); + + DataflowAssert.thatSingleton(pcollection).isEqualTo(43); + + pipeline.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testContainsInAnyOrder() throws Exception { + Pipeline pipeline = TestPipeline.create(); + + PCollection pcollection = pipeline + .apply(Create.of(1, 2, 3, 4)); + + DataflowAssert.that(pcollection).containsInAnyOrder(2, 1, 4, 3); + + pipeline.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testContainsInAnyOrderFalse() throws Exception { + // The actual AssertionError is deep in the stack + // TODO: dig it out + thrown.expect(RuntimeException.class); + + Pipeline pipeline = TestPipeline.create(); + + PCollection pcollection = pipeline + .apply(Create.of(1, 2, 3, 4)); + + DataflowAssert.that(pcollection).containsInAnyOrder(2, 1, 4, 3, 7); + + pipeline.run(); + } + + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testContainsInOrder() throws Exception { + Pipeline pipeline = TestPipeline.create(); + + PCollection pcollection = pipeline + .apply(Create.of(1, 2, 3, 4)) + .setOrdered(true); + + DataflowAssert.that(pcollection).containsInOrder(1, 2, 3, 4); + + pipeline.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testContainsInOrderFalse() throws Exception { + // The actual AssertionError is deep in the stack + // TODO: dig it out + thrown.expect(RuntimeException.class); + + Pipeline pipeline = TestPipeline.create(); + + PCollection pcollection = pipeline + .apply(Create.of(1, 2, 3, 4)) + .setOrdered(true); + + DataflowAssert.that(pcollection).containsInOrder(1, 2, 4, 3); + + pipeline.run(); + } +} From 02cc89f5a2fe65380e7bdfbc22f42f29743e8f33 Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 29 Dec 2014 11:46:32 -0800 Subject: [PATCH 0057/1541] Add DelegateCoder and StringDelegateCoder. Remove URICoder (now trivial). [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82962937 --- .../google/cloud/dataflow/examples/TfIdf.java | 5 +- .../cloud/dataflow/examples/TfIdfTest.java | 3 + .../dataflow/sdk/coders/CoderRegistry.java | 2 - .../dataflow/sdk/coders/DelegateCoder.java | 103 ++++++++++++++++++ .../sdk/coders/StringDelegateCoder.java | 72 ++++++++++++ .../cloud/dataflow/sdk/coders/URICoder.java | 78 ------------- .../dataflow/sdk/coders/CoderProperties.java | 5 + .../sdk/coders/DelegateCoderTest.java | 73 +++++++++++++ ...Test.java => StringDelegateCoderTest.java} | 33 ++++-- 9 files changed, 284 insertions(+), 90 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoder.java delete mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/URICoder.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/DelegateCoderTest.java rename sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/{URICoderTest.java => StringDelegateCoderTest.java} (60%) diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java b/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java index 3f22e27eab264..0349f3614f3be 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java @@ -19,8 +19,8 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; -import com.google.cloud.dataflow.sdk.coders.URICoder; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.options.Default; import com.google.cloud.dataflow.sdk.options.Description; @@ -153,7 +153,7 @@ public ReadDocuments(Iterable uris) { @Override public Coder getDefaultOutputCoder() { - return KvCoder.of(URICoder.of(), StringUtf8Coder.of()); + return KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of()); } @Override @@ -397,6 +397,7 @@ public void processElement(ProcessContext c) { public static void main(String[] args) throws Exception { Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); Pipeline pipeline = Pipeline.create(options); + pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); pipeline .apply(new ReadDocuments(listInputDocuments(options))) diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/TfIdfTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/TfIdfTest.java index 990458f5e059c..b3a115b522362 100644 --- a/examples/src/test/java/com/google/cloud/dataflow/examples/TfIdfTest.java +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/TfIdfTest.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.examples; import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.Create; @@ -45,6 +46,8 @@ public class TfIdfTest { public void testTfIdf() throws Exception { Pipeline pipeline = TestPipeline.create(); + pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); + PCollection>> wordToUriAndTfIdf = pipeline .apply(Create.of( KV.of(new URI("x"), "a b c d"), diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java index 670b4e3e320af..d2b61293c1dc6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java @@ -35,7 +35,6 @@ import java.lang.reflect.Type; import java.lang.reflect.TypeVariable; import java.lang.reflect.WildcardType; -import java.net.URI; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -116,7 +115,6 @@ public void registerStandardCoders() { registerCoder(TableRow.class, TableRowJsonCoder.class); registerCoder(Void.class, VoidCoder.class); registerCoder(byte[].class, ByteArrayCoder.class); - registerCoder(URI.class, URICoder.class); registerCoder(TimestampedValue.class, TimestampedValue.TimestampedValueCoder.class); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java new file mode 100644 index 0000000000000..9cc75167872fd --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java @@ -0,0 +1,103 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * A {@code DelegateCoder} wraps a {@link Coder Coder

} and + * encodes/decodes values of type {@code T}s by converting + * to/from {@code DT} and then encoding/decoding using the underlying + * {@link Coder Coder
}. + * + *

The conversions from {@code T} to {@code DT} and vice versa + * must be supplied as {@link CodingFunction}, a serializable + * function which may throw any {@code Exception}. If a thrown + * exception is an instance of {@link CoderException} or + * {@link IOException}, it will be re-thrown, otherwise it will be wrapped as + * a {@link CoderException}. + * + * @param The type of objects coded by this Coder. + * @param

The type of objects a {@code T} will be converted to for coding. + */ +public class DelegateCoder extends CustomCoder { + + /** + * A {@code CodingFunction} is a serializable function + * from {@code Input} to {@code Output} that + * may throw any {@code Exception}. + */ + public static interface CodingFunction extends Serializable { + public abstract Output apply(Input input) throws Exception; + } + + public static DelegateCoder of(Coder
coder, + CodingFunction toFn, + CodingFunction fromFn) { + return new DelegateCoder(coder, toFn, fromFn); + } + + @Override + public void encode(T value, OutputStream outStream, Context context) + throws CoderException, IOException { + coder.encode(applyAndWrapExceptions(toFn, value), outStream, context); + } + + @Override + public T decode(InputStream inStream, Context context) throws CoderException, IOException { + return applyAndWrapExceptions(fromFn, coder.decode(inStream, context)); + } + + @Override + public boolean isDeterministic() { + return coder.isDeterministic(); + } + + @Override + public String toString() { + return "DelegateCoder(" + coder + ")"; + } + + ///////////////////////////////////////////////////////////////////////////// + + private Output applyAndWrapExceptions( + CodingFunction fn, + Input input) throws CoderException, IOException { + try { + return fn.apply(input); + } catch (IOException exc) { + throw exc; + } catch (Exception exc) { + throw new CoderException(exc); + } + } + + private final Coder
coder; + private final CodingFunction toFn; + private final CodingFunction fromFn; + + protected DelegateCoder(Coder
coder, + CodingFunction toFn, + CodingFunction fromFn) { + this.coder = coder; + this.fromFn = fromFn; + this.toFn = toFn; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoder.java new file mode 100644 index 0000000000000..82fe806e43205 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoder.java @@ -0,0 +1,72 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import java.lang.reflect.InvocationTargetException; + +/** + * A {@code StringDelegateCoder} wraps a {@link Coder} + * and encodes/decodes values of type {@code T} via string representations. + * + *

To decode, the input byte stream is decoded to + * a {@code String}, and this is passed to the single-arg + * constructor for {@code T}. + * + *

To encode, the input value is converted via {@code toString()}, + * and this string is encoded. + * + *

In order for this to operate correctly for a class {@code Clazz}, + * it must be the case for any instance {@code x} that + * {@code x.equals(new Clazz(x.toString()))}. + * + * @param The type of objects coded. + */ +public class StringDelegateCoder extends DelegateCoder { + + public static StringDelegateCoder of(Class clazz) { + return new StringDelegateCoder(clazz); + } + + @Override + public String toString() { + return "StringDelegateCoder(" + clazz + ")"; + } + + private final Class clazz; + + protected StringDelegateCoder(final Class clazz) { + super(StringUtf8Coder.of(), + new CodingFunction() { + @Override + public String apply(T input) { + return input.toString(); + } + }, + new CodingFunction() { + @Override + public T apply(String input) throws + NoSuchMethodException, + InstantiationException, + IllegalAccessException, + InvocationTargetException { + return clazz.getConstructor(String.class).newInstance(input); + } + }); + + this.clazz = clazz; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/URICoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/URICoder.java deleted file mode 100644 index eedcddf787e10..0000000000000 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/URICoder.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Copyright (C) 2014 Google Inc. - * - * Licensed 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 com.google.cloud.dataflow.sdk.coders; - -import com.fasterxml.jackson.annotation.JsonCreator; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.net.URI; -import java.net.URISyntaxException; - -/** - * A {@code URICoder} encodes/decodes {@link URI}s by conversion to/from {@link String}, delegating - * encoding/decoding of the string to {@link StringUtf8Coder}. - */ -@SuppressWarnings("serial") -public class URICoder extends AtomicCoder { - - @JsonCreator - public static URICoder of() { - return INSTANCE; - } - - private static final URICoder INSTANCE = new URICoder(); - private static final StringUtf8Coder STRING_CODER = StringUtf8Coder.of(); - - private URICoder() {} - - ///////////////////////////////////////////////////////////////////////////// - - @Override - public void encode(URI value, OutputStream outStream, Context context) - throws IOException { - if (value == null) { - throw new CoderException("cannot encode a null URI"); - } - STRING_CODER.encode(value.toString(), outStream, context); - } - - @Override - public URI decode(InputStream inStream, Context context) - throws IOException { - try { - return new URI(STRING_CODER.decode(inStream, context)); - } catch (URISyntaxException exn) { - throw new CoderException(exn); - } - } - - @Override - public boolean isDeterministic() { - return STRING_CODER.isDeterministic(); - } - - @Override - protected long getEncodedElementByteSize(URI value, Context context) - throws Exception { - if (value == null) { - throw new CoderException("cannot encode a null URI"); - } - return STRING_CODER.getEncodedElementByteSize(value.toString(), context); - } -} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java index 89754789cc17f..5fbaf1f5a59e6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assume.assumeThat; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; import com.google.common.collect.Iterables; import java.io.ByteArrayInputStream; @@ -159,6 +160,10 @@ public static > void coderDecodeEncodeContentsInSameOr } } + public static void coderSerializable(Coder coder) { + SerializableUtils.ensureSerializable(coder); + } + ////////////////////////////////////////////////////////////////////////// private static byte[] encode( diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/DelegateCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/DelegateCoderTest.java new file mode 100644 index 0000000000000..e6f5cb83b5016 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/DelegateCoderTest.java @@ -0,0 +1,73 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** Unit tests for {@link DelegateCoder}. */ +@RunWith(JUnit4.class) +public class DelegateCoderTest { + + private static final List> TEST_VALUES = Arrays.>asList( + Collections.emptySet(), + Collections.singleton(13), + new HashSet<>(Arrays.asList(31, -5, 83))); + + private static final Coder> coder = DelegateCoder.of( + ListCoder.of(VarIntCoder.of()), + new DelegateCoder.CodingFunction, List>() { + public List apply(Set input) { + return Lists.newArrayList(input); + } + }, + new DelegateCoder.CodingFunction, Set>() { + public Set apply(List input) { + return Sets.newHashSet(input); + } + }); + + @Test + public void testDeterministic() throws Exception { + for (Set value : TEST_VALUES) { + CoderProperties.coderDeterministic( + coder, value, Sets.newHashSet(value)); + } + } + + @Test + public void testDecodeEncodeEqual() throws Exception { + for (Set value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeEqual(coder, value); + } + } + + @Test + public void testSerializable() throws Exception { + CoderProperties.coderSerializable(coder); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/URICoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoderTest.java similarity index 60% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/URICoderTest.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoderTest.java index dd0d32ec419b9..f930dfafbda68 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/URICoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoderTest.java @@ -16,6 +16,9 @@ package com.google.cloud.dataflow.sdk.coders; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -24,30 +27,44 @@ import java.util.Arrays; import java.util.List; -/** Unit tests for {@link URICoder}. */ +/** Unit tests for {@link StringDelegateCoder}. */ @RunWith(JUnit4.class) -public class URICoderTest { +public class StringDelegateCoderTest { + + // Test data + + private static final Coder uriCoder = StringDelegateCoder.of(URI.class); private static final List TEST_URI_STRINGS = Arrays.asList( "http://www.example.com", - "gs://myproject/mybucket/a/gcs/path", - "/just/a/path", + "gs://myproject/mybucket/some/gcs/path", + "/just/some/path", "file:/path/with/no/authority", "file:///path/with/empty/authority"); + // Tests + + private static final List TEST_CONTEXTS = Arrays.asList( + Coder.Context.NESTED, + Coder.Context.OUTER); + @Test public void testDeterministic() throws Exception { - Coder coder = URICoder.of(); + assertThat(uriCoder.isDeterministic(), equalTo(true)); for (String uriString : TEST_URI_STRINGS) { - CoderProperties.coderDeterministic(coder, new URI(uriString), new URI(uriString)); + CoderProperties.coderDeterministic(uriCoder, new URI(uriString), new URI(uriString)); } } @Test public void testDecodeEncodeEqual() throws Exception { - Coder coder = URICoder.of(); for (String uriString : TEST_URI_STRINGS) { - CoderProperties.coderDecodeEncodeEqual(coder, new URI(uriString)); + CoderProperties.coderDecodeEncodeEqual(uriCoder, new URI(uriString)); } } + + @Test + public void testSerializable() throws Exception { + CoderProperties.coderSerializable(uriCoder); + } } From c2a39f9d6c9342cae89b377398ee102b996e035f Mon Sep 17 00:00:00 2001 From: vanya Date: Mon, 29 Dec 2014 13:10:33 -0800 Subject: [PATCH 0058/1541] Clean up unused variables in TextIOTest. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=82967222 --- .../google/cloud/dataflow/sdk/io/TextIOTest.java | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java index 16748cf566d17..28270949c328a 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java @@ -228,9 +228,9 @@ void runTestWrite(T[] elems, Coder coder) throws Exception { write = TextIO.Write.to(filename).withCoder(coder).withoutSharding(); } - PDone output = input.apply(write); + input.apply(write); - EvaluationResults results = p.run(); + p.run(); BufferedReader reader = new BufferedReader(new FileReader(tmpFile)); List actual = new ArrayList<>(); @@ -285,10 +285,9 @@ public void testWriteSharded() throws IOException { p.apply(Create.of(Arrays.asList(LINES_ARRAY))) .setCoder(StringUtf8Coder.of()); - PDone done = input.apply( - TextIO.Write.to(filename).withNumShards(2).withSuffix(".txt")); + input.apply(TextIO.Write.to(filename).withNumShards(2).withSuffix(".txt")); - EvaluationResults results = p.run(); + p.run(); String[] files = outFolder.list(); @@ -334,9 +333,9 @@ public void testUnsupportedFilePattern() throws IOException { p.apply(Create.of(Arrays.asList(LINES_ARRAY))) .setCoder(StringUtf8Coder.of()); - PDone done = input.apply(TextIO.Write.to(filename)); + input.apply(TextIO.Write.to(filename)); - EvaluationResults results = p.run(); + p.run(); Assert.fail("Expected failure due to unsupported output pattern"); } From 0cbe51924de7243de0418a20ce3ecd5e85da2369 Mon Sep 17 00:00:00 2001 From: robertwb Date: Mon, 5 Jan 2015 17:55:00 -0800 Subject: [PATCH 0059/1541] Take bucket into account when running the PGBK(-and-bucket) operation. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83305116 --- .../worker/MapTaskExecutorFactory.java | 59 +++++++++++----- .../PartialGroupByKeyOperationTest.java | 69 ++++++++++--------- 2 files changed, 77 insertions(+), 51 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java index dd0133e7b3763..b1c4f6a63a8d5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java @@ -44,6 +44,7 @@ import com.google.cloud.dataflow.sdk.util.common.worker.ParDoFn; import com.google.cloud.dataflow.sdk.util.common.worker.ParDoOperation; import com.google.cloud.dataflow.sdk.util.common.worker.PartialGroupByKeyOperation; +import com.google.cloud.dataflow.sdk.util.common.worker.PartialGroupByKeyOperation.GroupingKeyCreator; import com.google.cloud.dataflow.sdk.util.common.worker.ReadOperation; import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.cloud.dataflow.sdk.util.common.worker.ReceivingOperation; @@ -52,6 +53,8 @@ import com.google.cloud.dataflow.sdk.util.common.worker.WriteOperation; import com.google.cloud.dataflow.sdk.values.KV; +import org.joda.time.Instant; + import java.util.ArrayList; import java.util.List; @@ -168,26 +171,27 @@ static PartialGroupByKeyOperation createPartialGroupByKeyOperation(PipelineOptio CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { PartialGroupByKeyInstruction pgbk = instruction.getPartialGroupByKey(); - Coder coder = Serializer.deserialize(pgbk.getInputElementCodec(), Coder.class); - if (!(coder instanceof WindowedValueCoder)) { + Coder windowedCoder = Serializer.deserialize(pgbk.getInputElementCodec(), Coder.class); + if (!(windowedCoder instanceof WindowedValueCoder)) { throw new Exception( - "unexpected kind of input coder for PartialGroupByKeyOperation: " + coder); + "unexpected kind of input coder for PartialGroupByKeyOperation: " + windowedCoder); } - Coder elemCoder = ((WindowedValueCoder) coder).getValueCoder(); + Coder elemCoder = ((WindowedValueCoder) windowedCoder).getValueCoder(); if (!(elemCoder instanceof KvCoder)) { throw new Exception( "unexpected kind of input element coder for PartialGroupByKeyOperation: " + elemCoder); } - KvCoder kvCoder = (KvCoder) elemCoder; - Coder keyCoder = kvCoder.getKeyCoder(); - Coder valueCoder = kvCoder.getValueCoder(); + KvCoder kvCoder = (KvCoder) elemCoder; + Coder keyCoder = kvCoder.getKeyCoder(); + Coder valueCoder = kvCoder.getValueCoder(); OutputReceiver[] receivers = createOutputReceivers(instruction, counterPrefix, addCounterMutator, stateSampler, 1); PartialGroupByKeyOperation operation = new PartialGroupByKeyOperation(instruction.getSystemName(), - new CoderGroupingKeyCreator(keyCoder), new CoderSizeEstimator(keyCoder), + new WindowingCoderGroupingKeyCreator(keyCoder), + new CoderSizeEstimator(WindowedValue.getValueOnlyCoder(keyCoder)), new CoderSizeEstimator(valueCoder), 0.001/*sizeEstimatorSampleRate*/, PairInfo.create(), receivers, counterPrefix, addCounterMutator, stateSampler); @@ -207,35 +211,52 @@ public static PairInfo create() { private PairInfo() {} @Override public Object getKeyFromInputPair(Object pair) { - WindowedValue> windowedKv = (WindowedValue>) pair; - return windowedKv.getValue().getKey(); + @SuppressWarnings("unchecked") + WindowedValue> windowedKv = (WindowedValue>) pair; + return WindowedValue.of( + windowedKv.getValue().getKey(), windowedKv.getTimestamp(), windowedKv.getWindows()); } @Override public Object getValueFromInputPair(Object pair) { - WindowedValue> windowedKv = (WindowedValue>) pair; + @SuppressWarnings("unchecked") + WindowedValue> windowedKv = (WindowedValue>) pair; return windowedKv.getValue().getValue(); } @Override public Object makeOutputPair(Object key, Object values) { - return WindowedValue.valueInEmptyWindows(KV.of(key, values)); + WindowedValue windowedKey = (WindowedValue) key; + return WindowedValue.of( + KV.of(windowedKey.getValue(), values), + windowedKey.getTimestamp(), + windowedKey.getWindows()); } } /** * Implements PGBKOp.GroupingKeyCreator via Coder. */ - public static class CoderGroupingKeyCreator - implements PartialGroupByKeyOperation.GroupingKeyCreator { - final Coder coder; + // TODO: Actually support window merging in the combiner table. + public static class WindowingCoderGroupingKeyCreator + implements GroupingKeyCreator { + + private static final Instant ignored = new Instant(0); + + private final Coder coder; - public CoderGroupingKeyCreator(Coder coder) { + public WindowingCoderGroupingKeyCreator(Coder coder) { this.coder = coder; } @Override - public Object createGroupingKey(Object value) throws Exception { - return new PartialGroupByKeyOperation.StructuralByteArray( - CoderUtils.encodeToByteArray(coder, value)); + public Object createGroupingKey(Object key) throws Exception { + WindowedValue windowedKey = (WindowedValue) key; + // Ignore timestamp for grouping purposes. + // The PGBK output will inherit the timestamp of one of its inputs. + return WindowedValue.of( + new PartialGroupByKeyOperation.StructuralByteArray( + CoderUtils.encodeToByteArray(coder, windowedKey.getValue())), + ignored, + windowedKey.getWindows()); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperationTest.java index 38aa5a35b3057..8a1b56f955681 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperationTest.java @@ -31,10 +31,10 @@ import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; -import com.google.cloud.dataflow.sdk.runners.worker.MapTaskExecutorFactory.CoderGroupingKeyCreator; import com.google.cloud.dataflow.sdk.runners.worker.MapTaskExecutorFactory.CoderSizeEstimator; import com.google.cloud.dataflow.sdk.runners.worker.MapTaskExecutorFactory.ElementByteSizeObservableCoder; import com.google.cloud.dataflow.sdk.runners.worker.MapTaskExecutorFactory.PairInfo; +import com.google.cloud.dataflow.sdk.runners.worker.MapTaskExecutorFactory.WindowingCoderGroupingKeyCreator; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.common.Counter; import com.google.cloud.dataflow.sdk.util.common.CounterSet; @@ -54,7 +54,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Random; @@ -82,14 +81,15 @@ public void testRunPartialGroupByKeyOperation() throws Exception { counterSet, counterPrefix); PartialGroupByKeyOperation pgbkOperation = - new PartialGroupByKeyOperation(new CoderGroupingKeyCreator(keyCoder), - new CoderSizeEstimator(keyCoder), - new CoderSizeEstimator(valueCoder), - PairInfo.create(), - receiver, - counterPrefix, - counterSet.getAddCounterMutator(), - stateSampler); + new PartialGroupByKeyOperation( + new WindowingCoderGroupingKeyCreator(keyCoder), + new CoderSizeEstimator(WindowedValue.getValueOnlyCoder(keyCoder)), + new CoderSizeEstimator(valueCoder), + PairInfo.create(), + receiver, + counterPrefix, + counterSet.getAddCounterMutator(), + stateSampler); pgbkOperation.start(); @@ -165,32 +165,46 @@ public long estimateSize(String element) { } } + private static class KvPairInfo implements PartialGroupByKeyOperation.PairInfo { + @Override + public Object getKeyFromInputPair(Object pair) { + return ((KV) pair).getKey(); + } + @Override + public Object getValueFromInputPair(Object pair) { + return ((KV) pair).getValue(); + } + @Override + public Object makeOutputPair(Object key, Object value) { + return KV.of(key, value); + } + } + @Test public void testBufferingGroupingTable() throws Exception { BufferingGroupingTable table = new BufferingGroupingTable<>( - 1000, new IdentityGroupingKeyCreator(), PairInfo.create(), + 1000, new IdentityGroupingKeyCreator(), new KvPairInfo(), new StringPowerSizeEstimator(), new StringPowerSizeEstimator()); TestReceiver receiver = new TestReceiver( - WindowedValue.getValueOnlyCoder( - KvCoder.of(StringUtf8Coder.of(), IterableCoder.of(StringUtf8Coder.of())))); + KvCoder.of(StringUtf8Coder.of(), IterableCoder.of(StringUtf8Coder.of()))); table.put("A", "a", receiver); table.put("B", "b1", receiver); table.put("B", "b2", receiver); table.put("C", "c", receiver); - assertThat(unwindowed(receiver.outputElems), empty()); + assertThat(receiver.outputElems, empty()); table.put("C", "cccc", receiver); - assertThat(unwindowed(receiver.outputElems), + assertThat(receiver.outputElems, hasItem((Object) KV.of("C", Arrays.asList("c", "cccc")))); table.put("DDDD", "d", receiver); - assertThat(unwindowed(receiver.outputElems), + assertThat(receiver.outputElems, hasItem((Object) KV.of("DDDD", Arrays.asList("d")))); table.flush(receiver); - assertThat(unwindowed(receiver.outputElems), + assertThat(receiver.outputElems, IsIterableContainingInAnyOrder.containsInAnyOrder( KV.of("A", Arrays.asList("a")), KV.of("B", Arrays.asList("b1", "b2")), @@ -220,28 +234,27 @@ public Long extract(Object key, Long accumulator) { CombiningGroupingTable table = new CombiningGroupingTable( - 1000, new IdentityGroupingKeyCreator(), PairInfo.create(), + 1000, new IdentityGroupingKeyCreator(), new KvPairInfo(), summingCombineFn, new StringPowerSizeEstimator(), new IdentitySizeEstimator()); TestReceiver receiver = new TestReceiver( - WindowedValue.getValueOnlyCoder( - KvCoder.of(StringUtf8Coder.of(), BigEndianLongCoder.of()))); + KvCoder.of(StringUtf8Coder.of(), BigEndianLongCoder.of())); table.put("A", 1, receiver); table.put("B", 2, receiver); table.put("B", 3, receiver); table.put("C", 4, receiver); - assertThat(unwindowed(receiver.outputElems), empty()); + assertThat(receiver.outputElems, empty()); table.put("C", 5000, receiver); - assertThat(unwindowed(receiver.outputElems), hasItem((Object) KV.of("C", 5004L))); + assertThat(receiver.outputElems, hasItem((Object) KV.of("C", 5004L))); table.put("DDDD", 6, receiver); - assertThat(unwindowed(receiver.outputElems), hasItem((Object) KV.of("DDDD", 6L))); + assertThat(receiver.outputElems, hasItem((Object) KV.of("DDDD", 6L))); table.flush(receiver); - assertThat(unwindowed(receiver.outputElems), + assertThat(receiver.outputElems, IsIterableContainingInAnyOrder.containsInAnyOrder( KV.of("A", 1L), KV.of("B", 2L + 3), @@ -249,14 +262,6 @@ public Long extract(Object key, Long accumulator) { KV.of("DDDD", 6L))); } - private List unwindowed(Iterable windowed) { - List unwindowed = new ArrayList<>(); - for (Object withWindow : windowed) { - unwindowed.add(((WindowedValue) withWindow).getValue()); - } - return unwindowed; - } - //////////////////////////////////////////////////////////////////////////// // Tests for the sampling size estimator. From bc77eeeec2f4dbd9b779ac080e0c2f5e70ebc557 Mon Sep 17 00:00:00 2001 From: liangzhang Date: Tue, 6 Jan 2015 17:39:02 -0800 Subject: [PATCH 0060/1541] Add control code for Cloud Debugger: * New option for Debugger version string * Modify worker pool config for Debugger in DataflowPipelineTranslator [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83391175 --- .../sdk/options/CloudDebuggerOptions.java | 55 +++++++++++++++++++ .../sdk/options/DataflowPipelineOptions.java | 2 +- .../runners/DataflowPipelineTranslator.java | 20 +++++++ .../DataflowPipelineTranslatorTest.java | 19 +++++++ 4 files changed, 95 insertions(+), 1 deletion(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/CloudDebuggerOptions.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/CloudDebuggerOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/CloudDebuggerOptions.java new file mode 100644 index 0000000000000..c3632ed1b4bfe --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/CloudDebuggerOptions.java @@ -0,0 +1,55 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + +/** + * Options for controlling Cloud Debugger. These options are experimental and subject to change. + */ +public interface CloudDebuggerOptions { + + /** + * User defined application version. Cloud Debugger uses it to group all + * running debuggee processes. Version should be different if users have + * multiple parallel runs of the same application with different inputs. + */ + String getCdbgVersion(); + void setCdbgVersion(String value); + + /** + * Return a JSON string for the Debugger metadata item. + */ + public static class DebuggerConfig { + private String version; + public String getVersion() { return version; } + public void setVersion(String version) { this.version = version; } + + /** + * Compute the string of Debugger config. + * @return JSON string of Debugger config metadata. + * @throws JsonProcessingException when converting to Json fails. + */ + public String computeMetadataString() throws JsonProcessingException { + ObjectMapper mapper = new ObjectMapper(); + String debuggerConfigString = mapper.writeValueAsString(this); + return debuggerConfigString; + } + } +} + diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java index d30f7dc4d96b3..907d020cad23d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java @@ -34,7 +34,7 @@ public interface DataflowPipelineOptions extends PipelineOptions, GcpOptions, ApplicationNameOptions, DataflowPipelineDebugOptions, DataflowPipelineWorkerPoolOptions, BigQueryOptions, - GcsOptions, StreamingOptions { + GcsOptions, StreamingOptions, CloudDebuggerOptions { /** * GCS path for temporary files. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index aff881ddfe999..f3ddbd1cb444a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -48,6 +48,7 @@ import com.google.cloud.dataflow.sdk.io.DatastoreIO; import com.google.cloud.dataflow.sdk.io.PubsubIO; import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.CloudDebuggerOptions.DebuggerConfig; import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; import com.google.cloud.dataflow.sdk.options.DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType; import com.google.cloud.dataflow.sdk.runners.dataflow.AvroIOTranslator; @@ -363,6 +364,25 @@ public Job translate(List packages) { workerPool.setTaskrunnerSettings(taskRunnerSettings); + // Config Cloud Debugger + if (!Strings.isNullOrEmpty(options.getCdbgVersion())) { + String cdbgVersion = options.getCdbgVersion(); + DebuggerConfig debuggerConfig = new DebuggerConfig(); + debuggerConfig.setVersion(cdbgVersion); + + Map metadata = workerPool.getMetadata(); + if (metadata == null) { + metadata = new HashMap(); + } + + try { + metadata.put("debugger", debuggerConfig.computeMetadataString()); + } catch (JsonProcessingException e) { + throw new IllegalArgumentException("Cannot format Debugger version.", e); + } + workerPool.setMetadata(metadata); + } + if (options.isStreaming()) { job.setType("JOB_TYPE_STREAMING"); } else { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java index 3eb90edbf78e6..e877ea33d3eac 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java @@ -175,6 +175,25 @@ public void testWorkerMachineTypeConfig() throws IOException { assertEquals(testMachineType, workerPool.getMachineType()); } + @Test + public void testDebuggerConfig() throws IOException { + final String cdbgVersion = "test-v1"; + DataflowPipelineOptions options = buildPipelineOptions(); + options.setCdbgVersion(cdbgVersion); + String expectedConfig = "{\"version\":\"test-v1\"}"; + + Pipeline p = buildPipeline(options); + p.traverseTopologically(new RecordingPipelineVisitor()); + Job job = DataflowPipelineTranslator.fromOptions(options).translate( + p, Collections.emptyList()); + + for (WorkerPool pool : job.getEnvironment().getWorkerPools()) { + if (pool.getKind() == DataflowPipelineTranslator.HARNESS_WORKER_POOL) { + assertEquals(pool.getMetadata().get("debugger"), expectedConfig); + } + } + } + @Test public void testDiskSizeGbConfig() throws IOException { final Integer diskSizeGb = 1234; From d54dcd1cce96efc90c51191300f9903e2fa5808a Mon Sep 17 00:00:00 2001 From: chamikara Date: Wed, 7 Jan 2015 13:58:11 -0800 Subject: [PATCH 0061/1541] We now mostly support wildcard matching for GCS paths so removing the checks. We still do not support recursive wildcards so adding a check for that. Additionally this modifies the wildcard expansion logic by removing the delimiter when listing objects. This results in all objects after the first wildcard being listed out of which we select objects that match regex derived by the user specified glob pattern. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83460566 --- .../runners/dataflow/TextIOTranslator.java | 13 +---- .../cloud/dataflow/sdk/util/GcsUtil.java | 58 +++++++++---------- .../cloud/dataflow/sdk/io/TextIOTest.java | 45 +++----------- .../runners/DataflowPipelineRunnerTest.java | 1 + .../DataflowPipelineTranslatorTest.java | 47 +++------------ 5 files changed, 48 insertions(+), 116 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java index 92aaf210e4ec8..302cea67a55c5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java @@ -21,7 +21,6 @@ import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator; import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext; -import com.google.cloud.dataflow.sdk.util.GcsUtil; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; @@ -50,16 +49,10 @@ private void translateReadHelper( throw new IllegalArgumentException("TextIO not supported in streaming mode."); } - // Only GCS paths are permitted for filepatterns in the DataflowPipelineRunner. + // Validate the provided GCS path. GcsPath gcsPath = GcsPath.fromUri(transform.getFilepattern()); - // Furthermore, on the service there is currently a limitation - // that the first wildcard character must occur after the last - // delimiter, and that the delimiter is fixed to '/' - if (!GcsUtil.GCS_READ_PATTERN.matcher(gcsPath.getObject()).matches()) { - throw new IllegalArgumentException( - "Unsupported wildcard usage in \"" + gcsPath + "\": " - + " all wildcards must occur after the final '/' delimiter."); - } + Preconditions.checkArgument( + context.getPipelineOptions().getGcsUtil().isGcsPatternSupported(gcsPath.getObject())); context.addStep(transform, "ParallelRead"); // TODO: How do we want to specify format and diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java index bcb387f29a6ed..dd51b669e2820 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java @@ -73,20 +73,13 @@ public GcsUtil create(PipelineOptions options) { /** Matches a glob containing a wildcard, capturing the portion before the first wildcard. */ private static final Pattern GLOB_PREFIX = Pattern.compile("(?[^*?]*)[*?].*"); - private static final String WILDCARD = "[\\[\\]*?]"; - private static final String NON_WILDCARD = "[^\\[\\]*?]"; - private static final String NON_DELIMITER = "[^/]"; - private static final String OPTIONAL_WILDCARD_AND_SUFFIX = "(" + WILDCARD + NON_DELIMITER + "*)?"; + private static final String RECURSIVE_WILDCARD = "[*]{2}"; /** - * A {@link Pattern} that matches globs in which every wildcard is interpreted as such, - * assuming a delimiter of {@code '/'}. - * - *

Most importantly, if a {@code '*'} or {@code '?'} occurs before the - * final delimiter it will not be interpreted as a wildcard. + * A {@link Pattern} for globs with a recursive wildcard. */ - public static final Pattern GCS_READ_PATTERN = Pattern.compile( - NON_WILDCARD + "*" + OPTIONAL_WILDCARD_AND_SUFFIX); + private static final Pattern RECURSIVE_GCS_PATTERN = + Pattern.compile(".*" + RECURSIVE_WILDCARD + ".*"); ///////////////////////////////////////////////////////////////////////////// @@ -99,37 +92,44 @@ public GcsUtil create(PipelineOptions options) { // Exposed for testing. final ExecutorService executorService; + /** + * Returns true if the given GCS pattern is supported otherwise fails with an + * exception. + */ + public boolean isGcsPatternSupported(String gcsPattern) { + if (RECURSIVE_GCS_PATTERN.matcher(gcsPattern).matches()) { + throw new IllegalArgumentException("Unsupported wildcard usage in \"" + gcsPattern + "\": " + + " recursive wildcards are not supported."); + } + + return true; + } + private GcsUtil(Storage storageClient, ExecutorService executorService) { storage = storageClient; this.executorService = executorService; } /** - * Expands a pattern into matched paths. The input path may contain - * globs (in the last component only!), which are expanded in the result. - * - *

TODO: add support for full path matching. + * Expands a pattern into matched paths. The pattern path may contain + * globs, which are expanded in the result. */ - public List expand(GcsPath path) throws IOException { - if (!GCS_READ_PATTERN.matcher(path.getObject()).matches()) { - throw new IllegalArgumentException( - "Unsupported wildcard usage in \"" + path + "\": " - + " all wildcards must occur after the final '/' delimiter."); - } - - Matcher m = GLOB_PREFIX.matcher(path.getObject()); + public List expand(GcsPath gcsPattern) throws IOException { + Preconditions.checkArgument(isGcsPatternSupported(gcsPattern.getObject())); + Matcher m = GLOB_PREFIX.matcher(gcsPattern.getObject()); if (!m.matches()) { - return Arrays.asList(path); + return Arrays.asList(gcsPattern); } + // Part before the first wildcard character. String prefix = m.group("PREFIX"); - Pattern p = Pattern.compile(globToRegexp(path.getObject())); + Pattern p = Pattern.compile(globToRegexp(gcsPattern.getObject())); LOG.info("matching files in bucket {}, prefix {} against pattern {}", - path.getBucket(), prefix, p.toString()); + gcsPattern.getBucket(), prefix, p.toString()); - Storage.Objects.List listObject = storage.objects().list(path.getBucket()); + // List all objects that start with the prefix (including objects in sub-directories). + Storage.Objects.List listObject = storage.objects().list(gcsPattern.getBucket()); listObject.setMaxResults(MAX_LIST_ITEMS_PER_CALL); - listObject.setDelimiter("/"); listObject.setPrefix(prefix); String pageToken = null; @@ -146,7 +146,7 @@ public List expand(GcsPath path) throws IOException { break; } - // Filter + // Filter objects based on the regex. for (StorageObject o : objects.getItems()) { String name = o.getName(); // Skip directories, which end with a slash. diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java index 28270949c328a..1cc3bf64030dd 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java @@ -340,7 +340,6 @@ public void testUnsupportedFilePattern() throws IOException { } /** - * The first wildcard must occur after the last directory delimiter. * This tests a few corner cases that should not crash. */ @Test @@ -360,54 +359,26 @@ public void testGoodWildcards() throws Exception { pipeline.apply(TextIO.Read.from("gs://bucket/foo/[0-9]baz?")); pipeline.apply(TextIO.Read.from("gs://bucket/foo/baz/*")); pipeline.apply(TextIO.Read.from("gs://bucket/foo/baz/*wonka*")); - - // Check that running doesn't fail. - pipeline.run(); - } - - /** - * The first wildcard must occur after the last directory delimiter. - * This tests "*". - */ - @Test - public void testBadWildcardStar() throws Exception { - Pipeline pipeline = Pipeline.create(buildTestPipelineOptions()); - + pipeline.apply(TextIO.Read.from("gs://bucket/foo/*baz/wonka*")); pipeline.apply(TextIO.Read.from("gs://bucket/foo*/baz")); - - // Check that running does fail. - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("wildcard"); - pipeline.run(); - } - - /** - * The first wildcard must occur after the last directory delimiter. - * This tests "?". - */ - @Test - public void testBadWildcardOptional() throws Exception { - Pipeline pipeline = Pipeline.create(buildTestPipelineOptions()); - pipeline.apply(TextIO.Read.from("gs://bucket/foo?/baz")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo[0-9]/baz")); - // Check that running does fail. - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("wildcard"); + // Check that running doesn't fail. pipeline.run(); } /** - * The first wildcard must occur after the last directory delimiter. - * This tests "[]" based character classes. + * Recursive wildcards are not supported. + * This tests "**". */ @Test - public void testBadWildcardBrackets() throws Exception { + public void testBadWildcardRecursive() throws Exception { Pipeline pipeline = Pipeline.create(buildTestPipelineOptions()); - pipeline.apply(TextIO.Read.from("gs://bucket/foo[0-9]/baz")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo**/baz")); - // Check that translation does fail. + // Check that running does fail. thrown.expect(IllegalArgumentException.class); thrown.expectMessage("wildcard"); pipeline.run(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java index 3d6f804ed3a21..38f2fb407c367 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java @@ -122,6 +122,7 @@ private GcsUtil buildMockGcsUtil() throws IOException { .thenReturn(FileChannel.open( Files.createTempFile("channel-", ".tmp"), StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE)); + when(mockGcsUtil.isGcsPatternSupported(anyString())).thenReturn(true); return mockGcsUtil; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java index e877ea33d3eac..bfa3ed7a21eb8 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java @@ -420,7 +420,6 @@ public void testPartiallyBoundFailure() throws IOException { } /** - * The first wildcard must occur after the last directory delimiter. * This tests a few corner cases that should not crash. */ @Test @@ -439,58 +438,26 @@ public void testGoodWildcards() throws Exception { pipeline.apply(TextIO.Read.from("gs://bucket/foo/[0-9]baz?")); pipeline.apply(TextIO.Read.from("gs://bucket/foo/baz/*")); pipeline.apply(TextIO.Read.from("gs://bucket/foo/baz/*wonka*")); - - // Check that translation doesn't fail. - t.translate(pipeline, Collections.emptyList()); - } - - /** - * The first wildcard must occur after the last directory delimiter. - * This tests "*". - */ - @Test - public void testBadWildcardStar() throws Exception { - DataflowPipelineOptions options = buildPipelineOptions(); - Pipeline pipeline = DataflowPipeline.create(options); - DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options); - + pipeline.apply(TextIO.Read.from("gs://bucket/foo/*/baz*")); pipeline.apply(TextIO.Read.from("gs://bucket/foo*/baz")); - - // Check that translation does fail. - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Unsupported wildcard usage"); - t.translate(pipeline, Collections.emptyList()); - } - - /** - * The first wildcard must occur after the last directory delimiter. - * This tests "?". - */ - @Test - public void testBadWildcardOptional() throws Exception { - DataflowPipelineOptions options = buildPipelineOptions(); - Pipeline pipeline = DataflowPipeline.create(options); - DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options); - pipeline.apply(TextIO.Read.from("gs://bucket/foo?/baz")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo[0-9]/baz")); - // Check that translation does fail. - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Unsupported wildcard usage"); + // Check that translation doesn't fail. t.translate(pipeline, Collections.emptyList()); } /** - * The first wildcard must occur after the last directory delimiter. - * This tests "[]" based character classes. + * Recursive wildcards are not supported. + * This tests "**". */ @Test - public void testBadWildcardBrackets() throws Exception { + public void testBadWildcardRecursive() throws Exception { DataflowPipelineOptions options = buildPipelineOptions(); Pipeline pipeline = DataflowPipeline.create(options); DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options); - pipeline.apply(TextIO.Read.from("gs://bucket/foo[0-9]/baz")); + pipeline.apply(TextIO.Read.from("gs://bucket/foo**/baz")); // Check that translation does fail. thrown.expect(IllegalArgumentException.class); From 224445d3d3d64c112efed072785bddfbf17bb0e9 Mon Sep 17 00:00:00 2001 From: amyu Date: Wed, 7 Jan 2015 14:13:39 -0800 Subject: [PATCH 0062/1541] Addition of MaxPerKeyExamples, an example that reads public samples of weather data from BigQuery, and finds the maximum temperature for each month. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83461984 --- .../dataflow/examples/MaxPerKeyExamples.java | 163 ++++++++++++++++++ .../examples/MaxPerKeyExamplesTest.java | 88 ++++++++++ 2 files changed, 251 insertions(+) create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/MaxPerKeyExamples.java create mode 100644 examples/src/test/java/com/google/cloud/dataflow/examples/MaxPerKeyExamplesTest.java diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/MaxPerKeyExamples.java b/examples/src/main/java/com/google/cloud/dataflow/examples/MaxPerKeyExamples.java new file mode 100644 index 0000000000000..a4c0a106da0a0 --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/MaxPerKeyExamples.java @@ -0,0 +1,163 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Max; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import java.util.ArrayList; +import java.util.List; + +/** + * An example that reads the public samples of weather data from BigQuery, and finds + * the maximum temperature ('mean_temp') for each month. + * + * Concepts: The 'Max' statistical combination function, and how to find the max per + * key group. + * + *

Note: Before running this example, you must create a BigQuery dataset to contain your output + * table. + * + *

To execute this pipeline locally, specify general pipeline configuration: + * --project= + * and the BigQuery table for the output: + * --output=:. + * + *

To execute this pipeline using the Dataflow service, specify pipeline configuration: + * --project= + * --stagingLocation=gs:// + * --runner=BlockingDataflowPipelineRunner + * and the BigQuery table for the output: + * --output=:. + * + *

The BigQuery input table defaults to clouddataflow-readonly:samples.weather_stations and can + * be overridden with --input. + */ +public class MaxPerKeyExamples { + // Default to using a 1000 row subset of the public weather station table publicdata:samples.gsod. + private static final String WEATHER_SAMPLES_TABLE = + "clouddataflow-readonly:samples.weather_stations"; + + /** + * Examines each row (weather reading) in the input table. Output the month of the reading, + * and the mean_temp. + */ + static class ExtractTempFn extends DoFn> { + @Override + public void processElement(ProcessContext c) { + TableRow row = c.element(); + Integer month = Integer.parseInt((String) row.get("month")); + Double meanTemp = Double.parseDouble(row.get("mean_temp").toString()); + c.output(KV.of(month, meanTemp)); + } + } + + /** + * Format the results to a TableRow, to save to BigQuery. + * + */ + static class FormatMaxesFn extends DoFn, TableRow> { + @Override + public void processElement(ProcessContext c) { + TableRow row = new TableRow() + .set("month", c.element().getKey()) + .set("max_mean_temp", c.element().getValue()); + c.output(row); + } + } + + /** + * Reads rows from a weather data table, and finds the max mean_temp for each + * month via the 'Max' statistical combination function. + */ + static class MaxMeanTemp + extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection rows) { + + // row... => ... + PCollection> temps = rows.apply( + ParDo.of(new ExtractTempFn())); + + // month, mean_temp... => ... + PCollection> tempMaxes = + temps.apply(Max.doublesPerKey()); + + // ... => row... + PCollection results = tempMaxes.apply( + ParDo.of(new FormatMaxesFn())); + + return results; + } + } + + /** + * Options supported by {@link MaxPerKeyExamples}. + *

+ * Inherits standard configuration options. + */ + private static interface Options extends PipelineOptions { + @Description("Table to read from, specified as " + + ":.") + @Default.String(WEATHER_SAMPLES_TABLE) + String getInput(); + void setInput(String value); + + @Description("Table to write to, specified as " + + ":.") + @Validation.Required + String getOutput(); + void setOutput(String value); + } + + public static void main(String[] args) + throws Exception { + + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + Pipeline p = Pipeline.create(options); + + // Build the table schema for the output table. + List fields = new ArrayList<>(); + fields.add(new TableFieldSchema().setName("month").setType("INTEGER")); + fields.add(new TableFieldSchema().setName("max_mean_temp").setType("FLOAT")); + TableSchema schema = new TableSchema().setFields(fields); + + p.apply(BigQueryIO.Read.from(options.getInput())) + .apply(new MaxMeanTemp()) + .apply(BigQueryIO.Write + .to(options.getOutput()) + .withSchema(schema) + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE)); + + p.run(); + } +} diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/MaxPerKeyExamplesTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/MaxPerKeyExamplesTest.java new file mode 100644 index 0000000000000..abb5710dbe480 --- /dev/null +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/MaxPerKeyExamplesTest.java @@ -0,0 +1,88 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.dataflow.examples.MaxPerKeyExamples.ExtractTempFn; +import com.google.cloud.dataflow.examples.MaxPerKeyExamples.FormatMaxesFn; +import com.google.cloud.dataflow.sdk.transforms.DoFnTester; +import com.google.cloud.dataflow.sdk.values.KV; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.List; + +/** Unit tests for {@link MaxPerKeyExamples}. */ +@RunWith(JUnit4.class) +public class MaxPerKeyExamplesTest { + + private static final TableRow row1 = new TableRow() + .set("month", "6").set("day", "21") + .set("year", "2014").set("mean_temp", "85.3") + .set("tornado", true); + private static final TableRow row2 = new TableRow() + .set("month", "7").set("day", "20") + .set("year", "2014").set("mean_temp", "75.4") + .set("tornado", false); + private static final TableRow row3 = new TableRow() + .set("month", "6").set("day", "18") + .set("year", "2014").set("mean_temp", "45.3") + .set("tornado", true); + private static final TableRow[] ROWS_ARRAY = new TableRow[] { + row1, row2, row3 + }; + + private static final KV kv1 = KV.of(6, 85.3); + private static final KV kv2 = KV.of(6, 45.3); + private static final KV kv3 = KV.of(7, 75.4); + + static final KV[] TUPLES_ARRAY = new KV[] { + kv1, kv2, kv3 + }; + + private static final TableRow resultRow1 = new TableRow() + .set("month", 6) + .set("max_mean_temp", 85.3); + private static final TableRow resultRow2 = new TableRow() + .set("month", 7) + .set("max_mean_temp", 75.4); + + + @Test + public void testExtractTempFn() { + DoFnTester> extractTempFn = + DoFnTester.of(new ExtractTempFn()); + List> results = extractTempFn.processBatch(ROWS_ARRAY); + Assert.assertThat(results, CoreMatchers.hasItem(kv1)); + Assert.assertThat(results, CoreMatchers.hasItem(kv2)); + Assert.assertThat(results, CoreMatchers.hasItem(kv3)); + } + + @Test + public void testFormatMaxesFn() { + DoFnTester, TableRow> formatMaxesFnFn = + DoFnTester.of(new FormatMaxesFn()); + List results = formatMaxesFnFn.processBatch(TUPLES_ARRAY); + Assert.assertThat(results, CoreMatchers.hasItem(resultRow1)); + Assert.assertThat(results, CoreMatchers.hasItem(resultRow2)); + } + +} From cbd598c5f245eac9fae8abcd69a4ca481da9016a Mon Sep 17 00:00:00 2001 From: amyu Date: Wed, 7 Jan 2015 15:22:26 -0800 Subject: [PATCH 0063/1541] Addition of filtering and deduplication examples. FilterExamples demonstrates several approaches to filtering, and use of the Mean transform. It shows how to dynamically set parameters by defining and using new pipeline options, and how to use a value derived by the pipeline. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83468417 --- .../cloud/dataflow/examples/DeDupExample.java | 99 +++++++ .../dataflow/examples/FilterExamples.java | 255 ++++++++++++++++++ .../dataflow/examples/DeDupExampleTest.java | 83 ++++++ .../dataflow/examples/FilterExamplesTest.java | 85 ++++++ 4 files changed, 522 insertions(+) create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/DeDupExample.java create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/FilterExamples.java create mode 100644 examples/src/test/java/com/google/cloud/dataflow/examples/DeDupExampleTest.java create mode 100644 examples/src/test/java/com/google/cloud/dataflow/examples/FilterExamplesTest.java diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/DeDupExample.java b/examples/src/main/java/com/google/cloud/dataflow/examples/DeDupExample.java new file mode 100644 index 0000000000000..2407c7ad25640 --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/DeDupExample.java @@ -0,0 +1,99 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.DefaultValueFactory; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; + +/** + * This example uses as input Shakespeare's plays as plaintext files, and will remove any + * duplicate lines across all the files. (The output does not preserve any input order). + * + * Concepts: the RemoveDuplicates transform, and how to wire transforms together. + * Demonstrates TextIO.Read/RemoveDuplicates/TextIO.Write. + * + *

To execute this pipeline locally, specify general pipeline configuration: + * --project= + * and a local output file or output prefix on GCS: + * --output=[ | gs://] + * + *

To execute this pipeline using the Dataflow service, specify pipeline configuration: + * --project= + * --stagingLocation=gs:// + * --runner=BlockingDataflowPipelineRunner + * and an output prefix on GCS: + * --output=gs:// + * + *

The input defaults to gs://dataflow-samples/shakespeare/* and can be + * overridden with --input. + */ +public class DeDupExample { + + /** + * Options supported by {@link DeDupExample}. + *

+ * Inherits standard configuration options. + */ + private static interface Options extends PipelineOptions { + @Description("Path to the directory or GCS prefix containing files to read from") + @Default.String("gs://dataflow-samples/shakespeare/*") + String getInput(); + void setInput(String value); + + @Description("Path of the file to write to") + @Default.InstanceFactory(OutputFactory.class) + String getOutput(); + void setOutput(String value); + + /** Returns gs://${STAGING_LOCATION}/"deduped.txt". */ + public static class OutputFactory implements DefaultValueFactory { + @Override + public String create(PipelineOptions options) { + DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); + if (dataflowOptions.getStagingLocation() != null) { + return GcsPath.fromUri(dataflowOptions.getStagingLocation()) + .resolve("deduped.txt").toString(); + } else { + throw new IllegalArgumentException("Must specify --output or --stagingLocation"); + } + } + } + } + + + public static void main(String[] args) + throws Exception { + + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + Pipeline p = Pipeline.create(options); + + p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) + .apply(RemoveDuplicates.create()) + .apply(TextIO.Write.named("DedupedShakespeare") + .to(options.getOutput())); + + p.run(); + } +} diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/FilterExamples.java b/examples/src/main/java/com/google/cloud/dataflow/examples/FilterExamples.java new file mode 100644 index 0000000000000..45646911a0a36 --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/FilterExamples.java @@ -0,0 +1,255 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Mean; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionView; + +import java.util.ArrayList; +import java.util.List; +import java.util.logging.Logger; + +/** + * This is an example that demonstrates several approaches to filtering, and use of the Mean + * transform. It shows how to dynamically set parameters by defining and using new pipeline options, + * and how to use a value derived by the pipeline. + * + *

Concepts: The Mean transform; Options configuration; using pipeline-derived data as a side + * input; approaches to filtering, selection, and projection. + * + *

The example reads public samples of weather data from BigQuery. It performs a + * projection on the data, finds the global mean of the temperature readings, filters on readings + * for a single given month, and then outputs only data (for that month) that has a mean temp + * smaller than the derived global mean. +* + *

Note: Before running this example, you must create a BigQuery dataset to contain your output + * table. + * + *

To execute this pipeline locally, specify general pipeline configuration: + * --project= + * and the BigQuery table for the output: + * --output=:. + * [--monthFilter=] + * where optional parameter --monthFilter is set to a number 1-12. + * + *

To execute this pipeline using the Dataflow service, specify pipeline configuration: + * --project= + * --stagingLocation=gs:// + * --runner=BlockingDataflowPipelineRunner + * and the BigQuery table for the output: + * --output=:. + * [--monthFilter=] + * where optional parameter --monthFilter is set to a number 1-12. + * + *

The BigQuery input table defaults to clouddataflow-readonly:samples.weather_stations and can + * be overridden with --input. + */ +public class FilterExamples { + // Default to using a 1000 row subset of the public weather station table publicdata:samples.gsod. + private static final String WEATHER_SAMPLES_TABLE = + "clouddataflow-readonly:samples.weather_stations"; + static final Logger LOG = Logger.getLogger(FilterExamples.class.getName()); + static final int MONTH_TO_FILTER = 7; + + /** + * Examines each row in the input table. Outputs only the subset of the cells this example + * is interested in-- the mean_temp and year, month, and day-- as a bigquery table row. + */ + static class ProjectionFn extends DoFn { + @Override + public void processElement(ProcessContext c){ + TableRow row = c.element(); + // Grab year, month, day, mean_temp from the row + Integer year = Integer.parseInt((String) row.get("year")); + Integer month = Integer.parseInt((String) row.get("month")); + Integer day = Integer.parseInt((String) row.get("day")); + Double meanTemp = Double.parseDouble(row.get("mean_temp").toString()); + // Prepares the data for writing to BigQuery by building a TableRow object + TableRow outRow = new TableRow() + .set("year", year).set("month", month) + .set("day", day).set("mean_temp", meanTemp); + c.output(outRow); + } + } + + /** + * Implements 'filter' functionality. + * + *

Examines each row in the input table. Outputs only rows from the month + * monthFilter, which is passed in as a parameter during construction of this DoFn. + */ + static class FilterSingleMonthDataFn extends DoFn { + + Integer monthFilter; + + public FilterSingleMonthDataFn(Integer monthFilter) { + this.monthFilter = monthFilter; + } + + @Override + public void processElement(ProcessContext c){ + TableRow row = c.element(); + Integer month; + month = (Integer) row.get("month"); + if (month.equals(this.monthFilter)) { + c.output(row); + } + } + } + + /** + * Examines each row (weather reading) in the input table. Output the temperature + * reading for that row ('mean_temp'). + */ + static class ExtractTempFn extends DoFn { + @Override + public void processElement(ProcessContext c){ + TableRow row = c.element(); + Double meanTemp = Double.parseDouble(row.get("mean_temp").toString()); + c.output(meanTemp); + } + } + + + + /* + * Finds the global mean of the mean_temp for each day/record, and outputs + * only data that has a mean temp larger than this global mean. + **/ + static class BelowGlobalMean + extends PTransform, PCollection> { + + Integer monthFilter; + + public BelowGlobalMean(Integer monthFilter) { + this.monthFilter = monthFilter; + } + + + @Override + public PCollection apply(PCollection rows) { + + // Extract the mean_temp from each row. + PCollection meanTemps = rows.apply( + ParDo.of(new ExtractTempFn())); + + // Find the global mean, of all the mean_temp readings in the weather data, + // and prepare this singleton PCollectionView for use as a side input. + final PCollectionView globalMeanTemp = + meanTemps.apply(Mean.globally()) + .apply(View.asSingleton()); + + // Rows filtered to remove all but a single month + PCollection monthFilteredRows = rows + .apply(ParDo.of(new FilterSingleMonthDataFn(monthFilter))); + + // Then, use the global mean as a side input, to further filter the weather data. + // By using a side input to pass in the filtering criteria, we can use a value + // that is computed earlier in pipeline execution. + // We'll only output readings with temperatures below this mean. + PCollection filteredRows = monthFilteredRows + .apply(ParDo + .withSideInputs(globalMeanTemp) + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + Double meanTemp = Double.parseDouble(c.element().get("mean_temp").toString()); + Double gTemp = c.sideInput(globalMeanTemp); + if (meanTemp < gTemp) { + c.output(c.element()); + } + } + })); + + return filteredRows; + } + } + + + /** + * Options supported by {@link FilterExamples}. + *

+ * Inherits standard configuration options. + */ + private static interface Options extends PipelineOptions { + @Description("Table to read from, specified as " + + ":.") + @Default.String(WEATHER_SAMPLES_TABLE) + String getInput(); + void setInput(String value); + + @Description("Table to write to, specified as " + + ":.. " + + "The dataset_id must already exist") + @Validation.Required + String getOutput(); + void setOutput(String value); + + @Description("Numeric value of month to filter on") + @Default.Integer(MONTH_TO_FILTER) + Integer getMonthFilter(); + void setMonthFilter(Integer value); + } + + /** + * Helper method to build the table schema for the output table. + */ + private static TableSchema buildWeatherSchemaProjection() { + List fields = new ArrayList<>(); + fields.add(new TableFieldSchema().setName("year").setType("INTEGER")); + fields.add(new TableFieldSchema().setName("month").setType("INTEGER")); + fields.add(new TableFieldSchema().setName("day").setType("INTEGER")); + fields.add(new TableFieldSchema().setName("mean_temp").setType("FLOAT")); + TableSchema schema = new TableSchema().setFields(fields); + return schema; + } + + public static void main(String[] args) + throws Exception { + + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + Pipeline p = Pipeline.create(options); + + TableSchema schema = buildWeatherSchemaProjection(); + + p.apply(BigQueryIO.Read.from(options.getInput())) + .apply(ParDo.of(new ProjectionFn())) + .apply(new BelowGlobalMean(options.getMonthFilter())) + .apply(BigQueryIO.Write + .to(options.getOutput()) + .withSchema(schema) + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE)); + + p.run(); + } +} diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/DeDupExampleTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/DeDupExampleTest.java new file mode 100644 index 0000000000000..c52f675c01bce --- /dev/null +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/DeDupExampleTest.java @@ -0,0 +1,83 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** Unit tests for {@link DeDupExample}. */ +@RunWith(JUnit4.class) +public class DeDupExampleTest { + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testRemoveDuplicates() { + List strings = Arrays.asList( + "k1", + "k5", + "k5", + "k2", + "k1", + "k2", + "k3"); + + Pipeline p = TestPipeline.create(); + + PCollection input = + p.apply(Create.of(strings)) + .setCoder(StringUtf8Coder.of()); + + PCollection output = + input.apply(RemoveDuplicates.create()); + + DataflowAssert.that(output) + .containsInAnyOrder("k1", "k5", "k2", "k3"); + p.run(); + } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testRemoveDuplicatesEmpty() { + List strings = Arrays.asList(); + + Pipeline p = TestPipeline.create(); + + PCollection input = + p.apply(Create.of(strings)) + .setCoder(StringUtf8Coder.of()); + + PCollection output = + input.apply(RemoveDuplicates.create()); + + DataflowAssert.that(output) + .containsInAnyOrder(); + p.run(); + } +} diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/FilterExamplesTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/FilterExamplesTest.java new file mode 100644 index 0000000000000..5845eb4b53611 --- /dev/null +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/FilterExamplesTest.java @@ -0,0 +1,85 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.dataflow.examples.FilterExamples.FilterSingleMonthDataFn; +import com.google.cloud.dataflow.examples.FilterExamples.ProjectionFn; +import com.google.cloud.dataflow.sdk.transforms.DoFnTester; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** Unit tests for {@link FilterExamples}. */ +@RunWith(JUnit4.class) +public class FilterExamplesTest { + + private static final TableRow row1 = new TableRow() + .set("month", "6").set("day", "21") + .set("year", "2014").set("mean_temp", "85.3") + .set("tornado", true); + private static final TableRow row2 = new TableRow() + .set("month", "7").set("day", "20") + .set("year", "2014").set("mean_temp", "75.4") + .set("tornado", false); + private static final TableRow row3 = new TableRow() + .set("month", "6").set("day", "18") + .set("year", "2014").set("mean_temp", "45.3") + .set("tornado", true); + static final TableRow[] ROWS_ARRAY = new TableRow[] { + row1, row2, row3 + }; + static final List ROWS = Arrays.asList(ROWS_ARRAY); + + private static final TableRow outRow1 = new TableRow() + .set("year", 2014).set("month", 6) + .set("day", 21).set("mean_temp", 85.3); + private static final TableRow outRow2 = new TableRow() + .set("year", 2014).set("month", 7) + .set("day", 20).set("mean_temp", 75.4); + private static final TableRow outRow3 = new TableRow() + .set("year", 2014).set("month", 6) + .set("day", 18).set("mean_temp", 45.3); + private static final TableRow[] PROJROWS_ARRAY = new TableRow[] { + outRow1, outRow2, outRow3 + }; + + + @Test + public void testProjectionFn() { + DoFnTester projectionFn = + DoFnTester.of(new ProjectionFn()); + List results = projectionFn.processBatch(ROWS_ARRAY); + Assert.assertThat(results, CoreMatchers.hasItem(outRow1)); + Assert.assertThat(results, CoreMatchers.hasItem(outRow2)); + Assert.assertThat(results, CoreMatchers.hasItem(outRow3)); + } + + @Test + public void testFilterSingleMonthDataFn() { + DoFnTester filterSingleMonthDataFn = + DoFnTester.of(new FilterSingleMonthDataFn(7)); + List results = filterSingleMonthDataFn.processBatch(PROJROWS_ARRAY); + Assert.assertThat(results, CoreMatchers.hasItem(outRow2)); + } +} From 8bdee5c6d981ba31f362b2cda53549afbee8d1f5 Mon Sep 17 00:00:00 2001 From: amyu Date: Wed, 7 Jan 2015 16:02:18 -0800 Subject: [PATCH 0064/1541] Addition of CombinePerKeyExamples. This example reads public 'Shakespeare' data, and for each word in the dataset that is over a given length, generates a string containing the list of play names in which that word appears. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83471835 --- .../examples/CombinePerKeyExamples.java | 218 ++++++++++++++++++ .../examples/CombinePerKeyExamplesTest.java | 90 ++++++++ 2 files changed, 308 insertions(+) create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/CombinePerKeyExamples.java create mode 100644 examples/src/test/java/com/google/cloud/dataflow/examples/CombinePerKeyExamplesTest.java diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/CombinePerKeyExamples.java b/examples/src/main/java/com/google/cloud/dataflow/examples/CombinePerKeyExamples.java new file mode 100644 index 0000000000000..14df95e27408b --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/CombinePerKeyExamples.java @@ -0,0 +1,218 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import java.util.ArrayList; +import java.util.List; + +/** + * An example that reads the public 'Shakespeare' data, and for each word in + * the dataset that is over a given length, generates a string containing the + * list of play names in which that word appears, and saves this information + * to a bigquery table. + * + *

Concepts: the Combine.perKey transform, which lets you combine the values in a + * key-grouped Collection, and how to use an Aggregator to track information in the + * Monitoring UI. + * + *

Note: Before running this example, you must create a BigQuery dataset to contain your output + * table. + * + *

To execute this pipeline locally, specify general pipeline configuration: + * --project= + * and the BigQuery table for the output: + * --output=:. + * + *

To execute this pipeline using the Dataflow service, specify pipeline configuration: + * --project= + * --stagingLocation=gs:// + * --runner=BlockingDataflowPipelineRunner + * and the BigQuery table for the output: + * --output=:. + * + *

The BigQuery input table defaults to publicdata:samples.shakespeare and can + * be overridden with --input. + */ +public class CombinePerKeyExamples { + // Use the shakespeare public BigQuery sample + private static final String SHAKESPEARE_TABLE = + "publicdata:samples.shakespeare"; + // We'll track words >= this word length across all plays in the table. + private static final int MIN_WORD_LENGTH = 9; + + /** + * Examines each row in the input table. If the word is >= MIN_WORD_LENGTH, + * outputs word, play_name. + */ + static class ExtractLargeWordsFn extends DoFn> { + + private Aggregator smallerWords; + + @Override + public void startBundle(Context c) { + smallerWords = c.createAggregator("smallerWords", new Sum.SumLongFn()); + } + + @Override + public void processElement(ProcessContext c){ + TableRow row = c.element(); + String playName = (String) row.get("corpus"); + String word = (String) row.get("word"); + if (word.length() >= MIN_WORD_LENGTH) { + c.output(KV.of(word, playName)); + } else { + // Track how many smaller words we're not including. This information will be + // visible in the Monitoring UI. + smallerWords.addValue(1L); + } + } + } + + + /** + * Prepares the data for writing to BigQuery by building a TableRow object + * containing a word with a string listing the plays in which it appeared. + */ + static class FormatShakespeareOutputFn extends DoFn, TableRow> { + @Override + public void processElement(ProcessContext c) { + TableRow row = new TableRow() + .set("word", c.element().getKey()) + .set("all_plays", c.element().getValue()); + c.output(row); + } + } + + /** + * Reads the public 'Shakespeare' data, and for each word in the dataset + * over a given length, generates a string containing the list of play names + * in which that word appears. It does this via the Combine.perKey + * transform, with the ConcatWords combine function. + * + *

Combine.perKey is similar to a GroupByKey followed by a ParDo, but + * has more restricted semantics that allow it to be executed more + * efficiently. These records are then formatted as BQ table rows. + */ + static class PlaysForWord + extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection rows) { + + // row... => ... + PCollection> words = rows.apply( + ParDo.of(new ExtractLargeWordsFn())); + + // word, play_name => word, all_plays ... + PCollection> wordAllPlays = + words.apply(Combine.perKey( + new ConcatWords())); + + // ... => row... + PCollection results = wordAllPlays.apply( + ParDo.of(new FormatShakespeareOutputFn())); + + return results; + } + } + + /** + * A 'combine function' used with the Combine.perKey transform. Builds a + * comma-separated string of all input items. So, it will build a string + * containing all the different Shakespeare plays in which the given input + * word has appeared. + */ + public static class ConcatWords implements SerializableFunction, String> { + @Override + public String apply(Iterable input) { + StringBuilder all = new StringBuilder(); + for (String item : input) { + if (!item.isEmpty()) { + if (all.length() == 0) { + all.append(item); + } else { + all.append(","); + all.append(item); + } + } + } + return all.toString(); + } + } + + /** + * Options supported by {@link CombinePerKeyExamples}. + *

+ * Inherits standard configuration options. + */ + private static interface Options extends PipelineOptions { + @Description("Table to read from, specified as " + + ":.") + @Default.String(SHAKESPEARE_TABLE) + String getInput(); + void setInput(String value); + + @Description("Table to write to, specified as " + + ":.. " + + "The dataset_id must already exist") + @Validation.Required + String getOutput(); + void setOutput(String value); + } + + public static void main(String[] args) + throws Exception { + + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + Pipeline p = Pipeline.create(options); + + // Build the table schema for the output table. + List fields = new ArrayList<>(); + fields.add(new TableFieldSchema().setName("word").setType("STRING")); + fields.add(new TableFieldSchema().setName("all_plays").setType("STRING")); + TableSchema schema = new TableSchema().setFields(fields); + + p.apply(BigQueryIO.Read.from(options.getInput())) + .apply(new PlaysForWord()) + .apply(BigQueryIO.Write + .to(options.getOutput()) + .withSchema(schema) + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE)); + + p.run(); + } +} diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/CombinePerKeyExamplesTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/CombinePerKeyExamplesTest.java new file mode 100644 index 0000000000000..d30432afaac8f --- /dev/null +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/CombinePerKeyExamplesTest.java @@ -0,0 +1,90 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.dataflow.examples.CombinePerKeyExamples.ExtractLargeWordsFn; +import com.google.cloud.dataflow.examples.CombinePerKeyExamples.FormatShakespeareOutputFn; +import com.google.cloud.dataflow.sdk.transforms.DoFnTester; +import com.google.cloud.dataflow.sdk.values.KV; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.List; + +/** Unit tests for {@link CombinePerKeyExamples}. */ +@RunWith(JUnit4.class) +public class CombinePerKeyExamplesTest { + + private static final TableRow row1 = new TableRow() + .set("corpus", "king_lear").set("word", "snuffleupaguses"); + private static final TableRow row2 = new TableRow() + .set("corpus", "macbeth").set("word", "antidisestablishmentarianism"); + private static final TableRow row3 = new TableRow() + .set("corpus", "king_lear").set("word", "antidisestablishmentarianism"); + private static final TableRow row4 = new TableRow() + .set("corpus", "macbeth").set("word", "bob"); + private static final TableRow row5 = new TableRow() + .set("corpus", "king_lear").set("word", "hi"); + + static final TableRow[] ROWS_ARRAY = new TableRow[] { + row1, row2, row3, row4, row5 + }; + + private static final KV tuple1 = KV.of("snuffleupaguses", "king_lear"); + private static final KV tuple2 = KV.of("antidisestablishmentarianism", "macbeth"); + private static final KV tuple3 = KV.of("antidisestablishmentarianism", + "king_lear"); + + private static final KV combinedTuple1 = KV.of("antidisestablishmentarianism", + "king_lear,macbeth"); + private static final KV combinedTuple2 = KV.of("snuffleupaguses", "king_lear"); + + static final KV[] COMBINED_TUPLES_ARRAY = new KV[] { + combinedTuple1, combinedTuple2 + }; + + private static final TableRow resultRow1 = new TableRow() + .set("word", "snuffleupaguses").set("all_plays", "king_lear"); + private static final TableRow resultRow2 = new TableRow() + .set("word", "antidisestablishmentarianism") + .set("all_plays", "king_lear,macbeth"); + + @Test + public void testExtractLargeWordsFn() { + DoFnTester> extractLargeWordsFn = + DoFnTester.of(new ExtractLargeWordsFn()); + List> results = extractLargeWordsFn.processBatch(ROWS_ARRAY); + Assert.assertThat(results, CoreMatchers.hasItem(tuple1)); + Assert.assertThat(results, CoreMatchers.hasItem(tuple2)); + Assert.assertThat(results, CoreMatchers.hasItem(tuple2)); + } + + @Test + public void testFormatShakespeareOutputFn() { + DoFnTester, TableRow> formatShakespeareOutputFn = + DoFnTester.of(new FormatShakespeareOutputFn()); + List results = formatShakespeareOutputFn.processBatch(COMBINED_TUPLES_ARRAY); + Assert.assertThat(results, CoreMatchers.hasItem(resultRow1)); + Assert.assertThat(results, CoreMatchers.hasItem(resultRow2)); + } + +} From 8955e2576fd61827aa5ad71f19dc9f45ddb6e160 Mon Sep 17 00:00:00 2001 From: amyu Date: Wed, 7 Jan 2015 17:12:46 -0800 Subject: [PATCH 0065/1541] Addition of an example that shows how to do a join on two collections. It uses a sample of the GDELT 'world event' data, joining the event 'action' country code against a table that maps country codes to country names. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83477930 --- .../cloud/dataflow/examples/JoinExamples.java | 178 ++++++++++++++++++ .../dataflow/examples/JoinExamplesTest.java | 113 +++++++++++ 2 files changed, 291 insertions(+) create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/JoinExamples.java create mode 100644 examples/src/test/java/com/google/cloud/dataflow/examples/JoinExamplesTest.java diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/JoinExamples.java b/examples/src/main/java/com/google/cloud/dataflow/examples/JoinExamples.java new file mode 100644 index 0000000000000..5f910d6dfbcf4 --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/JoinExamples.java @@ -0,0 +1,178 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; +import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; +import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TupleTag; + +/** + * This example shows how to do a join on two collections. + * It uses a sample of the GDELT 'world event' data (http://goo.gl/OB6oin), joining the event + * 'action' country code against a table that maps country codes to country names. + * + *

Concepts: Join operation; multiple input sources. + * + *

To execute this pipeline locally, specify general pipeline configuration: + * --project= + * and a local output file or output prefix on GCS: + * --output=[ | gs://] + * + *

To execute this pipeline using the Dataflow service, specify pipeline configuration: + * --project= + * --stagingLocation=gs:// + * --runner=BlockingDataflowPipelineRunner + * and an output prefix on GCS: + * --output=gs:// + */ +public class JoinExamples { + + // A 1000-row sample of the GDELT data here: gdelt-bq:full.events. + private static final String GDELT_EVENTS_TABLE = + "clouddataflow-readonly:samples.gdelt_sample"; + // A table that maps country codes to country names. + private static final String COUNTRY_CODES = + "gdelt-bq:full.crosswalk_geocountrycodetohuman"; + private static final TupleTag eventInfoTag = new TupleTag(); + private static final TupleTag countryInfoTag = new TupleTag(); + + /** + * Process the CoGbkResult elements generated by the CoGroupByKey transform. + */ + static class ProcessJoinFn extends DoFn, KV> { + + @Override + public void processElement(ProcessContext c) { + KV e = c.element(); + CoGbkResult val = e.getValue(); + String countryCode = e.getKey(); + String countryName = "none"; + countryName = e.getValue().getOnly(countryInfoTag); + for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) { + // Generate a string that combines information from both collection values + c.output(KV.of(countryCode, "Country name: " + countryName + ", Event info: " + eventInfo)); + } + } + } + + /** + * Join two collections, using country code as the key. + */ + static PCollection joinEvents(PCollection eventsTable, + PCollection countryCodes) throws Exception { + + // transform both input collections to tuple collections, where the keys are country + // codes in both cases. + PCollection> eventInfo = eventsTable.apply( + ParDo.of(new ExtractEventDataFn())); + PCollection> countryInfo = countryCodes.apply( + ParDo.of(new ExtractCountryInfoFn())); + + // country code 'key' -> CGBKR (, ) + PCollection> kvpCollection = KeyedPCollectionTuple + .of(eventInfoTag, eventInfo) + .and(countryInfoTag, countryInfo) + .apply(CoGroupByKey.create()); + + // country code 'key' -> string of , + PCollection> finalResultCollection = + kvpCollection.apply(ParDo.of(new ProcessJoinFn())); + + // write to GCS + PCollection formattedResults = finalResultCollection + .apply(ParDo.of(new DoFn, String>() { + @Override + public void processElement(ProcessContext c) { + String outputstring = "Country code: " + c.element().getKey() + + ", " + c.element().getValue(); + c.output(outputstring); + } + })); + return formattedResults; + } + + /** + * Examines each row (event) in the input table. Output a KV with the key the country + * code of the event, and the value a string encoding event information. + */ + static class ExtractEventDataFn extends DoFn> { + @Override + public void processElement(ProcessContext c) { + TableRow row = c.element(); + String countryCode = (String) row.get("ActionGeo_CountryCode"); + String sqlDate = (String) row.get("SQLDATE"); + String actor1Name = (String) row.get("Actor1Name"); + String sourceUrl = (String) row.get("SOURCEURL"); + String eventInfo = "Date: " + sqlDate + ", Actor1: " + actor1Name + ", url: " + sourceUrl; + c.output(KV.of(countryCode, eventInfo)); + } + } + + + /** + * Examines each row (country info) in the input table. Output a KV with the key the country + * code, and the value the country name. + */ + static class ExtractCountryInfoFn extends DoFn> { + @Override + public void processElement(ProcessContext c) { + TableRow row = c.element(); + String countryCode = (String) row.get("FIPSCC"); + String countryName = (String) row.get("HumanName"); + c.output(KV.of(countryCode, countryName)); + } + } + + + /** + * Options supported by {@link JoinExamples}. + *

+ * Inherits standard configuration options. + */ + private static interface Options extends PipelineOptions { + @Description("Path of the file to write to") + @Validation.Required + String getOutput(); + void setOutput(String value); + } + + public static void main(String[] args) throws Exception { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + Pipeline p = Pipeline.create(options); + // the following two 'applys' create multiple inputs to our pipeline, one for each + // of our two input sources. + PCollection eventsTable = p.apply(BigQueryIO.Read.from(GDELT_EVENTS_TABLE)); + PCollection countryCodes = p.apply(BigQueryIO.Read.from(COUNTRY_CODES)); + PCollection formattedResults = joinEvents(eventsTable, countryCodes); + formattedResults.apply(TextIO.Write.to(options.getOutput())); + p.run(); + } + +} diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/JoinExamplesTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/JoinExamplesTest.java new file mode 100644 index 0000000000000..0d51495d57496 --- /dev/null +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/JoinExamplesTest.java @@ -0,0 +1,113 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.dataflow.examples.JoinExamples.ExtractCountryInfoFn; +import com.google.cloud.dataflow.examples.JoinExamples.ExtractEventDataFn; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFnTester; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** Unit tests for {@link JoinExamples}. */ +@RunWith(JUnit4.class) +public class JoinExamplesTest { + + private static final TableRow row1 = new TableRow() + .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212") + .set("Actor1Name", "BANGKOK").set("SOURCEURL", "http://cnn.com"); + private static final TableRow row2 = new TableRow() + .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212") + .set("Actor1Name", "LAOS").set("SOURCEURL", "http://www.chicagotribune.com"); + private static final TableRow row3 = new TableRow() + .set("ActionGeo_CountryCode", "BE").set("SQLDATE", "20141213") + .set("Actor1Name", "AFGHANISTAN").set("SOURCEURL", "http://cnn.com"); + static final TableRow[] EVENTS = new TableRow[] { + row1, row2, row3 + }; + static final List EVENT_ARRAY = Arrays.asList(EVENTS); + + private static final KV kv1 = KV.of("VM", + "Date: 20141212, Actor1: LAOS, url: http://www.chicagotribune.com"); + private static final KV kv2 = KV.of("BE", + "Date: 20141213, Actor1: AFGHANISTAN, url: http://cnn.com"); + private static final KV kv3 = KV.of("BE", "Belgium"); + private static final KV kv4 = KV.of("VM", "Vietnam"); + + private static final TableRow cc1 = new TableRow() + .set("FIPSCC", "VM").set("HumanName", "Vietnam"); + private static final TableRow cc2 = new TableRow() + .set("FIPSCC", "BE").set("HumanName", "Belgium"); + static final TableRow[] CCS = new TableRow[] { + cc1, cc2 + }; + static final List CC_ARRAY = Arrays.asList(CCS); + + static final String[] JOINED_EVENTS = new String[] { + "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: LAOS, " + + "url: http://www.chicagotribune.com", + "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: BANGKOK, " + + "url: http://cnn.com", + "Country code: BE, Country name: Belgium, Event info: Date: 20141213, Actor1: AFGHANISTAN, " + + "url: http://cnn.com" + }; + + @Test + public void testExtractEventDataFn() { + DoFnTester> extractEventDataFn = + DoFnTester.of(new ExtractEventDataFn()); + List> results = extractEventDataFn.processBatch(EVENTS); + Assert.assertThat(results, CoreMatchers.hasItem(kv1)); + Assert.assertThat(results, CoreMatchers.hasItem(kv2)); + } + + @Test + public void testExtractCountryInfoFn() { + DoFnTester> extractCountryInfoFn = + DoFnTester.of(new ExtractCountryInfoFn()); + List> results = extractCountryInfoFn.processBatch(CCS); + Assert.assertThat(results, CoreMatchers.hasItem(kv3)); + Assert.assertThat(results, CoreMatchers.hasItem(kv4)); + } + + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testJoin() throws java.lang.Exception { + Pipeline p = TestPipeline.create(); + PCollection input1 = p.apply(Create.of(EVENT_ARRAY)); + PCollection input2 = p.apply(Create.of(CC_ARRAY)); + + PCollection output = JoinExamples.joinEvents(input1, input2); + DataflowAssert.that(output).containsInAnyOrder(JOINED_EVENTS); + p.run(); + } +} From 65a6a10e75a2c262555c02517bbacf7fe92b2f3e Mon Sep 17 00:00:00 2001 From: davor Date: Wed, 7 Jan 2015 23:36:11 -0800 Subject: [PATCH 0066/1541] Dataflow on Windows: fix local file matching by replacing backslash with a double-backslash. This fixes #2 on GitHub. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83495198 --- .../dataflow/sdk/util/FileIOChannelFactory.java | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java index 1bb7566a526b5..5b27e277f338c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java @@ -34,6 +34,7 @@ import java.util.Collection; import java.util.LinkedList; import java.util.List; +import java.util.regex.Matcher; /** * Implements IOChannelFactory for local files. @@ -52,8 +53,18 @@ public Collection match(String spec) throws IOException { throw new IOException("Unable to find parent directory of " + spec); } + // Method getAbsolutePath() on Windows platform may return something like + // "c:\temp\file.txt". FileSystem.getPathMatcher() call below will treat + // '\' (backslash) as an escape character, instead of a directory + // separator. Replacing backslash with double-backslash solves the problem. + // We perform the replacement on all platforms, even those that allow + // backslash as a part of the filename, because Globs.toRegexPattern will + // eat one backslash. + String pathToMatch = file.getAbsolutePath().replaceAll(Matcher.quoteReplacement("\\"), + Matcher.quoteReplacement("\\\\")); + final PathMatcher matcher = - FileSystems.getDefault().getPathMatcher("glob:" + file.getAbsolutePath()); + FileSystems.getDefault().getPathMatcher("glob:" + pathToMatch); File[] files = parent.listFiles(new FileFilter() { @Override public boolean accept(File pathname) { From ec1d2d8a52d53885fd1b92ceeb55f51f0a0c347c Mon Sep 17 00:00:00 2001 From: davor Date: Thu, 8 Jan 2015 17:04:55 -0800 Subject: [PATCH 0067/1541] Dataflow on Windows: remove usage of "/tmp/", since that is a platform-specific directory. * In DataflowWorkerLoggingInitializer, we replace it with System.getProperty("java.io.tmpdir") making the environmentally-specified temporary directory the default logging location. This part may not be completely safe, in the unlikely case that there is some code depends on this being /tmp/, environmentally-specified temporary directory is something else, and the property isn't user-specified. * In DataflowPipelineRunnerTest, we use actual files created by TemporaryFolder, instead of mock file names. The actual files aren't needed, but that allows us to easily construct valid file names. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83565800 --- .../DataflowWorkerLoggingInitializer.java | 4 +++- .../sdk/runners/DataflowPipelineRunnerTest.java | 17 ++++++++++------- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java index 6be47019e3c86..e89d6d73b94c2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java @@ -18,6 +18,7 @@ import com.google.common.collect.ImmutableBiMap; +import java.io.File; import java.io.IOException; import java.util.logging.ConsoleHandler; import java.util.logging.FileHandler; @@ -37,7 +38,8 @@ * within the systems temporary directory. */ public class DataflowWorkerLoggingInitializer { - private static final String DEFAULT_LOGGING_LOCATION = "/tmp/dataflow-worker.log"; + private static final String DEFAULT_LOGGING_LOCATION = + new File(System.getProperty("java.io.tmpdir"), "dataflow-worker.log").getPath(); private static final String ROOT_LOGGER_NAME = ""; public static final String DATAFLOW_WORKER_LOGGING_LEVEL = "dataflow.worker.logging.level"; public static final String DATAFLOW_WORKER_LOGGING_LOCATION = "dataflow.worker.logging.location"; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java index 38f2fb407c367..86a307e1594c5 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java @@ -53,6 +53,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.ArgumentCaptor; @@ -76,8 +77,10 @@ @RunWith(JUnit4.class) @SuppressWarnings("serial") public class DataflowPipelineRunnerTest { - - @Rule public ExpectedException thrown = ExpectedException.none(); + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + @Rule + public ExpectedException thrown = ExpectedException.none(); // Asserts that the given Job has all expected fields set. private static void assertValidJob(Job job) { @@ -235,14 +238,14 @@ public void runWithDefaultFilesToStage() throws Exception { @Test public void detectClassPathResourceWithFileResources() throws Exception { - String path = "/tmp/file"; - String path2 = "/tmp/file2"; + File file = tmpFolder.newFile("file"); + File file2 = tmpFolder.newFile("file2"); URLClassLoader classLoader = new URLClassLoader(new URL[]{ - new URL("file://" + path), - new URL("file://" + path2) + file.toURI().toURL(), + file2.toURI().toURL() }); - assertEquals(ImmutableList.of(path, path2), + assertEquals(ImmutableList.of(file.getAbsolutePath(), file2.getAbsolutePath()), DataflowPipelineRunner.detectClassPathResourcesToStage(classLoader)); } From a5a2d9e07003f7e5859cb6dec4cc500f2381f2c8 Mon Sep 17 00:00:00 2001 From: davor Date: Thu, 8 Jan 2015 17:06:50 -0800 Subject: [PATCH 0068/1541] Dataflow on Windows: fix testReadEmptyFile in TextReaderTest that depends on "/dev/null". Replacing "/dev/null" with an empty temporary file. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83565917 --- .../cloud/dataflow/sdk/runners/worker/TextReaderTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java index 4ebd671361eda..5d2934c4535fc 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java @@ -78,7 +78,7 @@ private File initTestFile() throws IOException { @Test public void testReadEmptyFile() throws Exception { TextReader textReader = - new TextReader<>("/dev/null", true, null, null, StringUtf8Coder.of()); + new TextReader<>(tmpFolder.newFile().getPath(), true, null, null, StringUtf8Coder.of()); try (Reader.ReaderIterator iterator = textReader.iterator()) { Assert.assertFalse(iterator.hasNext()); } From 19c14c6522d1d48fbe8f4d28f2c054a4b0fb3139 Mon Sep 17 00:00:00 2001 From: davor Date: Thu, 8 Jan 2015 17:22:50 -0800 Subject: [PATCH 0069/1541] Dataflow on Windows: fix issues with hardcoded newline characters. We cannot manually construct strings using "\n" and expect equality, since various APIs will return "\r\n" on various platforms instead. The tests, however, need to expect a combination of '\n' and platform-specific newlines because the entire codebase is not platform-agnostic. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83566994 --- .../DataflowWorkerLoggingFormatter.java | 2 +- .../DataflowWorkerLoggingFormatterTest.java | 28 ++++++++++--------- 2 files changed, 16 insertions(+), 14 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java index a8f54b48668fd..911b3fe2e5e68 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java @@ -54,7 +54,7 @@ public String format(LogRecord record) { + " " + MoreObjects.firstNonNull(MDC.get(MDC_DATAFLOW_WORK_ID), "unknown") + " " + record.getThreadID() + " " + record.getLoggerName() - + " " + record.getMessage() + "\n" + + " " + record.getMessage() + System.lineSeparator() + (exception != null ? exception : ""); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java index 065092aeaab14..4fd2b5f4688d4 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java @@ -40,7 +40,7 @@ public class DataflowWorkerLoggingFormatterTest { public void testWithUnsetValuesInMDC() { assertEquals( "1970-01-01T00:00:00.001Z INFO unknown unknown unknown 2 LoggerName " - + "test.message\n", + + "test.message" + System.lineSeparator(), new DataflowWorkerLoggingFormatter().format( createLogRecord("test.message", null))); } @@ -53,7 +53,7 @@ public void testWithMessage() { "dataflow.workId", "testWorkId")); assertEquals( "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName " - + "test.message\n", + + "test.message" + System.lineSeparator(), new DataflowWorkerLoggingFormatter().format( createLogRecord("test.message", null))); } @@ -66,11 +66,11 @@ public void testWithMessageAndException() { "dataflow.workId", "testWorkId")); assertEquals( "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName " - + "test.message\n" - + "java.lang.Throwable: exception.test.message\n" - + "\tat declaringClass1.method1(file1.java:1)\n" - + "\tat declaringClass2.method2(file2.java:1)\n" - + "\tat declaringClass3.method3(file3.java:1)\n", + + "test.message" + System.lineSeparator() + + "java.lang.Throwable: exception.test.message" + System.lineSeparator() + + "\tat declaringClass1.method1(file1.java:1)" + System.lineSeparator() + + "\tat declaringClass2.method2(file2.java:1)" + System.lineSeparator() + + "\tat declaringClass3.method3(file3.java:1)" + System.lineSeparator(), new DataflowWorkerLoggingFormatter().format( createLogRecord("test.message", createThrowable()))); } @@ -82,11 +82,12 @@ public void testWithException() { "dataflow.workerId", "testWorkerId", "dataflow.workId", "testWorkId")); assertEquals( - "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName null\n" - + "java.lang.Throwable: exception.test.message\n" - + "\tat declaringClass1.method1(file1.java:1)\n" - + "\tat declaringClass2.method2(file2.java:1)\n" - + "\tat declaringClass3.method3(file3.java:1)\n", + "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName null" + + System.lineSeparator() + + "java.lang.Throwable: exception.test.message" + System.lineSeparator() + + "\tat declaringClass1.method1(file1.java:1)" + System.lineSeparator() + + "\tat declaringClass2.method2(file2.java:1)" + System.lineSeparator() + + "\tat declaringClass3.method3(file3.java:1)" + System.lineSeparator(), new DataflowWorkerLoggingFormatter().format( createLogRecord(null, createThrowable()))); } @@ -98,7 +99,8 @@ public void testWithoutExceptionOrMessage() { "dataflow.workerId", "testWorkerId", "dataflow.workId", "testWorkId")); assertEquals( - "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName null\n", + "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName null" + + System.lineSeparator(), new DataflowWorkerLoggingFormatter().format( createLogRecord(null, null))); } From 7598e29c5055d97b3798d17414fc2673f1592843 Mon Sep 17 00:00:00 2001 From: amancuso Date: Thu, 8 Jan 2015 22:22:40 -0800 Subject: [PATCH 0070/1541] Update Dataflow GitHub examples README.MD file to clarify local and cloud mvn options. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83580687 --- README.md | 23 ++++++++++++++--------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/README.md b/README.md index 18f78fc9f99d6..eea690f0a3f44 100644 --- a/README.md +++ b/README.md @@ -81,25 +81,30 @@ You can speed up the build and install process by using the following options: mvn -T 4 install -After building and installing, the following command will execute the WordCount -example using the DirectPipelineRunner on your local machine: +After building and installing, you can execute the Wordcount and other example +pipelines locally or in the cloud using `mvn` with command-line options. + +To execute the Wordcount pipeline locally (using the default +`DirectPipelineRunner`) and write output to a local or +Google Cloud Storage (GCS) location, use the following command-line syntax: mvn exec:java -pl examples \ -Dexec.mainClass=com.google.cloud.dataflow.examples.WordCount \ - -Dexec.args="--input= --output=" + -Dexec.args=" ---output=[ | gs://] If you have been whitelisted for Alpha access to the Dataflow Service and followed the [developer setup](https://cloud.google.com/dataflow/java-sdk/getting-started#DeveloperSetup) -steps, you can use the BlockingDataflowPipelineRunner to run the same program in -the Google Cloud Platform (GCP): +steps, you can use the BlockingDataflowPipelineRunner to execute the Wordcount +program in the Google Cloud Platform (GCP). In this case, you specify +your project name, pipeline runner, the GCS staging location (staging +location should be entered in the form of `gs://bucket/staging-directory`), +and the GCS output (in the form of `gs://bucket/filename_prefix`). mvn exec:java -pl examples \ -Dexec.mainClass=com.google.cloud.dataflow.examples.WordCount \ - -Dexec.args="--project= --stagingLocation= --runner=BlockingDataflowPipelineRunner" + -Dexec.args="--project= --runner=BlockingDataflowPipelineRunner \ + --stagingLocation= --output=" -Google Cloud Storage (GCS) location should be entered in the form of -gs://bucket/path/to/staging/directory. Google Cloud Platform (GCP) project -refers to its name (not number), which has been whitelisted for Cloud Dataflow. Refer [here](https://cloud.google.com/) for instructions to get started with Google Cloud Platform. From adadb61037cf5a0fa1d0cc630334a417c25573b2 Mon Sep 17 00:00:00 2001 From: davor Date: Fri, 9 Jan 2015 11:50:40 -0800 Subject: [PATCH 0071/1541] Dataflow on Windows: fix an issue where the ZIP archive would contain file names with a '\' (backslash) on Windows. Using '/' (slash) is compatible with most platforms. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83623330 --- .../cloud/dataflow/sdk/util/PackageUtil.java | 84 ++++++++++++++----- .../dataflow/sdk/util/PackageUtilTest.java | 9 +- 2 files changed, 68 insertions(+), 25 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java index 46f610836f57d..5afa04a8af1b3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java @@ -16,6 +16,9 @@ package com.google.cloud.dataflow.sdk.util; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.Sleeper; @@ -36,6 +39,7 @@ import java.io.File; import java.io.IOException; +import java.io.OutputStream; import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; import java.nio.charset.StandardCharsets; @@ -253,46 +257,84 @@ private static String computeContentHash(File classpathElement) throws IOExcepti private static void copyContent(String classpathElement, WritableByteChannel outputChannel) throws IOException { final File classpathElementFile = new File(classpathElement); - if (!classpathElementFile.isDirectory()) { + if (classpathElementFile.isDirectory()) { + zipDirectory(classpathElementFile, Channels.newOutputStream(outputChannel)); + } else { Files.asByteSource(classpathElementFile).copyTo(Channels.newOutputStream(outputChannel)); - return; } + } - ZipOutputStream zos = new ZipOutputStream(Channels.newOutputStream(outputChannel)); - zipDirectoryRecursive(classpathElementFile, classpathElementFile, zos); + /** + * Zips an entire directory specified by the path. + * + * @param sourceDirectory the directory to read from. This directory and all + * subdirectories will be added to the zip-file. The path within the zip + * file is relative to the directory given as parameter, not absolute. + * @param outputStream the stream to write the zip-file to. This method does not close + * outputStream. + * @throws IOException the zipping failed, e.g. because the input was not + * readable. + */ + private static void zipDirectory( + File sourceDirectory, + OutputStream outputStream) throws IOException { + checkNotNull(sourceDirectory); + checkNotNull(outputStream); + checkArgument( + sourceDirectory.isDirectory(), + "%s is not a valid directory", + sourceDirectory.getAbsolutePath()); + ZipOutputStream zos = new ZipOutputStream(outputStream); + for (File file : sourceDirectory.listFiles()) { + zipDirectoryInternal(file, "", zos); + } zos.finish(); } /** - * Private helper function for zipping files. This one goes recursively through the input - * directory and all of its subdirectories and adds the single zip entries. + * Private helper function for zipping files. This one goes recursively + * through the input directory and all of its subdirectories and adds the + * single zip entries. * - * @param file the file or directory to be added to the zip file. - * @param root each file uses the root directory to generate its relative path within the zip. - * @param zos the zipstream to write to. - * @throws IOException the zipping failed, e.g. because the output was not writable. + * @param inputFile the file or directory to be added to the zip file + * @param directoryName the string-representation of the parent directory + * name. Might be an empty name, or a name containing multiple directory + * names separated by "/". The directory name must be a valid name + * according to the file system limitations. + * @param zos the zipstream to write to + * @throws IOException the zipping failed, e.g. because the output was not + * writeable. */ - private static void zipDirectoryRecursive(File file, File root, ZipOutputStream zos) - throws IOException { - final String entryName = relativize(file, root); - if (file.isDirectory()) { - // We are hitting a directory. Start the recursion. - // Add the empty entry if it is a subdirectory and the subdirectory has no children. - // Don't add it otherwise, as this is incompatible with certain implementations of unzip. - if (file.list().length == 0 && !file.equals(root)) { + private static void zipDirectoryInternal( + File inputFile, + String directoryName, + ZipOutputStream zos) throws IOException { + final String entryName; + if ("".equals(directoryName)) { + // no parent directories yet. + entryName = inputFile.getName(); + } else { + entryName = directoryName + "/" + inputFile.getName(); + } + if (inputFile.isDirectory()) { + // We are hitting a sub-directory. Start the recursion. + // Add the empty entry for a subdirectory if we have no children files. + // Don't add it if we have them, as this is incompatible with certain + // implementations of unzip. + if (inputFile.list().length == 0) { ZipEntry entry = new ZipEntry(entryName + "/"); zos.putNextEntry(entry); } else { // loop through the directory content, and zip the files - for (File currentFile : file.listFiles()) { - zipDirectoryRecursive(currentFile, root, zos); + for (File file : inputFile.listFiles()) { + zipDirectoryInternal(file, entryName, zos); } } } else { // Put the next zip-entry into the zipoutputstream. ZipEntry entry = new ZipEntry(entryName); zos.putNextEntry(entry); - Files.asByteSource(file).copyTo(zos); + Files.asByteSource(inputFile).copyTo(zos); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java index e49782f6b221b..751824b24282d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/PackageUtilTest.java @@ -16,10 +16,11 @@ package com.google.cloud.dataflow.sdk.util; +import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertThat; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.times; @@ -35,7 +36,6 @@ import com.google.common.io.Files; import com.google.common.io.LineReader; -import org.hamcrest.CoreMatchers; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -210,8 +210,9 @@ public void testPackageUploadWithDirectorySucceeds() throws Exception { entry = inputStream.getNextEntry()) { zipEntryNames.add(entry.getName()); } - assertTrue(CoreMatchers.hasItems("directory/file.txt", "empty_directory/", "file.txt").matches( - zipEntryNames)); + + assertThat(zipEntryNames, + containsInAnyOrder("directory/file.txt", "empty_directory/", "file.txt")); } @Test From 801a95b9e6fff9b167c2a16b73e30543daae0cac Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Fri, 9 Jan 2015 15:57:56 -0800 Subject: [PATCH 0072/1541] Update README.md A few improvements, mostly formatting. --- README.md | 46 +++++++++++++++++++++++----------------------- 1 file changed, 23 insertions(+), 23 deletions(-) diff --git a/README.md b/README.md index eea690f0a3f44..93fbfede73871 100644 --- a/README.md +++ b/README.md @@ -20,29 +20,29 @@ working with you to improve Cloud Dataflow. The key concepts in this programming model are: -* [PCollection](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java): +* [`PCollection`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java): represents a collection of data, which could be bounded or unbounded in size. -* [PTransform](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java): +* [`PTransform`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java): represents a computation that transforms input PCollections into output PCollections. -* [Pipeline](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java): +* [`Pipeline`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java): manages a directed acyclic graph of PTransforms and PCollections, which is ready for execution. -* [PipelineRunner](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java): +* [`PipelineRunner`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java): specifies where and how the pipeline should execute. -Currently there are three PipelineRunners: +Currently there are three `PipelineRunners`: - 1. The [DirectPipelineRunner](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java) + 1. The [`DirectPipelineRunner`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java) runs the pipeline on your local machine. 2. The -[DataflowPipelineRunner](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java) +[`DataflowPipelineRunner`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java) submits the pipeline to the Dataflow Service, where it runs using managed resources in the [Google Cloud Platform](http://cloud.google.com). 3. The -[BlockingDataflowPipelineRunner](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunner.java) -submits the pipeline to the Dataflow Service via the DataflowPipelineRunner and -then prints messages about the job status until execution is complete. +[`BlockingDataflowPipelineRunner`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunner.java) +submits the pipeline to the Dataflow Service via the `DataflowPipelineRunner` +and then prints messages about the job status until execution is complete. _The Dataflow Service is currently in the Alpha phase of development and access is limited to whitelisted users._ @@ -51,9 +51,9 @@ access is limited to whitelisted users._ This repository consists of two modules: -* [Java SDK](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk) +* [`SDK`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk) module provides a set of basic Java APIs to program against. -* [Examples](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples) +* [`Examples`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples) module provides a few samples to get started. We recommend starting with the WordCount example. @@ -69,7 +69,7 @@ You can speed up the build and install process by using the following options: mvn install -DskipTests 2. While iterating on a specific module, use the following command to compile - and reinstall it. For example, to reinstall the 'examples' module, run: + and reinstall it. For example, to reinstall the `examples` module, run: mvn install -pl examples @@ -81,22 +81,22 @@ You can speed up the build and install process by using the following options: mvn -T 4 install -After building and installing, you can execute the Wordcount and other example -pipelines locally or in the cloud using `mvn` with command-line options. +After building and installing, you can execute the `WordCount` and other example +pipelines locally or in the cloud using Maven with command-line options. -To execute the Wordcount pipeline locally (using the default +To execute the WordCount pipeline locally (using the default `DirectPipelineRunner`) and write output to a local or Google Cloud Storage (GCS) location, use the following command-line syntax: mvn exec:java -pl examples \ -Dexec.mainClass=com.google.cloud.dataflow.examples.WordCount \ - -Dexec.args=" ---output=[ | gs://] + -Dexec.args="--output=[ | gs://] If you have been whitelisted for Alpha access to the Dataflow Service and followed the [developer setup](https://cloud.google.com/dataflow/java-sdk/getting-started#DeveloperSetup) -steps, you can use the BlockingDataflowPipelineRunner to execute the Wordcount -program in the Google Cloud Platform (GCP). In this case, you specify -your project name, pipeline runner, the GCS staging location (staging +steps, you can use the `BlockingDataflowPipelineRunner` to execute the +`WordCount` example in the Google Cloud Platform (GCP). In this case, you +specify your project name, pipeline runner, the GCS staging location (staging location should be entered in the form of `gs://bucket/staging-directory`), and the GCS output (in the form of `gs://bucket/filename_prefix`). @@ -108,9 +108,9 @@ and the GCS output (in the form of `gs://bucket/filename_prefix`). Refer [here](https://cloud.google.com/) for instructions to get started with Google Cloud Platform. -Other examples can be run similarly by replacing the WordCount class name with -BigQueryTornadoes, DatastoreWordCount, TfIdf, TopWikipediaSessions, etc. and -adjusting runtime options under the Dexec.args parameter, as specified in the +Other examples can be run similarly by replacing the `WordCount` class name with +`BigQueryTornadoes`, `DatastoreWordCount`, `TfIdf`, `TopWikipediaSessions`, etc. and +adjusting runtime options under the `Dexec.args` parameter, as specified in the example itself. ## More Information From 8e4c7fedf82dc5c40c9464d480ae949bdf041f16 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Mon, 12 Jan 2015 15:33:07 -0800 Subject: [PATCH 0073/1541] Revert "Generalize type signature of KV.of" --- .../main/java/com/google/cloud/dataflow/sdk/values/KV.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java index 81b6806a1125f..febb1302bd8c1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java @@ -35,8 +35,8 @@ public class KV implements Serializable { private static final long serialVersionUID = 0; /** Returns a KV with the given key and value. */ - public static KV of(SK key, SV value) { - return new KV(key, value); + public static KV of(K key, V value) { + return new KV<>(key, value); } /** Returns the key of this KV. */ From 0cc72d62f6a9b4cf9c9f4ad7eb15c7365790fbc5 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Wed, 21 Jan 2015 16:07:11 -0800 Subject: [PATCH 0074/1541] Enable Travis continuous integration coverage. --- .travis.yml | 14 ++++++++++++++ 1 file changed, 14 insertions(+) create mode 100644 .travis.yml diff --git a/.travis.yml b/.travis.yml new file mode 100644 index 0000000000000..9dc6a4497a094 --- /dev/null +++ b/.travis.yml @@ -0,0 +1,14 @@ +language: java + +jdk: + - oraclejdk8 + - oraclejdk7 + - openjdk7 + - openjdk6 + +install: + - mvn install clean -U -DskipTests=true + +script: + - mvn -U verify + From 6842469f9f6b80d06a59b81fa8a0870845b1c05b Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Wed, 21 Jan 2015 16:24:18 -0800 Subject: [PATCH 0075/1541] Disable continuous integration coverage on openjdk6, as we don't support it. --- .travis.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 9dc6a4497a094..4922b290ff69e 100644 --- a/.travis.yml +++ b/.travis.yml @@ -4,7 +4,6 @@ jdk: - oraclejdk8 - oraclejdk7 - openjdk7 - - openjdk6 install: - mvn install clean -U -DskipTests=true From f255ed5fe3b07f4a41847324e7b3eb1ca841dbd1 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Thu, 22 Jan 2015 13:31:53 -0800 Subject: [PATCH 0076/1541] Update README.md Display the build status as reported by Travis continuous integration. --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 55629a751ca9f..3e6bb3136a548 100644 --- a/README.md +++ b/README.md @@ -4,7 +4,7 @@ provides a simple, powerful programming model for building both batch and streaming parallel data processing pipelines. -## Status +## Status [![Build Status](https://travis-ci.org/GoogleCloudPlatform/DataflowJavaSDK.svg?branch=master)](https://travis-ci.org/GoogleCloudPlatform/DataflowJavaSDK) The Cloud Dataflow SDK is used to access the Google Cloud Dataflow service, which is currently in Alpha and restricted to whitelisted users. From 9623f20d5e4ae9db781bb82957175eec901e6eb3 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Thu, 22 Jan 2015 15:27:35 -0800 Subject: [PATCH 0077/1541] Update .travis.yml Enable email notification of build status. --- .travis.yml | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 4922b290ff69e..8cdc4ca197fe3 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,5 +1,12 @@ language: java +notifications: + email: + recipients: + - dataflow-sdk+travis@google.com + on_success: change + on_failure: always + jdk: - oraclejdk8 - oraclejdk7 @@ -9,5 +16,4 @@ install: - mvn install clean -U -DskipTests=true script: - - mvn -U verify - + - mvn verify -U From 4549ebcbcfef60eba3f8db1c3acea0ac80040b47 Mon Sep 17 00:00:00 2001 From: kirpichov Date: Fri, 9 Jan 2015 14:05:06 -0800 Subject: [PATCH 0078/1541] * High-level API for defining basic custom sources (doesn't support any of the advanced features, like bases and multi-level splitting): see the ReadSource and Source classes. Note that Source.Reader implements the advance/getCurrent model rather than the Java collection hasNext/next model, for reasons outlined in its documentation. * DatastoreIO rewritten using this API. * Propagates PipelineOptions through the worker harness to SourceFormat (renamed from CustomSourceFormat) constructor, so that options such as credentials are available during splitting. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83633581 --- .../dataflow/examples/DatastoreWordCount.java | 6 +- sdk/pom.xml | 6 + .../cloud/dataflow/sdk/io/DatastoreIO.java | 781 ++++++++++-------- .../dataflow/sdk/io/DatastoreIterator.java | 141 ---- .../cloud/dataflow/sdk/io/ReadSource.java | 123 +++ .../google/cloud/dataflow/sdk/io/Source.java | 174 ++++ .../runners/DataflowPipelineTranslator.java | 7 +- .../BasicSerializableSourceFormat.java | 277 +++++++ .../dataflow/DatastoreIOTranslator.java | 4 +- .../dataflow/ReadSourceTranslator.java | 32 + .../sdk/runners/worker/DataflowWorker.java | 18 +- ...tFactory.java => SourceFormatFactory.java} | 16 +- .../worker/SourceOperationExecutor.java | 15 +- .../SourceOperationExecutorFactory.java | 7 +- .../worker/SourceTranslationUtils.java | 39 +- .../dataflow/sdk/util/CloudSourceUtils.java | 6 +- .../dataflow/sdk/util/PropertyNames.java | 14 +- .../dataflow/sdk/util/TestCredential.java | 4 +- .../sdk/util/common/worker/ReadOperation.java | 17 +- .../sdk/util/common/worker/Reader.java | 6 +- ...tomSourceFormat.java => SourceFormat.java} | 2 +- .../dataflow/sdk/io/DatastoreIOTest.java | 101 ++- .../BasicSerializableSourceFormatTest.java | 218 +++++ .../runners/worker/BigQueryReaderTest.java | 2 +- .../runners/worker/DatastoreReaderTest.java | 143 ++++ 25 files changed, 1571 insertions(+), 588 deletions(-) delete mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIterator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ReadSource.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormat.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/ReadSourceTranslator.java rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{CustomSourceFormatFactory.java => SourceFormatFactory.java} (71%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/{CustomSourceFormat.java => SourceFormat.java} (98%) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormatTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DatastoreReaderTest.java diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/DatastoreWordCount.java b/examples/src/main/java/com/google/cloud/dataflow/examples/DatastoreWordCount.java index 1e00589281aad..ab31aae15ab76 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/DatastoreWordCount.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/DatastoreWordCount.java @@ -148,7 +148,7 @@ public static void writeDataToDatastore(Options options) { Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) .apply(ParDo.of(new CreateEntityFn(options.getKind()))) - .apply(DatastoreIO.Write.to(options.getDataset())); + .apply(DatastoreIO.write().to(options.getDataset())); p.run(); } @@ -163,8 +163,8 @@ public static void readDataFromDatastore(Options options) { Query query = q.build(); Pipeline p = Pipeline.create(options); - p.apply(DatastoreIO.Read.named("ReadShakespeareFromDatastore") - .from(options.getDataset(), query)) + p.apply(DatastoreIO.readFrom(options.getDataset(), query) + .named("ReadShakespeareFromDatastore")) .apply(ParDo.of(new GetContentFn())) .apply(new WordCount.CountWords()) .apply(TextIO.Write.named("WriteLines").to(options.getOutput())); diff --git a/sdk/pom.xml b/sdk/pom.xml index 1e9633695d10d..c761197a78b3d 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -270,6 +270,12 @@ + + com.google.http-client + google-http-client + 1.19.0 + + com.google.oauth-client google-oauth-client-java6 diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java index 1b66b868c97ed..dfd5041720dd2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java @@ -16,7 +16,16 @@ package com.google.cloud.dataflow.sdk.io; +import static com.google.api.services.datastore.DatastoreV1.PropertyFilter.Operator.EQUAL; +import static com.google.api.services.datastore.DatastoreV1.PropertyOrder.Direction.DESCENDING; +import static com.google.api.services.datastore.DatastoreV1.QueryResultBatch.MoreResultsType.NOT_FINISHED; +import static com.google.api.services.datastore.client.DatastoreHelper.getPropertyMap; +import static com.google.api.services.datastore.client.DatastoreHelper.makeFilter; +import static com.google.api.services.datastore.client.DatastoreHelper.makeOrder; +import static com.google.api.services.datastore.client.DatastoreHelper.makeValue; + import com.google.api.client.auth.oauth2.Credential; +import com.google.api.services.datastore.DatastoreV1; import com.google.api.services.datastore.DatastoreV1.BeginTransactionRequest; import com.google.api.services.datastore.DatastoreV1.BeginTransactionResponse; import com.google.api.services.datastore.DatastoreV1.CommitRequest; @@ -27,34 +36,32 @@ import com.google.api.services.datastore.client.DatastoreFactory; import com.google.api.services.datastore.client.DatastoreHelper; import com.google.api.services.datastore.client.DatastoreOptions; -import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.api.services.datastore.client.QuerySplitter; import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.coders.DefaultCoder; import com.google.cloud.dataflow.sdk.coders.EntityCoder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; import com.google.cloud.dataflow.sdk.options.GcpOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; -import com.google.cloud.dataflow.sdk.transforms.Create; -import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.Flatten; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.util.Credentials; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.RetryHttpRequestInitializer; -import com.google.cloud.dataflow.sdk.values.PBegin; import com.google.cloud.dataflow.sdk.values.PCollection; -import com.google.cloud.dataflow.sdk.values.PCollectionList; import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.security.GeneralSecurityException; import java.util.ArrayList; import java.util.HashMap; -import java.util.LinkedList; import java.util.List; +import java.util.NoSuchElementException; /** * Transforms for reading and writing @@ -84,10 +91,10 @@ * cache is keyed by the requested scopes. * *

To read a {@link PCollection} from a query to Datastore, use - * {@link DatastoreIO.Read}, specifying {@link DatastoreIO.Read#from} to specify - * dataset to read, the query to read from, and optionally - * {@link DatastoreIO.Read#named} and {@link DatastoreIO.Read#withHost} to specify - * the name of the pipeline step and the host of Datastore, respectively. + * {@link DatastoreIO#read} and its methods {#link DatastoreIO.Read#withDataset} + * and {#link DatastoreIO.Read#withQuery} to specify dataset to read, the query + * to read from, and optionally {@link DatastoreIO.Source#withHost} to specify + * the host of Datastore. * For example: * *

 {@code
@@ -95,16 +102,27 @@
  * PipelineOptions options =
  *     PipelineOptionsFactory.fromArgs(args).create();
  * Pipeline p = Pipeline.create(options);
- * PCollection entities =
- *     p.apply(DatastoreIO.Read
- *             .named("Read Datastore")
- *             .from(datasetId, query)
- *             .withHost(host));
+ * PCollection entities = p.apply(
+ *     ReadSource.from(DatastoreIO.read()
+ *         .withDataset(datasetId)
+ *         .withQuery(query)
+ *         .withHost(host)));
+ * p.run();
+ * } 
+ * + * or: + * + *
 {@code
+ * // Read a query from Datastore
+ * PipelineOptions options =
+ *     PipelineOptionsFactory.fromArgs(args).create();
+ * Pipeline p = Pipeline.create(options);
+ * PCollection entities = p.apply(DatastoreIO.readFrom(datasetId, query));
  * p.run();
  * } 
* *

To write a {@link PCollection} to a datastore, use - * {@link DatastoreIO.Write}, specifying {@link DatastoreIO.Write#to} to specify + * {@link DatastoreIO.Sink}, specifying {@link DatastoreIO.Sink#to} to specify * the datastore to write to, and optionally {@link TextIO.Write#named} to specify * the name of the pipeline step. For example: * @@ -119,427 +137,353 @@ */ public class DatastoreIO { - private static final Logger LOG = LoggerFactory.getLogger(DatastoreIO.class); private static final String DEFAULT_HOST = "https://www.googleapis.com"; /** - * A PTransform that reads from a Datastore query and returns a - * {@code PCollection} containing each of the rows of the table. + * Returns an empty {@code DatastoreIO.Read} builder with the default host. + * You'll need to configure the dataset and query using {@link DatastoreIO.Source#withDataset} + * and {@link DatastoreIO.Source#withQuery}. */ - public static class Read { + public static Source read() { + return new Source(DEFAULT_HOST, null, null); + } - /** - * Returns a DatastoreIO.Read PTransform with the given step name. - */ - public static Bound named(String name) { - return new Bound(DEFAULT_HOST).named(name); - } + /** + * Returns a {@code PTransform} which reads Datastore entities from the query + * against the given dataset. + */ + public static ReadSource.Bound readFrom(String datasetId, Query query) { + return ReadSource.from(new Source(DEFAULT_HOST, datasetId, query)); + } - /** - * Reads entities retrieved from the dataset and a given query. - */ - public static Bound from(String datasetId, Query query) { - return new Bound(DEFAULT_HOST).from(datasetId, query); + /** + * Returns a {@code PTransform} which reads Datastore entities from the query + * against the given dataset and host. + */ + public static ReadSource.Bound readFrom(String host, String datasetId, Query query) { + return ReadSource.from(new Source(host, datasetId, query)); + } + + /** + * A source that reads the result rows of a Datastore query as {@code Entity} objects. + */ + @SuppressWarnings("serial") + public static class Source extends com.google.cloud.dataflow.sdk.io.Source { + String host; + String datasetId; + Query query; + /** For testing only. */ + private QuerySplitter mockSplitter; + private Supplier mockEstimateSizeBytes; + + private Source(String host, String datasetId, Query query) { + this.host = host; + this.datasetId = datasetId; + this.query = query; } - /** - * Returns a DatastoreIO.Read PTransform with specified host. - */ - public static Bound withHost(String host) { - return new Bound(host); + public Source withDataset(String datasetId) { + return new Source(host, datasetId, query); } - /** - * A PTransform that reads from a Datastore query and returns a bounded - * {@code PCollection}. - */ - @SuppressWarnings("serial") - public static class Bound extends PTransform> { - String host; - String datasetId; - Query query; - - /** - * Returns a DatastoreIO.Bound object with given query. - * Sets the name, Datastore host, datasetId, query associated - * with this PTransform, and options for this Pipeline. - */ - Bound(String name, String host, String datasetId, Query query) { - super(name); - this.host = host; - this.datasetId = datasetId; - this.query = query; - } + public Source withQuery(Query query) { + return new Source(host, datasetId, query); + } - /** - * Returns a DatastoreIO.Read PTransform with host set up. - */ - Bound(String host) { - this.host = host; - } + public Source withHost(String host) { + return new Source(host, datasetId, query); + } - /** - * Returns a new DatastoreIO.Read PTransform with the name - * associated with this transformation. - */ - public Bound named(String name) { - return new Bound(name, host, datasetId, query); - } + @Override + public Coder getDefaultOutputCoder() { + return EntityCoder.of(); + } - /** - * Returns a new DatastoreIO.Read PTransform with datasetId, - * and query associated with this transformation, and options - * associated with this Pipleine. - */ - public Bound from(String datasetId, Query query) { - return new Bound(name, host, datasetId, query); + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + // Datastore provides no way to get a good estimate of how large the result of a query + // will be. As a rough approximation, we attempt to fetch the statistics of the whole + // entity kind being queried, using the __Stat_Kind__ system table, assuming exactly 1 kind + // is specified in the query. + if (mockEstimateSizeBytes != null) { + return mockEstimateSizeBytes.get(); } - /** - * Returns a new DatastoreIO.Read PTransform with the host - * specified. - */ - public Bound withHost(String host) { - return new Bound(name, host, datasetId, query); + Datastore datastore = getDatastore(options); + if (query.getKindCount() != 1) { + throw new UnsupportedOperationException( + "Can only estimate size for queries specifying exactly 1 kind"); } - - @Override - public PCollection apply(PBegin input) { - if (datasetId == null || query == null) { - throw new IllegalStateException( - "need to set datasetId, and query " - + "of a DatastoreIO.Read transform"); - } - - QueryOptions queryOptions = QueryOptions.create(host, datasetId, query); - PCollection output; - try { - DataflowPipelineOptions options = - getPipeline().getOptions().as(DataflowPipelineOptions.class); - PCollection queries = splitQueryOptions(queryOptions, options, input); - - output = queries.apply(ParDo.of(new ReadEntitiesFn())); - getCoderRegistry().registerCoder(Entity.class, EntityCoder.class); - } catch (DatastoreException e) { - LOG.warn("DatastoreException: error while doing Datastore query splitting.", e); - throw new RuntimeException("Error while splitting Datastore query."); - } - - return output; + String ourKind = query.getKind(0).getName(); + long latestTimestamp = queryLatestStatisticsTimestamp(datastore); + Query.Builder query = Query.newBuilder(); + query.addKindBuilder().setName("__Stat_Kind__"); + query.setFilter(makeFilter( + makeFilter("kind_name", EQUAL, makeValue(ourKind)).build(), + makeFilter("timestamp", EQUAL, makeValue(latestTimestamp)).build())); + DatastoreV1.RunQueryRequest request = + DatastoreV1.RunQueryRequest.newBuilder().setQuery(query).build(); + + long now = System.currentTimeMillis(); + DatastoreV1.RunQueryResponse response = datastore.runQuery(request); + LOG.info("Query for per-kind statistics took " + (System.currentTimeMillis() - now) + "ms"); + + DatastoreV1.QueryResultBatch batch = response.getBatch(); + if (batch.getEntityResultCount() == 0) { + throw new NoSuchElementException( + "Datastore statistics for kind " + ourKind + " unavailable"); } - } - } - - ///////////////////// Write Class ///////////////////////////////// - /** - * A {@link PTransform} that writes a {@code PCollection} containing - * entities to a Datastore kind. - * - *

Current version only supports Write operation running on - * {@link DirectPipelineRunner}. If Write is used on {@link DataflowPipelineRunner}, - * it throws {@link UnsupportedOperationException} and won't continue on the - * operation. - * - */ - public static class Write { - /** - * Returns a DatastoreIO.Write PTransform with the name - * associated with this PTransform. - */ - public static Bound named(String name) { - return new Bound(DEFAULT_HOST).named(name); + Entity entity = batch.getEntityResult(0).getEntity(); + return getPropertyMap(entity).get("entity_bytes").getIntegerValue(); } /** - * Returns a DatastoreIO.Write PTransform with given datasetId. + * Datastore system tables with statistics are periodically updated. This method fetches + * the latest timestamp of statistics update using the __Stat_Total__ table. */ - public static Bound to(String datasetId) { - return new Bound(DEFAULT_HOST).to(datasetId); + private long queryLatestStatisticsTimestamp(Datastore datastore) throws DatastoreException { + Query.Builder query = Query.newBuilder(); + query.addKindBuilder().setName("__Stat_Total__"); + query.addOrder(makeOrder("timestamp", DESCENDING)); + query.setLimit(1); + DatastoreV1.RunQueryRequest request = + DatastoreV1.RunQueryRequest.newBuilder().setQuery(query).build(); + + long now = System.currentTimeMillis(); + DatastoreV1.RunQueryResponse response = datastore.runQuery(request); + LOG.info("Query for latest stats timestamp of dataset " + datasetId + " took " + + (System.currentTimeMillis() - now) + "ms"); + DatastoreV1.QueryResultBatch batch = response.getBatch(); + if (batch.getEntityResultCount() == 0) { + throw new NoSuchElementException( + "Datastore total statistics for dataset " + datasetId + " unavailable"); + } + Entity entity = batch.getEntityResult(0).getEntity(); + return getPropertyMap(entity).get("timestamp").getTimestampMicrosecondsValue(); } - /** - * Returns a DatastoreIO.Write PTransform with specified host. - */ - public static Bound withHost(String host) { - return new Bound(host); + @Override + public boolean producesSortedKeys(PipelineOptions options) { + // TODO: Perhaps this can be implemented by inspecting the query. + return false; } - /** - * A PTransform that writes a bounded {@code PCollection} - * to a Datastore. - */ - @SuppressWarnings("serial") - public static class Bound extends PTransform, PDone> { - String host; - String datasetId; - - /** - * Returns a DatastoreIO.Write PTransform with given host. - */ - Bound(String host) { - this.host = host; + @Override + public List splitIntoShards(long desiredShardSizeBytes, PipelineOptions options) + throws Exception { + DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); + long numSplits; + try { + numSplits = getEstimatedSizeBytes(options) / desiredShardSizeBytes; + } catch (Exception e) { + LOG.warn("Estimated size unavailable, using number of workers", e); + // Fallback in case estimated size is unavailable. + numSplits = dataflowOptions.getNumWorkers(); } - - /** - * Returns a DatastoreIO.Write.Bound object. - * Sets the name, datastore agent, and kind associated - * with this transformation. - */ - Bound(String name, String host, String datasetId) { - super(name); - this.host = host; - this.datasetId = datasetId; + List splitQueries; + if (mockSplitter == null) { + splitQueries = DatastoreHelper.getQuerySplitter().getSplits( + query, (int) numSplits, getUserDatastore(host, datasetId, options)); + } else { + splitQueries = mockSplitter.getSplits(query, (int) numSplits, null); } - - /** - * Returns a DatastoreIO.Write PTransform with the name - * associated with this PTransform. - */ - public Bound named(String name) { - return new Bound(name, host, datasetId); + List res = new ArrayList<>(); + for (Query splitQuery : splitQueries) { + res.add(new Source(host, datasetId, splitQuery)); } + return res; + } - /** - * Returns a DatastoreIO.Write PTransform with given datasetId. - */ - public Bound to(String datasetId) { - return new Bound(name, host, datasetId); + @Override + public Reader createBasicReader( + PipelineOptions pipelineOptions, Coder coder, ExecutionContext executionContext) + throws IOException { + try { + return new DatastoreReader(query, getDatastore(pipelineOptions)); + } catch (GeneralSecurityException e) { + throw new IOException(e); } + } - /** - * Returns a new DatastoreIO.Write PTransform with specified host. - */ - public Bound withHost(String host) { - return new Bound(name, host, datasetId); + @Override + public void validate() { + Preconditions.checkNotNull(host, "host"); + Preconditions.checkNotNull(query, "query"); + Preconditions.checkNotNull(datasetId, "datasetId"); + } + + private Datastore getDatastore(PipelineOptions pipelineOptions) + throws IOException, GeneralSecurityException { + Datastore datastore = getUserDatastore(host, datasetId, pipelineOptions); + if (datastore == null) { + datastore = getWorkerDatastore(host, datasetId, pipelineOptions); } + return datastore; + } - @Override - public PDone apply(PCollection input) { - if (this.host == null || this.datasetId == null) { - throw new IllegalStateException( - "need to set Datastore host and dataasetId" - + "of a DatastoreIO.Write transform"); - } + /** For testing only. */ + Source withMockSplitter(QuerySplitter splitter) { + Source res = new Source(host, datasetId, query); + res.mockSplitter = splitter; + res.mockEstimateSizeBytes = mockEstimateSizeBytes; + return res; + } - return new PDone(); - } + /** For testing only. */ + public Source withMockEstimateSizeBytes(Supplier estimateSizeBytes) { + Source res = new Source(host, datasetId, query); + res.mockSplitter = mockSplitter; + res.mockEstimateSizeBytes = estimateSizeBytes; + return res; + } + } - @Override - protected String getKindString() { return "DatastoreIO.Write"; } + public static Datastore getWorkerDatastore( + String host, String datasetId, PipelineOptions options) { + DatastoreOptions.Builder builder = + new DatastoreOptions.Builder().host(host).dataset(datasetId).initializer( + new RetryHttpRequestInitializer(null)); - @Override - protected Coder getDefaultOutputCoder() { - return VoidCoder.of(); - } + try { + Credential credential = + Credentials.getWorkerCredential(options.as(DataflowPipelineOptions.class)); + builder.credential(credential); + } catch (IOException e) { + LOG.warn("IOException: can't get credential for worker.", e); + throw new RuntimeException("Failed on getting credential for worker."); + } + return DatastoreFactory.get().create(builder.build()); + } - static { - DirectPipelineRunner.registerDefaultTransformEvaluator( - Bound.class, - new DirectPipelineRunner.TransformEvaluator() { - @Override - public void evaluate( - Bound transform, - DirectPipelineRunner.EvaluationContext context) { - evaluateWriteHelper(transform, context); - } - }); - } + public static Datastore getUserDatastore(String host, String datasetId, PipelineOptions options) + throws IOException, GeneralSecurityException { + DatastoreOptions.Builder builder = + new DatastoreOptions.Builder().host(host).dataset(datasetId).initializer( + new RetryHttpRequestInitializer(null)); + + Credential credential = Credentials.getUserCredential(options.as(GcpOptions.class)); + if (credential != null) { + builder.credential(credential); } + return DatastoreFactory.get().create(builder.build()); } - /////////////////////////////////////////////////////////////////// + ///////////////////// Write Class ///////////////////////////////// /** - * A DoFn that performs query request to Datastore and converts - * each QueryOptions into Entities. + * Returns a new {@link DatastoreIO.Sink} builder using the default host. + * You need to further configure it using {@link DatastoreIO.Sink#named}, + * {@link DatastoreIO.Sink#to}, and optionally {@link DatastoreIO.Sink#withHost}. */ - @SuppressWarnings("serial") - private static class ReadEntitiesFn extends DoFn { - @Override - public void processElement(ProcessContext c) { - Query query = c.element().getQuery(); - Datastore datastore = c.element().getWorkerDatastore( - c.getPipelineOptions().as(GcpOptions.class)); - DatastoreIterator entityIterator = new DatastoreIterator(query, datastore); - - while (entityIterator.hasNext()) { - c.output(entityIterator.next().getEntity()); - } - } + public static Sink write() { + return new Sink(DEFAULT_HOST); } /** - * A class that stores query and datastore setup environments - * (host and datasetId). + * Returns a new {@link DatastoreIO.Sink} builder using the default host and given dataset. + * You need to further configure it using {@link DatastoreIO.Sink#named}, + * and optionally {@link DatastoreIO.Sink#withHost}. */ - @DefaultCoder(AvroCoder.class) - private static class QueryOptions { - // Query to read in byte array. - public byte[] byteQuery; - - // Datastore host to read from. - public String host; - - // Datastore dataset ID to read from. - public String datasetId; + public static Sink writeTo(String datasetId) { + return write().to(datasetId); + } - @SuppressWarnings("unused") - QueryOptions() {} + /** + * A {@link PTransform} that writes a {@code PCollection} containing + * entities to a Datastore kind. + * + *

Current version only supports Write operation running on + * {@link DirectPipelineRunner}. If Write is used on {@link DataflowPipelineRunner}, + * it throws {@link UnsupportedOperationException} and won't continue on the + * operation. + * + */ + @SuppressWarnings("serial") + public static class Sink extends PTransform, PDone> { + String host; + String datasetId; /** - * Returns a QueryOption object without account and private key file - * (for supporting query on local Datastore). - * - * @param host the host of Datastore to connect - * @param datasetId the dataset ID of Datastore to query - * @param query the query to perform + * Returns a DatastoreIO.Write PTransform with given host. */ - QueryOptions(String host, String datasetId, Query query) { + Sink(String host) { this.host = host; - this.datasetId = datasetId; - this.setQuery(query); } /** - * Creates and returns a QueryOption object for query on local Datastore. - * - * @param host the host of Datastore to connect - * @param datasetId the dataset ID of Datastore to query - * @param query the query to perform + * Returns a DatastoreIO.Write.Bound object. + * Sets the name, datastore agent, and kind associated + * with this transformation. */ - public static QueryOptions create(String host, String datasetId, Query query) { - return new QueryOptions(host, datasetId, query); + Sink(String name, String host, String datasetId) { + super(name); + this.host = host; + this.datasetId = datasetId; } /** - * Sets up a query. - * Stores query in a byte array so that we can use AvroCoder to encode/decode - * QueryOptions. - * - * @param q the query to be addressed + * Returns a DatastoreIO.Write PTransform with the name + * associated with this PTransform. */ - public void setQuery(Query q) { - this.byteQuery = q.toByteArray(); + public Sink named(String name) { + return new Sink(name, host, datasetId); } /** - * Returns query. - * - * @return query in this option. + * Returns a DatastoreIO.Write PTransform with given datasetId. */ - public Query getQuery() { - try { - return Query.parseFrom(this.byteQuery); - } catch (IOException e) { - LOG.warn("IOException: parsing query failed.", e); - throw new RuntimeException("Cannot parse query from byte array."); - } + public Sink to(String datasetId) { + return new Sink(name, host, datasetId); } /** - * Returns the dataset ID. - * - * @return a dataset ID string for Datastore. + * Returns a new DatastoreIO.Write PTransform with specified host. */ - public String getDatasetId() { - return this.datasetId; + public Sink withHost(String host) { + return new Sink(name, host, datasetId); } - /** - * Returns a copy of QueryOptions from current options with given query. - * - * @param query a new query to be set - * @return A QueryOptions object for query - */ - public QueryOptions newQuery(Query query) { - return create(host, datasetId, query); - } + @Override + public PDone apply(PCollection input) { + if (this.host == null || this.datasetId == null) { + throw new IllegalStateException("need to set Datastore host and dataasetId" + + "of a DatastoreIO.Write transform"); + } - /** - * Returns a Datastore object for connecting to Datastore on workers. - * This method will try to get worker credential from Credentials - * library and constructs a Datastore object which is set up and - * ready to communicate with Datastore. - * - * @return a Datastore object setup with host and dataset. - */ - public Datastore getWorkerDatastore(GcpOptions options) { - DatastoreOptions.Builder builder = new DatastoreOptions.Builder() - .host(this.host) - .dataset(this.datasetId) - .initializer(new RetryHttpRequestInitializer(null)); + return new PDone(); + } - try { - Credential credential = Credentials.getWorkerCredential(options); - builder.credential(credential); - } catch (IOException e) { - LOG.warn("IOException: can't get credential for worker.", e); - throw new RuntimeException("Failed on getting credential for worker."); - } - return DatastoreFactory.get().create(builder.build()); + @Override + protected String getKindString() { + return "DatastoreIO.Write"; } - /** - * Returns a Datastore object for connecting to Datastore for users. - * This method will use the passed in credentials and construct a Datastore - * object which is set up and ready to communicate with Datastore. - * - * @return a Datastore object setup with host and dataset. - */ - public Datastore getUserDatastore(GcpOptions options) { - DatastoreOptions.Builder builder = new DatastoreOptions.Builder() - .host(this.host) - .dataset(this.datasetId) - .initializer(new RetryHttpRequestInitializer(null)); - - Credential credential = options.getGcpCredential(); - if (credential != null) { - builder.credential(credential); - } - return DatastoreFactory.get().create(builder.build()); + @Override + protected Coder getDefaultOutputCoder() { + return VoidCoder.of(); } - } - /** - * Returns a list of QueryOptions by splitting a QueryOptions into sub-queries. - * This method leverages the QuerySplitter in Datastore to split the - * query into sub-queries for further parallel query in Dataflow service. - * - * @return a PCollection of QueryOptions for split queries - */ - private static PCollection splitQueryOptions( - QueryOptions queryOptions, DataflowPipelineOptions options, - PBegin input) - throws DatastoreException { - Query query = queryOptions.getQuery(); - Datastore datastore = queryOptions.getUserDatastore(options); - - // Get splits from the QuerySplit interface. - List splitQueries = DatastoreHelper.getQuerySplitter() - .getSplits(query, options.getNumWorkers(), datastore); - - List> queryList = new LinkedList<>(); - for (Query q : splitQueries) { - PCollection newQuery = input - .apply(Create.of(queryOptions.newQuery(q))); - queryList.add(newQuery); - } - - // This is a workaround to allow for parallelism of a small collection. - return PCollectionList.of(queryList) - .apply(Flatten.create()); + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + Sink.class, new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate( + Sink transform, DirectPipelineRunner.EvaluationContext context) { + evaluateWriteHelper(transform, context); + } + }); + } } - ///////////////////////////////////////////////////////////////////// + /////////////////////////////////////////////////////////////////// /** * Direct mode write evaluator. * This writes the result to Datastore. */ private static void evaluateWriteHelper( - Write.Bound transform, - DirectPipelineRunner.EvaluationContext context) { + Sink transform, DirectPipelineRunner.EvaluationContext context) { LOG.info("Writing to Datastore"); GcpOptions options = context.getPipelineOptions(); Credential credential = options.getGcpCredential(); @@ -556,8 +500,8 @@ private static void evaluateWriteHelper( // Create a map to put entities with same ancestor for writing in a batch. HashMap> map = new HashMap<>(); for (Entity e : entityList) { - String keyOfAncestor = e.getKey().getPathElement(0).getKind() - + e.getKey().getPathElement(0).getName(); + String keyOfAncestor = + e.getKey().getPathElement(0).getKind() + e.getKey().getPathElement(0).getName(); List value = map.get(keyOfAncestor); if (value == null) { value = new ArrayList<>(); @@ -603,4 +547,127 @@ private static void writeBatch(List listOfEntities, Datastore datastore) throw new RuntimeException("Datastore exception", e); } } + + /** + * An iterator over the records from a query of the datastore. + * + *

Usage: + *

{@code
+   *   DatastoreIterator iterator = new DatastoreIterator(query, datastore);
+   *   while (iterator.advance()) {
+   *     Entity e = iterator.getCurrent();
+   *     ...
+   *   }
+   * }
+ */ + public static class DatastoreReader + implements Source.Reader { + /** + * Query to select records. + */ + private Query.Builder query; + + /** + * Datastore to read from. + */ + private Datastore datastore; + + /** + * True if more results may be available. + */ + private boolean moreResults; + + /** + * Iterator over records. + */ + private java.util.Iterator entities; + + /** + * Current batch of query results. + */ + private DatastoreV1.QueryResultBatch currentBatch; + + /** + * Maximum number of results to request per query. + * + *

Must be set, or it may result in an I/O error when querying + * Cloud Datastore. + */ + private static final int QUERY_LIMIT = 5000; + + private Entity currentEntity; + + /** + * Returns a DatastoreIterator with query and Datastore object set. + * + * @param query the query to select records. + * @param datastore a datastore connection to use. + */ + public DatastoreReader(Query query, Datastore datastore) { + this.query = query.toBuilder().clone(); + this.datastore = datastore; + this.query.setLimit(QUERY_LIMIT); + } + + @Override + public Entity getCurrent() { + return currentEntity; + } + + @Override + public boolean advance() throws IOException { + if (entities == null || (!entities.hasNext() && moreResults)) { + try { + entities = getIteratorAndMoveCursor(); + } catch (DatastoreException e) { + throw new IOException(e); + } + } + + if (entities == null || !entities.hasNext()) { + currentEntity = null; + return false; + } + + currentEntity = entities.next().getEntity(); + return true; + } + + @Override + public void close() throws IOException { + // Nothing + } + + /** + * Returns an iterator over the next batch of records for the query + * and updates the cursor to get the next batch as needed. + * Query has specified limit and offset from InputSplit. + */ + private java.util.Iterator getIteratorAndMoveCursor() + throws DatastoreException { + if (currentBatch != null && currentBatch.hasEndCursor()) { + query.setStartCursor(currentBatch.getEndCursor()); + } + + DatastoreV1.RunQueryRequest request = + DatastoreV1.RunQueryRequest.newBuilder().setQuery(query).build(); + DatastoreV1.RunQueryResponse response = datastore.runQuery(request); + + currentBatch = response.getBatch(); + + // MORE_RESULTS_AFTER_LIMIT is not implemented yet: + // https://groups.google.com/forum/#!topic/gcd-discuss/iNs6M1jA2Vw, so + // use result count to determine if more results might exist. + int numFetch = currentBatch.getEntityResultCount(); + moreResults = (numFetch == QUERY_LIMIT) || (currentBatch.getMoreResults() == NOT_FINISHED); + + // May receive a batch of 0 results if the number of records is a multiple + // of the request limit. + if (numFetch == 0) { + return null; + } + + return currentBatch.getEntityResultList().iterator(); + } + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIterator.java deleted file mode 100644 index 1b6d92e73c76a..0000000000000 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIterator.java +++ /dev/null @@ -1,141 +0,0 @@ -/* - * Copyright (C) 2014 Google Inc. - * - * Licensed 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 com.google.cloud.dataflow.sdk.io; - -import com.google.api.services.datastore.DatastoreV1.EntityResult; -import com.google.api.services.datastore.DatastoreV1.Query; -import com.google.api.services.datastore.DatastoreV1.QueryResultBatch; -import com.google.api.services.datastore.DatastoreV1.RunQueryRequest; -import com.google.api.services.datastore.DatastoreV1.RunQueryResponse; -import com.google.api.services.datastore.client.Datastore; -import com.google.api.services.datastore.client.DatastoreException; -import com.google.common.collect.AbstractIterator; - -import java.util.Iterator; - -/** - * An iterator over the records from a query of the datastore. - * - *

Usage: - *

{@code
- *   // Need to pass query and datastore object.
- *   DatastoreIterator iterator = new DatastoreIterator(query, datastore);
- *   while (iterator.hasNext()) {
- *     Entity e = iterator.next().getEntity();
- *     ...
- *   }
- * }
- */ -class DatastoreIterator extends AbstractIterator { - /** - * Query to select records. - */ - private Query.Builder query; - - /** - * Datastore to read from. - */ - private Datastore datastore; - - /** - * True if more results may be available. - */ - private boolean moreResults; - - /** - * Iterator over records. - */ - private Iterator entities; - - /** - * Current batch of query results. - */ - private QueryResultBatch currentBatch; - - /** - * Maximum number of results to request per query. - * - *

Must be set, or it may result in an I/O error when querying - * Cloud Datastore. - */ - private static final int QUERY_LIMIT = 5000; - - /** - * Returns a DatastoreIterator with query and Datastore object set. - * - * @param query the query to select records. - * @param datastore a datastore connection to use. - */ - public DatastoreIterator(Query query, Datastore datastore) { - this.query = query.toBuilder().clone(); - this.datastore = datastore; - this.query.setLimit(QUERY_LIMIT); - } - - /** - * Returns an iterator over the next batch of records for the query - * and updates the cursor to get the next batch as needed. - * Query has specified limit and offset from InputSplit. - */ - private Iterator getIteratorAndMoveCursor() - throws DatastoreException{ - if (this.currentBatch != null && this.currentBatch.hasEndCursor()) { - this.query.setStartCursor(this.currentBatch.getEndCursor()); - } - - RunQueryRequest request = RunQueryRequest.newBuilder() - .setQuery(this.query) - .build(); - RunQueryResponse response = this.datastore.runQuery(request); - - this.currentBatch = response.getBatch(); - - // MORE_RESULTS_AFTER_LIMIT is not implemented yet: - // https://groups.google.com/forum/#!topic/gcd-discuss/iNs6M1jA2Vw, so - // use result count to determine if more results might exist. - int numFetch = this.currentBatch.getEntityResultCount(); - moreResults = numFetch == QUERY_LIMIT; - - // May receive a batch of 0 results if the number of records is a multiple - // of the request limit. - if (numFetch == 0) { - return null; - } - - return this.currentBatch.getEntityResultList().iterator(); - } - - @Override - public EntityResult computeNext() { - try { - if (entities == null || (!entities.hasNext() && this.moreResults)) { - entities = getIteratorAndMoveCursor(); - } - - if (entities == null || !entities.hasNext()) { - return endOfData(); - } - - return entities.next(); - - } catch (DatastoreException e) { - throw new RuntimeException( - "Datastore error while iterating over entities", e); - } - } -} - diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ReadSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ReadSource.java new file mode 100644 index 0000000000000..fc44e34293382 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ReadSource.java @@ -0,0 +1,123 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.dataflow.BasicSerializableSourceFormat; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.common.base.Preconditions; + +import javax.annotation.Nullable; + +/** + * The {@code PTransform} for reading from a {@code Source}. + *

+ * Usage example: + *

+ * Pipeline p = Pipeline.create();
+ * p.apply(ReadSource.from(new MySource().withFoo("foo").withBar("bar"))
+ *                   .named("foobar"));
+ * 
+ */ +public class ReadSource { + /** + * Returns a new {@code ReadSource.Bound} {@code PTransform} with the given name. + */ + @SuppressWarnings("unchecked") + public static Bound named(String name) { + return new Bound(name, null); + } + + /** + * Returns a new unnamed {@code ReadSource.Bound} {@code PTransform} reading from the given + * {@code Source}. + */ + public static Bound from(Source source) { + return new Bound<>("", source); + } + + /** + * Implementation of the {@code ReadSource} {@code PTransform} builder. + */ + public static class Bound + extends PTransform> { + @Nullable + private Source source; + + private Bound(@Nullable String name, @Nullable Source source) { + super(name); + this.source = source; + } + + /** + * Returns a new {@code ReadSource} {@code PTransform} that's like this one but + * reads from the given {@code Source}. + * + *

Does not modify this object. + */ + public Bound from(Source source) { + return new Bound(getName(), source); + } + + /** + * Returns a new {@code ReadSource} {@code PTransform} that's like this one but + * has the given name. + * + *

Does not modify this object. + */ + public Bound named(String name) { + return new Bound(name, source); + } + + @Override + protected Coder getDefaultOutputCoder() { + Preconditions.checkNotNull(source, "source must be set"); + return source.getDefaultOutputCoder(); + } + + @Override + public final PCollection apply(PInput input) { + Preconditions.checkNotNull(source, "source must be set"); + source.validate(); + return PCollection.createPrimitiveOutputInternal(new GlobalWindow()) + .setCoder(getDefaultOutputCoder()); + } + + /** + * Returns the {@code Source} used to create this {@code ReadSource} {@code PTransform}. + */ + @Nullable + public Source getSource() { + return source; + } + + static { + DirectPipelineRunner.registerDefaultTransformEvaluator( + Bound.class, new DirectPipelineRunner.TransformEvaluator() { + @Override + public void evaluate(Bound transform, DirectPipelineRunner.EvaluationContext context) { + BasicSerializableSourceFormat.evaluateReadHelper(transform, context); + } + }); + } + } + +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java new file mode 100644 index 0000000000000..21f36c3b9c0f7 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java @@ -0,0 +1,174 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.WindowedValue; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +/** + * Base class for defining input formats, with custom logic for splitting the input + * into shards (parts of the input, each of which may be processed on a different worker) + * and creating a {@code Source} for reading the input. + * + *

To use this class for supporting your custom input type, derive your class + * class from it, and override the abstract methods. Also override either + * {@link #createWindowedReader} if your source supports timestamps and windows, + * or {@link #createBasicReader} otherwise. For an example, see {@link DatastoreIO}. + * + *

A {@code Source} passed to a {@code Read} transform must be + * {@code Serializable}. This allows the {@code Source} instance + * created in this "main program" to be sent (in serialized form) to + * remote worker machines and reconstituted for each batch of elements + * of the input {@code PCollection} being processed or for each source splitting + * operation. A {@code Source} can have instance variable state, and + * non-transient instance variable state will be serialized in the main program + * and then deserialized on remote worker machines. + * + *

This API is experimental and subject to change. + * + * @param Type of elements read by the source. + */ +public abstract class Source implements Serializable { + /** + * Splits the source into shards. + * + *

{@code PipelineOptions} can be used to get information such as + * credentials for accessing an external storage. + */ + public abstract List> splitIntoShards( + long desiredShardSizeBytes, PipelineOptions options) throws Exception; + + /** + * An estimate of the total size (in bytes) of the data that would be read from this source. + * This estimate is in terms of external storage size, before any decompression or other + * processing done by the reader. + */ + public abstract long getEstimatedSizeBytes(PipelineOptions options) throws Exception; + + /** + * Whether this source is known to produce key/value pairs with the (encoded) keys in + * lexicographically sorted order. + */ + public abstract boolean producesSortedKeys(PipelineOptions options) throws Exception; + + /** + * Creates a windowed reader for this source. The default implementation wraps + * {@link #createBasicReader}. Override this function if your reader supports timestamps + * and windows; otherwise, override {@link #createBasicReader} instead. + */ + public Reader> createWindowedReader(PipelineOptions options, + Coder> coder, @Nullable ExecutionContext executionContext) + throws IOException { + return new WindowedReaderWrapper(createBasicReader( + options, ((WindowedValue.WindowedValueCoder) coder).getValueCoder(), executionContext)); + } + + /** + * Creates a basic (non-windowed) reader for this source. If you override this method, each value + * returned by this reader will be wrapped into the global window. + */ + protected Reader createBasicReader(PipelineOptions options, Coder coder, + @Nullable ExecutionContext executionContext) throws IOException { + throw new UnsupportedOperationException(); + } + + /** + * Checks that this source is valid, before it can be used into a pipeline. + * It is recommended to use {@link com.google.common.base.Preconditions} for implementing + * this method. + */ + public abstract void validate(); + + /** + * Returns the default {@code Coder} to use for the data read from this source. + */ + public abstract Coder getDefaultOutputCoder(); + + /** + * The interface which readers of custom input sources must implement. + *

+ * This interface is deliberately distinct from {@link java.util.Iterator} because + * the current model tends to be easier to program and more efficient in practice + * for iterating over sources such as files, databases etc. (rather than pure collections). + *

+ * To read a {@code SourceIterator}: + *

+   * while (iterator.advance()) {
+   *   T item = iterator.getCurrent();
+   *   ...
+   * }
+   * 
+ *

+ * Note: this interface is work-in-progress and may change. + */ + public interface Reader extends AutoCloseable { + /** + * Advances the iterator to the next valid record. + * Invalidates the result of the previous {@link #getCurrent} call. + * @return {@code true} if a record was read, {@code false} if we're at the end of input. + */ + public boolean advance() throws IOException; + + /** + * Returns the value of the data item which was read by the last {@link #advance} call. + * @throws java.util.NoSuchElementException if the iterator is at the beginning of the input + * and {@link #advance} wasn't called, or if the last {@link #advance} returned {@code false}. + */ + public T getCurrent() throws NoSuchElementException; + + /** + * Closes the iterator. The iterator cannot be used after this method was called. + */ + @Override + public void close() throws IOException; + } + + /** + * An adapter from {@code SourceIterator} to {@code SourceIterator>}. + */ + private static class WindowedReaderWrapper implements Reader> { + private final Reader reader; + + public WindowedReaderWrapper(Reader reader) { + this.reader = reader; + } + + @Override + public boolean advance() throws IOException { + return reader.advance(); + } + + @Override + public WindowedValue getCurrent() throws NoSuchElementException { + return WindowedValue.valueInGlobalWindow(reader.getCurrent()); + } + + @Override + public void close() throws IOException { + reader.close(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index f3ddbd1cb444a..f948f5c689864 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -47,6 +47,7 @@ import com.google.cloud.dataflow.sdk.io.BigQueryIO; import com.google.cloud.dataflow.sdk.io.DatastoreIO; import com.google.cloud.dataflow.sdk.io.PubsubIO; +import com.google.cloud.dataflow.sdk.io.ReadSource; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.options.CloudDebuggerOptions.DebuggerConfig; import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; @@ -55,6 +56,7 @@ import com.google.cloud.dataflow.sdk.runners.dataflow.BigQueryIOTranslator; import com.google.cloud.dataflow.sdk.runners.dataflow.DatastoreIOTranslator; import com.google.cloud.dataflow.sdk.runners.dataflow.PubsubIOTranslator; +import com.google.cloud.dataflow.sdk.runners.dataflow.ReadSourceTranslator; import com.google.cloud.dataflow.sdk.runners.dataflow.TextIOTranslator; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.Create; @@ -923,7 +925,7 @@ private void translateSingleHelper( BigQueryIO.Write.Bound.class, new BigQueryIOTranslator.WriteTranslator()); registerTransformTranslator( - DatastoreIO.Write.Bound.class, new DatastoreIOTranslator.WriteTranslator()); + DatastoreIO.Sink.class, new DatastoreIOTranslator.WriteTranslator()); registerTransformTranslator( PubsubIO.Read.Bound.class, new PubsubIOTranslator.ReadTranslator()); @@ -934,6 +936,9 @@ private void translateSingleHelper( TextIO.Read.Bound.class, new TextIOTranslator.ReadTranslator()); registerTransformTranslator( TextIO.Write.Bound.class, new TextIOTranslator.WriteTranslator()); + + registerTransformTranslator( + ReadSource.Bound.class, new ReadSourceTranslator()); } private static void translateInputs( diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormat.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormat.java new file mode 100644 index 0000000000000..cb3f4c237e7d3 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormat.java @@ -0,0 +1,277 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.dataflow; + +import static com.google.api.client.util.Base64.encodeBase64String; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudSourceOperationResponseToSourceOperationResponse; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudSourceToDictionary; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceOperationRequestToCloudSourceOperationRequest; +import static com.google.cloud.dataflow.sdk.util.SerializableUtils.deserializeFromByteArray; +import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.api.client.util.Base64; +import com.google.api.services.dataflow.model.SourceGetMetadataRequest; +import com.google.api.services.dataflow.model.SourceGetMetadataResponse; +import com.google.api.services.dataflow.model.SourceMetadata; +import com.google.api.services.dataflow.model.SourceOperationRequest; +import com.google.api.services.dataflow.model.SourceOperationResponse; +import com.google.api.services.dataflow.model.SourceSplitOptions; +import com.google.api.services.dataflow.model.SourceSplitRequest; +import com.google.api.services.dataflow.model.SourceSplitResponse; +import com.google.api.services.dataflow.model.SourceSplitShard; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.ReadSource; +import com.google.cloud.dataflow.sdk.io.Source; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.SourceFormat; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; + +/** + * An helper class for supporting sources defined as {@code Source}. + * + * Provides a bridge between the high-level {@code Source} API and the raw + * API-level {@code SourceFormat} API, by encoding the serialized + * {@code Source} in a parameter of the API {@code Source} message. + *

+ */ +public class BasicSerializableSourceFormat implements SourceFormat { + private static final String SERIALIZED_SOURCE = "serialized_source"; + private static final long DEFAULT_DESIRED_SHARD_SIZE_BYTES = 64 * (1 << 20); + + private final PipelineOptions options; + + public BasicSerializableSourceFormat(PipelineOptions options) { + this.options = options; + } + + /** + * Executes a protocol-level split {@code SourceOperationRequest} by deserializing its source + * to a {@code Source}, splitting it, and serializing results back. + */ + @Override + public OperationResponse performSourceOperation(OperationRequest request) throws Exception { + SourceOperationRequest cloudRequest = + sourceOperationRequestToCloudSourceOperationRequest(request); + SourceOperationResponse cloudResponse = new SourceOperationResponse(); + if (cloudRequest.getGetMetadata() != null) { + cloudResponse.setGetMetadata(performGetMetadata(cloudRequest.getGetMetadata())); + } else if (cloudRequest.getSplit() != null) { + cloudResponse.setSplit(performSplit(cloudRequest.getSplit())); + } else { + throw new UnsupportedOperationException("Unknown source operation request"); + } + return cloudSourceOperationResponseToSourceOperationResponse(cloudResponse); + } + + /** + * Factory method allowing this class to satisfy the implicit contract of {@code SourceFactory}. + */ + @SuppressWarnings("unchecked") + public static com.google.cloud.dataflow.sdk.util.common.worker.Reader create( + final PipelineOptions options, CloudObject spec, + final Coder> coder, final ExecutionContext executionContext) + throws Exception { + final Source source = (Source) deserializeFromCloudSource(spec); + return new com.google.cloud.dataflow.sdk.util.common.worker.Reader() { + @Override + public ReaderIterator iterator() throws IOException { + return new BasicSerializableSourceFormat.ReaderIterator<>( + source.createWindowedReader(options, coder, executionContext)); + } + }; + } + + private SourceSplitResponse performSplit(SourceSplitRequest request) throws Exception { + Source source = deserializeFromCloudSource(request.getSource().getSpec()); + + // Produce simple independent, unsplittable shards with no metadata attached. + SourceSplitResponse response = new SourceSplitResponse(); + response.setShards(new ArrayList()); + SourceSplitOptions splitOptions = request.getOptions(); + Long desiredShardSizeBytes = + (splitOptions == null) ? null : splitOptions.getDesiredShardSizeBytes(); + if (desiredShardSizeBytes == null) { + desiredShardSizeBytes = DEFAULT_DESIRED_SHARD_SIZE_BYTES; + } + for (Source split : source.splitIntoShards(desiredShardSizeBytes, options)) { + SourceSplitShard shard = new SourceSplitShard(); + + com.google.api.services.dataflow.model.Source cloudSource = + serializeToCloudSource(split, options); + cloudSource.setDoesNotNeedSplitting(true); + + shard.setDerivationMode("SOURCE_DERIVATION_MODE_INDEPENDENT"); + shard.setSource(cloudSource); + response.getShards().add(shard); + } + response.setOutcome("SOURCE_SPLIT_OUTCOME_SPLITTING_HAPPENED"); + return response; + } + + private SourceGetMetadataResponse performGetMetadata(SourceGetMetadataRequest request) + throws Exception { + Source source = deserializeFromCloudSource(request.getSource().getSpec()); + SourceMetadata metadata = new SourceMetadata(); + metadata.setProducesSortedKeys(source.producesSortedKeys(options)); + metadata.setEstimatedSizeBytes(source.getEstimatedSizeBytes(options)); + SourceGetMetadataResponse response = new SourceGetMetadataResponse(); + response.setMetadata(metadata); + return response; + } + + private static Source deserializeFromCloudSource(Map spec) + throws Exception { + return (Source) deserializeFromByteArray( + Base64.decodeBase64(getString(spec, SERIALIZED_SOURCE)), "Source"); + } + + private static com.google.api.services.dataflow.model.Source serializeToCloudSource( + Source source, PipelineOptions options) throws Exception { + com.google.api.services.dataflow.model.Source cloudSource = + new com.google.api.services.dataflow.model.Source(); + // We ourselves act as the SourceFormat. + cloudSource.setSpec(CloudObject.forClass(BasicSerializableSourceFormat.class)); + addString( + cloudSource.getSpec(), SERIALIZED_SOURCE, encodeBase64String(serializeToByteArray(source))); + + SourceMetadata metadata = new SourceMetadata(); + metadata.setProducesSortedKeys(source.producesSortedKeys(options)); + metadata.setEstimatedSizeBytes(source.getEstimatedSizeBytes(options)); + cloudSource.setMetadata(metadata); + return cloudSource; + } + + public static void evaluateReadHelper( + ReadSource.Bound transform, DirectPipelineRunner.EvaluationContext context) { + try { + List> elems = new ArrayList<>(); + Source source = transform.getSource(); + try (Source.Reader> reader = + source.createWindowedReader(context.getPipelineOptions(), + WindowedValue.getValueOnlyCoder(source.getDefaultOutputCoder()), null)) { + while (reader.advance()) { + elems.add(reader.getCurrent()); + } + } + List> output = new ArrayList<>(); + for (WindowedValue elem : elems) { + output.add(DirectPipelineRunner.ValueWithMetadata.of(elem)); + } + context.setPCollectionValuesWithMetadata(transform.getOutput(), output); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public static void translateReadHelper( + ReadSource.Bound transform, DataflowPipelineTranslator.TranslationContext context) { + try { + context.addStep(transform, "ParallelRead"); + context.addInput(PropertyNames.FORMAT, PropertyNames.CUSTOM_SOURCE_FORMAT); + context.addInput( + PropertyNames.SOURCE_STEP_INPUT, + cloudSourceToDictionary( + serializeToCloudSource(transform.getSource(), context.getPipelineOptions()))); + context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Adapter from the {@code Source.Reader} interface to + * {@code Reader.ReaderIterator}. + * + * TODO: Consider changing the API of Reader.ReaderIterator so this adapter wouldn't be needed. + */ + private static class ReaderIterator + implements com.google.cloud.dataflow.sdk.util.common.worker.Reader.ReaderIterator { + private Source.Reader reader; + private boolean hasNext; + private T next; + private boolean advanced; + + private ReaderIterator(Source.Reader reader) { + this.reader = reader; + } + + @Override + public boolean hasNext() throws IOException { + if (!advanced) { + advanceInternal(); + } + return hasNext; + } + + @Override + public T next() throws IOException { + if (!hasNext()) { + throw new NoSuchElementException(); + } + T res = this.next; + advanceInternal(); + return res; + } + + private void advanceInternal() throws IOException { + try { + hasNext = reader.advance(); + if (hasNext) { + next = reader.getCurrent(); + } + advanced = true; + } catch (Exception e) { + throw new IOException(e); + } + } + + @Override + public com.google.cloud.dataflow.sdk.util.common.worker.Reader.ReaderIterator copy() + throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException { + reader.close(); + } + + @Override + public com.google.cloud.dataflow.sdk.util.common.worker.Reader.Progress getProgress() { + return null; + } + + @Override + public com.google.cloud.dataflow.sdk.util.common.worker.Reader.Position updateStopPosition( + com.google.cloud.dataflow.sdk.util.common.worker.Reader.Progress proposedStopPosition) { + return null; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DatastoreIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DatastoreIOTranslator.java index 4292199174a14..e809d692387c0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DatastoreIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/DatastoreIOTranslator.java @@ -28,10 +28,10 @@ public class DatastoreIOTranslator { /** * Implements DatastoreIO Write translation for the Dataflow backend. */ - public static class WriteTranslator implements TransformTranslator { + public static class WriteTranslator implements TransformTranslator { @Override public void translate( - DatastoreIO.Write.Bound transform, + DatastoreIO.Sink transform, TranslationContext context) { // TODO: Not implemented yet. // translateWriteHelper(transform, context); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/ReadSourceTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/ReadSourceTranslator.java new file mode 100644 index 0000000000000..7897c4cd17fac --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/ReadSourceTranslator.java @@ -0,0 +1,32 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.dataflow; + +import com.google.cloud.dataflow.sdk.io.ReadSource; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator; + +/** + * Translator for the {@code ReadSource} {@code PTransform} for the Dataflow back-end. + */ +public class ReadSourceTranslator + implements DataflowPipelineTranslator.TransformTranslator { + @Override + public void translate( + ReadSource.Bound transform, DataflowPipelineTranslator.TranslationContext context) { + BasicSerializableSourceFormat.translateReadHelper(transform, context); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java index 499653a11d0a7..555e15a0bc403 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java @@ -35,8 +35,8 @@ import com.google.cloud.dataflow.sdk.util.common.Counter; import com.google.cloud.dataflow.sdk.util.common.CounterSet; import com.google.cloud.dataflow.sdk.util.common.Metric; -import com.google.cloud.dataflow.sdk.util.common.worker.CustomSourceFormat; import com.google.cloud.dataflow.sdk.util.common.worker.Reader; +import com.google.cloud.dataflow.sdk.util.common.worker.SourceFormat; import com.google.cloud.dataflow.sdk.util.common.worker.WorkExecutor; import com.google.cloud.dataflow.sdk.util.common.worker.WorkProgressUpdater; @@ -115,7 +115,7 @@ private boolean doWork(WorkItem workItem) throws IOException { worker = MapTaskExecutorFactory.create(options, workItem.getMapTask(), executionContext); } else if (workItem.getSourceOperationTask() != null) { - worker = SourceOperationExecutorFactory.create(workItem.getSourceOperationTask()); + worker = SourceOperationExecutorFactory.create(options, workItem.getSourceOperationTask()); } else { throw new RuntimeException("unknown kind of work item: " + workItem.toString()); @@ -150,13 +150,13 @@ private boolean doWork(WorkItem workItem) throws IOException { // TODO: Find out a generic way for the WorkExecutor to report work-specific results // into the work update. - CustomSourceFormat.OperationResponse sourceOperationResponse = + SourceFormat.OperationResponse operationResponse = (worker instanceof SourceOperationExecutor) ? cloudSourceOperationResponseToSourceOperationResponse( ((SourceOperationExecutor) worker).getResponse()) : null; reportStatus( - options, "Success", workItem, counters, metrics, sourceOperationResponse, null/*errors*/); + options, "Success", workItem, counters, metrics, operationResponse, null/*errors*/); return true; @@ -219,11 +219,11 @@ private static String buildCloudStackTrace(Throwable t) { private void reportStatus(DataflowWorkerHarnessOptions options, String status, WorkItem workItem, @Nullable CounterSet counters, @Nullable Collection> metrics, - @Nullable CustomSourceFormat.OperationResponse sourceOperationResponse, + @Nullable SourceFormat.OperationResponse operationResponse, @Nullable List errors) throws IOException { LOG.info("{} processing work item {}", status, uniqueId(workItem)); WorkItemStatus workItemStatus = buildStatus(workItem, true/*completed*/, counters, metrics, - options, null, null, sourceOperationResponse, errors); + options, null, null, operationResponse, errors); workUnitClient.reportWorkItemStatus(workItemStatus); } @@ -231,7 +231,7 @@ static WorkItemStatus buildStatus(WorkItem workItem, boolean completed, @Nullable CounterSet counters, @Nullable Collection> metrics, DataflowWorkerHarnessOptions options, @Nullable Reader.Progress progress, @Nullable Reader.Position stopPosition, - @Nullable CustomSourceFormat.OperationResponse sourceOperationResponse, + @Nullable SourceFormat.OperationResponse operationResponse, @Nullable List errors) { WorkItemStatus status = new WorkItemStatus(); status.setWorkItemId(Long.toString(workItem.getId())); @@ -280,7 +280,7 @@ static WorkItemStatus buildStatus(WorkItem workItem, boolean completed, if (workItem.getSourceOperationTask() != null) { status.setSourceOperationResponse( - sourceOperationResponseToCloudSourceOperationResponse(sourceOperationResponse)); + sourceOperationResponseToCloudSourceOperationResponse(operationResponse)); } return status; @@ -304,7 +304,7 @@ public abstract static class WorkUnitClient { * Reports a {@link WorkItemStatus} for an assigned {@link WorkItem}. * * @param workItemStatus the status to report - * @return a {@link WorkServiceState} (e.g. a new stop position) + * @return a {@link WorkItemServiceState} (e.g. a new stop position) */ public abstract WorkItemServiceState reportWorkItemStatus(WorkItemStatus workItemStatus) throws IOException; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CustomSourceFormatFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFormatFactory.java similarity index 71% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CustomSourceFormatFactory.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFormatFactory.java index 1bb3db228a730..98e6aabd794e5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CustomSourceFormatFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceFormatFactory.java @@ -19,29 +19,31 @@ import static com.google.cloud.dataflow.sdk.util.Structs.getString; import com.google.api.services.dataflow.model.Source; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.util.InstanceBuilder; import com.google.cloud.dataflow.sdk.util.PropertyNames; -import com.google.cloud.dataflow.sdk.util.common.worker.CustomSourceFormat; +import com.google.cloud.dataflow.sdk.util.common.worker.SourceFormat; import java.util.Map; /** - * Creates {@code CustomSourceFormat} objects from {@code Source}. + * Creates {@code SourceFormat} objects from {@code Source}. */ -public class CustomSourceFormatFactory { - private CustomSourceFormatFactory() {} +public class SourceFormatFactory { + private SourceFormatFactory() {} - public static CustomSourceFormat create(Source source) throws Exception { + public static SourceFormat create(PipelineOptions options, Source source) throws Exception { Map spec = source.getSpec(); try { - return InstanceBuilder.ofType(CustomSourceFormat.class) + return InstanceBuilder.ofType(SourceFormat.class) .fromClassName(getString(spec, PropertyNames.OBJECT_TYPE_NAME)) + .withArg(PipelineOptions.class, options) .build(); } catch (ClassNotFoundException exn) { throw new Exception( - "unable to create a custom source format from " + source, exn); + "unable to create a source format from " + source, exn); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java index 34bd27966bfc7..3ad4528ee977d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java @@ -22,6 +22,7 @@ import com.google.api.services.dataflow.model.Source; import com.google.api.services.dataflow.model.SourceOperationRequest; import com.google.api.services.dataflow.model.SourceOperationResponse; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.util.common.CounterSet; import com.google.cloud.dataflow.sdk.util.common.worker.MapTaskExecutor; import com.google.cloud.dataflow.sdk.util.common.worker.WorkExecutor; @@ -36,12 +37,15 @@ public class SourceOperationExecutor extends WorkExecutor { private static final Logger LOG = LoggerFactory.getLogger(MapTaskExecutor.class); + private final PipelineOptions options; private final SourceOperationRequest request; private SourceOperationResponse response; - public SourceOperationExecutor(SourceOperationRequest request, + public SourceOperationExecutor(PipelineOptions options, + SourceOperationRequest request, CounterSet counters) { super(counters); + this.options = options; this.request = request; } @@ -58,11 +62,10 @@ public void execute() throws Exception { throw new UnsupportedOperationException("Unknown source operation"); } - this.response = - sourceOperationResponseToCloudSourceOperationResponse( - CustomSourceFormatFactory.create(sourceSpec) - .performSourceOperation( - cloudSourceOperationRequestToSourceOperationRequest(request))); + this.response = sourceOperationResponseToCloudSourceOperationResponse( + SourceFormatFactory.create(options, sourceSpec) + .performSourceOperation( + cloudSourceOperationRequestToSourceOperationRequest(request))); LOG.debug("Source operation execution complete"); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutorFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutorFactory.java index 10c862e464875..3da3d6ab94e08 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutorFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutorFactory.java @@ -17,15 +17,16 @@ package com.google.cloud.dataflow.sdk.runners.worker; import com.google.api.services.dataflow.model.SourceOperationRequest; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.util.common.CounterSet; /** * Creates a SourceOperationExecutor from a SourceOperation. */ public class SourceOperationExecutorFactory { - public static SourceOperationExecutor create(SourceOperationRequest request) - throws Exception { + public static SourceOperationExecutor create( + PipelineOptions options, SourceOperationRequest request) throws Exception { CounterSet counters = new CounterSet(); - return new SourceOperationExecutor(request, counters); + return new SourceOperationExecutor(options, request, counters); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java index 0f7ce18c480f5..07229941a82af 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java @@ -28,8 +28,8 @@ import com.google.api.services.dataflow.model.SourceOperationRequest; import com.google.api.services.dataflow.model.SourceOperationResponse; import com.google.cloud.dataflow.sdk.util.PropertyNames; -import com.google.cloud.dataflow.sdk.util.common.worker.CustomSourceFormat; import com.google.cloud.dataflow.sdk.util.common.worker.Reader; +import com.google.cloud.dataflow.sdk.util.common.worker.SourceFormat; import java.util.HashMap; import java.util.Map; @@ -50,19 +50,19 @@ public static Reader.Position cloudPositionToReaderPosition(@Nullable Position c return cloudPosition == null ? null : new DataflowReaderPosition(cloudPosition); } - public static CustomSourceFormat.OperationRequest + public static SourceFormat.OperationRequest cloudSourceOperationRequestToSourceOperationRequest( @Nullable SourceOperationRequest request) { return request == null ? null : new DataflowSourceOperationRequest(request); } - public static CustomSourceFormat.OperationResponse + public static SourceFormat.OperationResponse cloudSourceOperationResponseToSourceOperationResponse( @Nullable SourceOperationResponse response) { return response == null ? null : new DataflowSourceOperationResponse(response); } - public static CustomSourceFormat.SourceSpec cloudSourceToSourceSpec( + public static SourceFormat.SourceSpec cloudSourceToSourceSpec( @Nullable Source cloudSource) { return cloudSource == null ? null : new DataflowSourceSpec(cloudSource); } @@ -76,17 +76,18 @@ public static Position sourcePositionToCloudPosition(@Nullable Reader.Position s return sourcePosition == null ? null : ((DataflowReaderPosition) sourcePosition).cloudPosition; } + public static SourceOperationRequest sourceOperationRequestToCloudSourceOperationRequest( - @Nullable CustomSourceFormat.OperationRequest request) { + @Nullable SourceFormat.OperationRequest request) { return (request == null) ? null : ((DataflowSourceOperationRequest) request).cloudRequest; } public static SourceOperationResponse sourceOperationResponseToCloudSourceOperationResponse( - @Nullable CustomSourceFormat.OperationResponse response) { + @Nullable SourceFormat.OperationResponse response) { return (response == null) ? null : ((DataflowSourceOperationResponse) response).cloudResponse; } - public static Source sourceSpecToCloudSource(@Nullable CustomSourceFormat.SourceSpec spec) { + public static Source sourceSpecToCloudSource(@Nullable SourceFormat.SourceSpec spec) { return (spec == null) ? null : ((DataflowSourceSpec) spec).cloudSource; } @@ -104,40 +105,40 @@ public DataflowReaderPosition(Position cloudPosition) { } } - static class DataflowSourceOperationRequest implements CustomSourceFormat.OperationRequest { + static class DataflowSourceOperationRequest implements SourceFormat.OperationRequest { public final SourceOperationRequest cloudRequest; public DataflowSourceOperationRequest(SourceOperationRequest cloudRequest) { this.cloudRequest = cloudRequest; } } - static class DataflowSourceOperationResponse implements CustomSourceFormat.OperationResponse { + static class DataflowSourceOperationResponse implements SourceFormat.OperationResponse { public final SourceOperationResponse cloudResponse; public DataflowSourceOperationResponse(SourceOperationResponse cloudResponse) { this.cloudResponse = cloudResponse; } } - static class DataflowSourceSpec implements CustomSourceFormat.SourceSpec { + static class DataflowSourceSpec implements SourceFormat.SourceSpec { public final Source cloudSource; public DataflowSourceSpec(Source cloudSource) { this.cloudSource = cloudSource; } } - // Represents a cloud Source as a dictionary for encoding inside the CUSTOM_SOURCE + // Represents a cloud Source as a dictionary for encoding inside the {@code SOURCE_STEP_INPUT} // property of CloudWorkflowStep.input. public static Map cloudSourceToDictionary(Source source) { // Do not translate encoding - the source's encoding is translated elsewhere // to the step's output info. Map res = new HashMap<>(); - addDictionary(res, PropertyNames.CUSTOM_SOURCE_SPEC, source.getSpec()); + addDictionary(res, PropertyNames.SOURCE_SPEC, source.getSpec()); if (source.getMetadata() != null) { - addDictionary(res, PropertyNames.CUSTOM_SOURCE_METADATA, + addDictionary(res, PropertyNames.SOURCE_METADATA, cloudSourceMetadataToDictionary(source.getMetadata())); } if (source.getDoesNotNeedSplitting() != null) { - addBoolean(res, PropertyNames.CUSTOM_SOURCE_DOES_NOT_NEED_SPLITTING, + addBoolean(res, PropertyNames.SOURCE_DOES_NOT_NEED_SPLITTING, source.getDoesNotNeedSplitting()); } return res; @@ -147,22 +148,22 @@ private static Map cloudSourceMetadataToDictionary(SourceMetadat Map res = new HashMap<>(); if (metadata.getProducesSortedKeys() != null) { addBoolean( - res, PropertyNames.CUSTOM_SOURCE_PRODUCES_SORTED_KEYS, metadata.getProducesSortedKeys()); + res, PropertyNames.SOURCE_PRODUCES_SORTED_KEYS, metadata.getProducesSortedKeys()); } if (metadata.getEstimatedSizeBytes() != null) { addLong( - res, PropertyNames.CUSTOM_SOURCE_ESTIMATED_SIZE_BYTES, metadata.getEstimatedSizeBytes()); + res, PropertyNames.SOURCE_ESTIMATED_SIZE_BYTES, metadata.getEstimatedSizeBytes()); } if (metadata.getInfinite() != null) { - addBoolean(res, PropertyNames.CUSTOM_SOURCE_IS_INFINITE, metadata.getInfinite()); + addBoolean(res, PropertyNames.SOURCE_IS_INFINITE, metadata.getInfinite()); } return res; } public static Source dictionaryToCloudSource(Map params) throws Exception { Source res = new Source(); - res.setSpec(getDictionary(params, PropertyNames.CUSTOM_SOURCE_SPEC)); - // CUSTOM_SOURCE_METADATA and CUSTOM_SOURCE_DOES_NOT_NEED_SPLITTING do not have to be + res.setSpec(getDictionary(params, PropertyNames.SOURCE_SPEC)); + // SOURCE_METADATA and SOURCE_DOES_NOT_NEED_SPLITTING do not have to be // translated, because they only make sense in cloud Source objects produced by the user. return res; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtils.java index bcb4d343a35a4..e2d6ee6b6a72a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtils.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CloudSourceUtils.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.util; import com.google.api.services.dataflow.model.Source; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.worker.ReaderFactory; import java.util.HashMap; @@ -54,9 +55,10 @@ public static Source flattenBaseSpecs(Source source) { * Creates a {@link com.google.cloud.dataflow.sdk.util.common.worker.Reader} * from the given Dataflow Source API definition and reads all elements from it. */ - public static List readElemsFromSource(Source source) { + + public static List readElemsFromSource(PipelineOptions options, Source source) { try { - return ReaderUtils.readElemsFromReader(ReaderFactory.create(null, source, null)); + return ReaderUtils.readElemsFromReader(ReaderFactory.create(options, source, null)); } catch (Exception e) { throw new RuntimeException("Failed to read from source: " + source.toString(), e); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java index c7a5a307f1965..26b260bd50f2d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java @@ -31,13 +31,13 @@ public class PropertyNames { public static final String COMBINE_FN = "combine_fn"; public static final String COMPONENT_ENCODINGS = "component_encodings"; public static final String CUSTOM_SOURCE_FORMAT = "custom_source"; - public static final String CUSTOM_SOURCE_STEP_INPUT = "custom_source_step_input"; - public static final String CUSTOM_SOURCE_SPEC = "spec"; - public static final String CUSTOM_SOURCE_METADATA = "metadata"; - public static final String CUSTOM_SOURCE_DOES_NOT_NEED_SPLITTING = "does_not_need_splitting"; - public static final String CUSTOM_SOURCE_PRODUCES_SORTED_KEYS = "produces_sorted_keys"; - public static final String CUSTOM_SOURCE_IS_INFINITE = "is_infinite"; - public static final String CUSTOM_SOURCE_ESTIMATED_SIZE_BYTES = "estimated_size_bytes"; + public static final String SOURCE_STEP_INPUT = "custom_source_step_input"; + public static final String SOURCE_SPEC = "spec"; + public static final String SOURCE_METADATA = "metadata"; + public static final String SOURCE_DOES_NOT_NEED_SPLITTING = "does_not_need_splitting"; + public static final String SOURCE_PRODUCES_SORTED_KEYS = "produces_sorted_keys"; + public static final String SOURCE_IS_INFINITE = "is_infinite"; + public static final String SOURCE_ESTIMATED_SIZE_BYTES = "estimated_size_bytes"; public static final String ELEMENT = "element"; public static final String ELEMENTS = "elements"; public static final String ENCODING = "encoding"; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TestCredential.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TestCredential.java index fa02a6bf3185b..6b9bc3b53ecb1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TestCredential.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TestCredential.java @@ -19,6 +19,7 @@ import com.google.api.client.auth.oauth2.BearerToken; import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.auth.oauth2.TokenResponse; +import com.google.api.client.testing.http.MockHttpTransport; import java.io.IOException; @@ -35,7 +36,8 @@ public TestCredential() { public TestCredential(String token) { super(new Builder( - BearerToken.authorizationHeaderAccessMethod())); + BearerToken.authorizationHeaderAccessMethod()) + .setTransport(new MockHttpTransport())); this.token = token; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java index acd5d6468b691..e97cf2acb3daf 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java @@ -157,7 +157,7 @@ public void run() { try { // Force a progress update at the beginning and at the end. synchronized (sourceIteratorLock) { - progress.set(readerIterator.getProgress()); + setProgressFromIterator(); } while (true) { Object value; @@ -172,14 +172,14 @@ public void run() { value = readerIterator.next(); if (isProgressUpdateRequested.getAndSet(false) || progressUpdatePeriodMs == 0) { - progress.set(readerIterator.getProgress()); + setProgressFromIterator(); } } } receiver.process(value); } synchronized (sourceIteratorLock) { - progress.set(readerIterator.getProgress()); + setProgressFromIterator(); } } finally { synchronized (sourceIteratorLock) { @@ -193,6 +193,17 @@ public void run() { } } + private void setProgressFromIterator() { + try { + progress.set(readerIterator.getProgress()); + } catch (UnsupportedOperationException e) { + // Ignore: same semantics as null. + } catch (Exception e) { + // This is not a normal situation, but should not kill the task. + LOG.warn("Progress estimation failed", e); + } + } + /** * Returns a (possibly slightly stale) value of the progress of the task. * Guaranteed to not block indefinitely. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java index 2ecef5b4cc869..2b805c6f1d1fd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java @@ -71,7 +71,8 @@ public interface ReaderIterator extends AutoCloseable { * called concurrently to any other methods. * * @return the progress, or {@code null} if no progress measure - * can be provided + * can be provided (implementors are discouraged from throwing + * {@code UnsupportedOperationException} in this case). */ public Progress getProgress(); @@ -91,7 +92,8 @@ public interface ReaderIterator extends AutoCloseable { * @param proposedStopPosition a proposed position to stop * iterating through the source * @return the new stop position, or {@code null} on failure if the - * implementation does not support position updates. + * implementation does not support position updates(implementors are discouraged + * from throwing {@code UnsupportedOperationException} in this case). */ public Position updateStopPosition(Progress proposedStopPosition); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CustomSourceFormat.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/SourceFormat.java similarity index 98% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CustomSourceFormat.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/SourceFormat.java index 12cdf30ac468a..8b65c90877cb6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/CustomSourceFormat.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/SourceFormat.java @@ -20,7 +20,7 @@ * An interface for sources which can perform operations on source specifications, such as * splitting the source and computing its metadata. See {@code SourceOperationRequest} for details. */ -public interface CustomSourceFormat { +public interface SourceFormat { /** * Performs an operation on the specification of a source. * See {@code SourceOperationRequest} for details. diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java index fba8ec20b3fd0..279a4f16b9047 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java @@ -17,13 +17,25 @@ package com.google.cloud.dataflow.sdk.io; import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import com.google.api.services.datastore.DatastoreV1; import com.google.api.services.datastore.DatastoreV1.Entity; import com.google.api.services.datastore.DatastoreV1.Query; +import com.google.api.services.datastore.client.Datastore; +import com.google.api.services.datastore.client.DatastoreHelper; +import com.google.api.services.datastore.client.QuerySplitter; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.EntityCoder; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.runners.DirectPipeline; import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.util.TestCredential; +import com.google.common.base.Supplier; import org.junit.Before; import org.junit.Test; @@ -31,12 +43,14 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import java.util.ArrayList; +import java.util.List; + /** * Tests for DatastoreIO Read and Write transforms. */ @RunWith(JUnit4.class) public class DatastoreIOTest { - private String host; private String datasetId; private Query query; @@ -72,9 +86,8 @@ public void setUp() { @Test @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) public void testBuildRead() throws Exception { - DatastoreIO.Read.Bound readQuery = DatastoreIO.Read - .withHost(this.host) - .from(this.datasetId, this.query); + DatastoreIO.Source readQuery = + DatastoreIO.read().withHost(this.host).withDataset(this.datasetId).withQuery(this.query); assertEquals(this.query, readQuery.query); assertEquals(this.datasetId, readQuery.datasetId); assertEquals(this.host, readQuery.host); @@ -82,45 +95,87 @@ public void testBuildRead() throws Exception { @Test public void testBuildReadAlt() throws Exception { - DatastoreIO.Read.Bound readQuery = DatastoreIO.Read - .from(this.datasetId, this.query) - .withHost(this.host); + DatastoreIO.Source readQuery = + DatastoreIO.read().withDataset(this.datasetId).withQuery(this.query).withHost(this.host); assertEquals(this.query, readQuery.query); assertEquals(this.datasetId, readQuery.datasetId); assertEquals(this.host, readQuery.host); } - @Test(expected = IllegalStateException.class) - public void testBuildReadWithoutDatastoreSettingToCatchException() - throws Exception { + @Test(expected = NullPointerException.class) + public void testBuildReadWithoutDatastoreSettingToCatchException() throws Exception { // create pipeline and run the pipeline to get result Pipeline p = DirectPipeline.createForTest(); - p.apply(DatastoreIO.Read.named("ReadDatastore")); + p.apply(ReadSource.from(DatastoreIO.read().withHost(null))); + } + + @Test + public void testQuerySplitWithMockSplitter() throws Exception { + String dataset = "mydataset"; + DatastoreV1.KindExpression mykind = + DatastoreV1.KindExpression.newBuilder().setName("mykind").build(); + Query query = Query.newBuilder().addKind(mykind).build(); + + DataflowPipelineOptions options = PipelineOptionsFactory.create() + .as(DataflowPipelineOptions.class); + options.setGcpCredential(new TestCredential()); + + List mockSplits = new ArrayList<>(); + for (int i = 0; i < 8; ++i) { + mockSplits.add( + Query.newBuilder() + .addKind(mykind) + .setFilter( + DatastoreHelper.makeFilter("foo", DatastoreV1.PropertyFilter.Operator.EQUAL, + DatastoreV1.Value.newBuilder().setIntegerValue(i).build())) + .build()); + } + + QuerySplitter splitter = mock(QuerySplitter.class); + when(splitter.getSplits(any(Query.class), eq(8), any(Datastore.class))).thenReturn(mockSplits); + + DatastoreIO.Source io = + DatastoreIO.read() + .withDataset(dataset) + .withQuery(query) + .withMockSplitter(splitter) + .withMockEstimateSizeBytes(new Supplier() { + @Override + public Long get() { + return 8 * 1024L; + } + }); + + List shards = io.splitIntoShards(1024, options); + assertEquals(8, shards.size()); + for (int i = 0; i < 8; ++i) { + DatastoreIO.Source shard = shards.get(i); + Query shardQuery = shard.query; + assertEquals("mykind", shardQuery.getKind(0).getName()); + assertEquals(i, shardQuery.getFilter().getPropertyFilter().getValue().getIntegerValue()); + } } @Test public void testBuildWrite() throws Exception { - DatastoreIO.Write.Bound write = DatastoreIO.Write - .to(this.datasetId) - .withHost(this.host); - assertEquals(this.host, write.host); - assertEquals(this.datasetId, write.datasetId); + DatastoreIO.Sink sink = DatastoreIO.write().to(this.datasetId).withHost(this.host); + assertEquals(this.host, sink.host); + assertEquals(this.datasetId, sink.datasetId); } @Test public void testBuildWriteAlt() throws Exception { - DatastoreIO.Write.Bound write = DatastoreIO.Write - .withHost(this.host) - .to(this.datasetId); - assertEquals(this.host, write.host); - assertEquals(this.datasetId, write.datasetId); + DatastoreIO.Sink sink = DatastoreIO.write().withHost(this.host).to(this.datasetId); + assertEquals(this.host, sink.host); + assertEquals(this.datasetId, sink.datasetId); } @Test(expected = IllegalStateException.class) public void testBuildWriteWithoutDatastoreToCatchException() throws Exception { // create pipeline and run the pipeline to get result Pipeline p = DirectPipeline.createForTest(); - p.apply(Create.of()).setCoder(EntityCoder.of()) - .apply(DatastoreIO.Write.named("WriteDatastore")); + p.apply(Create.of()) + .setCoder(EntityCoder.of()) + .apply(DatastoreIO.write().named("WriteDatastore")); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormatTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormatTest.java new file mode 100644 index 0000000000000..91d0003eea20c --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormatTest.java @@ -0,0 +1,218 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.dataflow; + +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudSourceOperationRequestToSourceOperationRequest; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.dictionaryToCloudSource; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceOperationResponseToCloudSourceOperationResponse; +import static com.google.cloud.dataflow.sdk.util.Structs.getDictionary; +import static com.google.cloud.dataflow.sdk.util.Structs.getObject; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.SourceOperationRequest; +import com.google.api.services.dataflow.model.SourceSplitRequest; +import com.google.api.services.dataflow.model.SourceSplitResponse; +import com.google.api.services.dataflow.model.SourceSplitShard; +import com.google.api.services.dataflow.model.Step; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.ReadSource; +import com.google.cloud.dataflow.sdk.io.Source; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.CloudSourceUtils; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.SourceFormat; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import javax.annotation.Nullable; + +/** + * Tests for {@code BasicSerializableSourceFormat}. + */ +@RunWith(JUnit4.class) +public class BasicSerializableSourceFormatTest { + static class TestIO { + public static Read fromRange(int from, int to) { + return new Read(from, to); + } + + static class Read extends Source { + final int from; + final int to; + + Read(int from, int to) { + this.from = from; + this.to = to; + } + + @Override + public List splitIntoShards(long desiredShardSizeBytes, PipelineOptions options) + throws Exception { + List res = new ArrayList<>(); + DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); + float step = 1.0f * (to - from) / dataflowOptions.getNumWorkers(); + for (int i = 0; i < dataflowOptions.getNumWorkers(); ++i) { + res.add(new Read(Math.round(from + i * step), Math.round(from + (i + 1) * step))); + } + return res; + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + return 8 * (to - from); + } + + @Override + public boolean producesSortedKeys(PipelineOptions options) throws Exception { + return true; + } + + @Override + public Reader createBasicReader( + PipelineOptions options, Coder coder, + @Nullable ExecutionContext executionContext) throws IOException { + return new RangeReader(from, to); + } + + @Override + public void validate() {} + + @Override + public Coder getDefaultOutputCoder() { + return BigEndianIntegerCoder.of(); + } + + private class RangeReader implements Reader { + private int to; + private int current; + + public RangeReader(int from, int to) { + this.to = to; + this.current = from - 1; + } + + @Override + public boolean advance() throws IOException { + current++; + return (current < to); + } + + @Override + public Integer getCurrent() { + return current; + } + + @Override + public void close() throws IOException { + // Nothing + } + } + } + } + + @Test + public void testSplitAndReadShardsBack() throws Exception { + DataflowPipelineOptions options = + PipelineOptionsFactory.create().as(DataflowPipelineOptions.class); + options.setNumWorkers(5); + com.google.api.services.dataflow.model.Source source = translateIOToCloudSource( + TestIO.fromRange(10, 20), options); + List> elems = CloudSourceUtils.readElemsFromSource(options, source); + assertEquals(10, elems.size()); + for (int i = 0; i < 10; ++i) { + assertEquals(WindowedValue.valueInGlobalWindow(10 + i), elems.get(i)); + } + SourceSplitResponse response = performSplit(source, options); + assertEquals("SOURCE_SPLIT_OUTCOME_SPLITTING_HAPPENED", response.getOutcome()); + List shards = response.getShards(); + assertEquals(5, shards.size()); + for (int i = 0; i < 5; ++i) { + SourceSplitShard shard = shards.get(i); + assertEquals("SOURCE_DERIVATION_MODE_INDEPENDENT", shard.getDerivationMode()); + com.google.api.services.dataflow.model.Source shardSource = shard.getSource(); + assertTrue(shardSource.getDoesNotNeedSplitting()); + shardSource.setCodec(source.getCodec()); + List> xs = CloudSourceUtils.readElemsFromSource(options, shardSource); + assertThat( + xs, + contains( + WindowedValue.valueInGlobalWindow(10 + 2 * i), + WindowedValue.valueInGlobalWindow(11 + 2 * i))); + } + } + + private static com.google.api.services.dataflow.model.Source translateIOToCloudSource( + TestIO.Read io, DataflowPipelineOptions options) throws Exception { + DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options); + Pipeline p = Pipeline.create(options); + p.begin().apply(ReadSource.from(io)); + + Job workflow = translator.translate(p, new ArrayList()); + Step step = workflow.getSteps().get(0); + + return stepToCloudSource(step); + } + + private static com.google.api.services.dataflow.model.Source stepToCloudSource(Step step) + throws Exception { + com.google.api.services.dataflow.model.Source res = dictionaryToCloudSource( + getDictionary(step.getProperties(), PropertyNames.SOURCE_STEP_INPUT)); + // Encoding is specified in the step, not in the source itself. This is + // normal: incoming Dataflow API Source objects in map tasks will have the + // encoding filled in from the step's output encoding. + CloudObject encoding = CloudObject.fromSpec(getObject( + // TODO: This should be done via a Structs accessor. + ((List>) step.getProperties().get(PropertyNames.OUTPUT_INFO)).get(0), + PropertyNames.ENCODING)); + res.setCodec(encoding); + return res; + } + + private SourceSplitResponse performSplit( + com.google.api.services.dataflow.model.Source source, PipelineOptions options) + throws Exception { + SourceSplitRequest splitRequest = new SourceSplitRequest(); + splitRequest.setSource(source); + SourceOperationRequest request = new SourceOperationRequest(); + request.setSplit(splitRequest); + SourceFormat.OperationRequest request1 = + cloudSourceOperationRequestToSourceOperationRequest(request); + SourceFormat.OperationResponse response = + new BasicSerializableSourceFormat(options).performSourceOperation(request1); + return sourceOperationResponseToCloudSourceOperationResponse(response).getSplit(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java index 18248ef3183ad..f524e74744bfe 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java @@ -129,7 +129,7 @@ private TableDataList rawDataList(TableRow... rows) { } @Test - public void testRead() throws IOException { + public void testRead() throws Exception { onTableGet(basicTableSchema()); // BQ API data is always encoded as a string diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DatastoreReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DatastoreReaderTest.java new file mode 100644 index 0000000000000..93eb27cb57036 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DatastoreReaderTest.java @@ -0,0 +1,143 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.api.services.datastore.client.DatastoreHelper.makeProperty; +import static com.google.api.services.datastore.client.DatastoreHelper.makeValue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.argThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.api.services.datastore.DatastoreV1.Entity; +import com.google.api.services.datastore.DatastoreV1.EntityResult; +import com.google.api.services.datastore.DatastoreV1.EntityResult.ResultType; +import com.google.api.services.datastore.DatastoreV1.Query; +import com.google.api.services.datastore.DatastoreV1.QueryResultBatch; +import com.google.api.services.datastore.DatastoreV1.QueryResultBatch.MoreResultsType; +import com.google.api.services.datastore.DatastoreV1.RunQueryRequest; +import com.google.api.services.datastore.DatastoreV1.RunQueryResponse; +import com.google.api.services.datastore.client.Datastore; +import com.google.api.services.datastore.client.DatastoreException; +import com.google.cloud.dataflow.sdk.io.DatastoreIO; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentMatcher; + +import java.util.ArrayList; +import java.util.List; + +/** + * Unit tests for {@code DatastoreSource}. + */ +@RunWith(JUnit4.class) +public class DatastoreReaderTest { + private static final String TEST_HOST = "http://localhost:8080"; + private static final String TEST_KIND = "mykind"; + private static final String TEST_DATASET = "mydataset"; + private static final String TEST_PROPERTY = "myproperty"; + + private static class IsValidRequest extends ArgumentMatcher { + @Override + public boolean matches(Object o) { + RunQueryRequest request = (RunQueryRequest) o; + return request.hasQuery(); + } + } + + private EntityResult createEntityResult(String kind, String val) { + Entity entity = Entity.newBuilder().addProperty( + makeProperty(TEST_PROPERTY, makeValue(val))).build(); + return EntityResult.newBuilder().setEntity(entity).build(); + } + + private Datastore buildMockDatastore() throws DatastoreException { + Datastore datastore = mock(Datastore.class); + RunQueryResponse.Builder firstResponseBuilder = RunQueryResponse.newBuilder(); + RunQueryResponse.Builder secondResponseBuilder = RunQueryResponse.newBuilder(); + RunQueryResponse.Builder thirdResponseBuilder = RunQueryResponse.newBuilder(); + { + QueryResultBatch.Builder resultsBatch = QueryResultBatch.newBuilder(); + resultsBatch.addEntityResult(0, createEntityResult(TEST_KIND, "val0")); + resultsBatch.addEntityResult(1, createEntityResult(TEST_KIND, "val1")); + resultsBatch.addEntityResult(2, createEntityResult(TEST_KIND, "val2")); + resultsBatch.addEntityResult(3, createEntityResult(TEST_KIND, "val3")); + resultsBatch.addEntityResult(4, createEntityResult(TEST_KIND, "val4")); + resultsBatch.setEntityResultType(ResultType.FULL); + + resultsBatch.setMoreResults(MoreResultsType.NOT_FINISHED); + + firstResponseBuilder.setBatch(resultsBatch.build()); + } + { + QueryResultBatch.Builder resultsBatch = QueryResultBatch.newBuilder(); + resultsBatch.addEntityResult(0, createEntityResult(TEST_KIND, "val5")); + resultsBatch.addEntityResult(1, createEntityResult(TEST_KIND, "val6")); + resultsBatch.addEntityResult(2, createEntityResult(TEST_KIND, "val7")); + resultsBatch.addEntityResult(3, createEntityResult(TEST_KIND, "val8")); + resultsBatch.addEntityResult(4, createEntityResult(TEST_KIND, "val9")); + resultsBatch.setEntityResultType(ResultType.FULL); + + resultsBatch.setMoreResults(MoreResultsType.NOT_FINISHED); + + secondResponseBuilder.setBatch(resultsBatch.build()); + } + { + QueryResultBatch.Builder resultsBatch = QueryResultBatch.newBuilder(); + resultsBatch.setEntityResultType(ResultType.FULL); + + resultsBatch.setMoreResults(MoreResultsType.NO_MORE_RESULTS); + + thirdResponseBuilder.setBatch(resultsBatch.build()); + } + when(datastore.runQuery(argThat(new IsValidRequest()))) + .thenReturn(firstResponseBuilder.build()) + .thenReturn(secondResponseBuilder.build()) + .thenReturn(thirdResponseBuilder.build()); + return datastore; + } + + + @Test + public void testRead() throws Exception { + Datastore datastore = buildMockDatastore(); + + Query.Builder q = Query.newBuilder(); + q.addKindBuilder().setName(TEST_KIND); + Query query = q.build(); + + DatastoreIO.DatastoreReader iterator = new DatastoreIO.DatastoreReader(query, datastore); + + List entityResults = new ArrayList(); + while (iterator.advance()) { + entityResults.add(iterator.getCurrent()); + } + + assertEquals(10, entityResults.size()); + for (int i = 0; i < 10; i++) { + assertNotNull(entityResults.get(i).getPropertyList()); + assertEquals(entityResults.get(i).getPropertyList().size(), 1); + assertTrue(entityResults.get(i).getPropertyList().get(0).hasValue()); + assertTrue(entityResults.get(i).getPropertyList().get(0).getValue().hasStringValue()); + assertEquals( + entityResults.get(i).getPropertyList().get(0).getValue().getStringValue(), "val" + i); + } + } +} From 6cd24e2bc7ce82bd60710633a1791b48b253efe7 Mon Sep 17 00:00:00 2001 From: lcwik Date: Mon, 12 Jan 2015 09:28:58 -0800 Subject: [PATCH 0079/1541] Update the worker harness to request one task for each available processor. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83762900 --- .../runners/worker/DataflowWorkerHarness.java | 52 ++++++++++++++++--- .../worker/DataflowWorkerHarnessTest.java | 17 +++--- 2 files changed, 56 insertions(+), 13 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java index f46a8d4a00039..ed197b3b15147 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java @@ -19,6 +19,7 @@ import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudTime; +import com.google.api.client.util.Lists; import com.google.api.client.util.Preconditions; import com.google.api.services.dataflow.Dataflow; import com.google.api.services.dataflow.model.LeaseWorkItemRequest; @@ -40,7 +41,9 @@ import com.google.common.collect.ImmutableList; import org.joda.time.DateTime; +import org.joda.time.DateTimeUtils; import org.joda.time.Duration; +import org.joda.time.format.ISODateTimeFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.MDC; @@ -49,6 +52,9 @@ import java.lang.Thread.UncaughtExceptionHandler; import java.util.Collections; import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutorCompletionService; import javax.annotation.concurrent.ThreadSafe; @@ -92,17 +98,49 @@ public static void main(String[] args) throws Exception { Thread.currentThread().setUncaughtExceptionHandler(WorkerUncaughtExceptionHandler.INSTANCE); new DataflowWorkerLoggingInitializer().initialize(); - DataflowWorker worker = createFromSystemProperties(); - processWork(worker); + DataflowWorkerHarnessOptions pipelineOptions = + PipelineOptionsFactory.createFromSystemProperties(); + final DataflowWorker worker = create(pipelineOptions); + processWork(pipelineOptions, worker); } // Visible for testing. - static void processWork(DataflowWorker worker) throws IOException { - worker.getAndPerformWork(); - } + static void processWork(DataflowWorkerHarnessOptions pipelineOptions, + final DataflowWorker worker) { + + long startTime = DateTimeUtils.currentTimeMillis(); + int numThreads = Math.max(Runtime.getRuntime().availableProcessors() - 1, 1); + CompletionService completionService = + new ExecutorCompletionService<>(pipelineOptions.getExecutorService()); + for (int i = 0; i < numThreads; ++i) { + completionService.submit(new Callable() { + @Override + public Boolean call() throws Exception { + return worker.getAndPerformWork(); + } + }); + } + + List completionTimes = Lists.newArrayList(); + for (int i = 0; i < numThreads; ++i) { + try { + // CompletionService returns the tasks in the order in which the completed at. + completionService.take().get(); + } catch (Exception e) { + LOG.error("Failed waiting on thread to process work.", e); + } + completionTimes.add(DateTimeUtils.currentTimeMillis()); + } - static DataflowWorker createFromSystemProperties() { - return create(PipelineOptionsFactory.createFromSystemProperties()); + long endTime = DateTimeUtils.currentTimeMillis(); + LOG.info("processWork() start time: {}, end time: {}", + ISODateTimeFormat.dateTime().print(startTime), + ISODateTimeFormat.dateTime().print(endTime)); + for (long completionTime : completionTimes) { + LOG.info("Duration: {}ms Wasted Time: {}ms", + completionTime - startTime, + endTime - completionTime); + } } static DataflowWorker create(DataflowWorkerHarnessOptions options) { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java index d1d369fe99acd..e167aab7cf1ba 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertNull; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doCallRealMethod; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; @@ -64,6 +65,7 @@ public class DataflowWorkerHarnessTest { @Mock private MockHttpTransport transport; @Mock private MockLowLevelHttpRequest request; @Mock private DataflowWorker mockDataflowWorker; + private DataflowWorkerHarnessOptions pipelineOptions; private Dataflow service; @@ -74,21 +76,24 @@ public void setUp() throws Exception { doCallRealMethod().when(request).getContentAsString(); service = new Dataflow(transport, Transport.getJsonFactory(), null); + pipelineOptions = PipelineOptionsFactory.as(DataflowWorkerHarnessOptions.class); } @Test - public void testThatWeOnlyProcessWorkOnce() throws Exception { + public void testThatWeOnlyProcessWorkOncePerAvailableProcessor() throws Exception { + int numWorkers = Math.max(Runtime.getRuntime().availableProcessors() - 1, 1); when(mockDataflowWorker.getAndPerformWork()).thenReturn(true); - DataflowWorkerHarness.processWork(mockDataflowWorker); - verify(mockDataflowWorker).getAndPerformWork(); + DataflowWorkerHarness.processWork(pipelineOptions, mockDataflowWorker); + verify(mockDataflowWorker, times(numWorkers)).getAndPerformWork(); verifyNoMoreInteractions(mockDataflowWorker); } @Test - public void testThatWeOnlyProcessWorkOnceEvenWhenFailing() throws Exception { + public void testThatWeOnlyProcessWorkOncePerAvailableProcessorEvenWhenFailing() throws Exception { + int numWorkers = Math.max(Runtime.getRuntime().availableProcessors() - 1, 1); when(mockDataflowWorker.getAndPerformWork()).thenReturn(false); - DataflowWorkerHarness.processWork(mockDataflowWorker); - verify(mockDataflowWorker).getAndPerformWork(); + DataflowWorkerHarness.processWork(pipelineOptions, mockDataflowWorker); + verify(mockDataflowWorker, times(numWorkers)).getAndPerformWork(); verifyNoMoreInteractions(mockDataflowWorker); } From e8d903972989f950b8c174248e5f2577fb903575 Mon Sep 17 00:00:00 2001 From: lcwik Date: Wed, 14 Jan 2015 10:01:06 -0800 Subject: [PATCH 0080/1541] Updated command line parsing such that unrecognized or malformed command line options will result in an error. To ignore unknown or malformed command line options instead, call withoutStrictParsing() within the PipelineOptionsFactory when constructing PipelineOptions. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83947457 --- .../sdk/options/PipelineOptionsFactory.java | 134 ++++++++++++------ .../options/PipelineOptionsFactoryTest.java | 80 ++++++++++- 2 files changed, 164 insertions(+), 50 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java index 250baa62af277..5de1fa06cbbfe 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java @@ -128,6 +128,10 @@ public static T as(Class klass) { * List style properties are able to be bound to {@code boolean[]}, {@code char[]}, * {@code short[]}, {@code int[]}, {@code long[]}, {@code float[]}, {@code double[]}, * {@code String[]} and {@code List}. + *

+ * By default, strict parsing is enabled and arguments must conform to be either + * {@code --booleanArgName} or {@code --argName=argValue}. Strict parsing can be disabled with + * {@link Builder#withoutStrictParsing()}. */ public static Builder fromArgs(String[] args) { return new Builder(getAppName(3)).fromArgs(args); @@ -148,16 +152,19 @@ public static class Builder { private final String defaultAppName; private final String[] args; private final boolean validation; + private final boolean strictParsing; // Do not allow direct instantiation private Builder(String defaultAppName) { - this(defaultAppName, null, false); + this(defaultAppName, null, false, true); } - private Builder(String defaultAppName, String[] args, boolean validation) { + private Builder(String defaultAppName, String[] args, boolean validation, + boolean strictParsing) { this.defaultAppName = defaultAppName; this.args = args; this.validation = validation; + this.strictParsing = strictParsing; } /** @@ -178,10 +185,14 @@ private Builder(String defaultAppName, String[] args, boolean validation) { * List style properties are able to be bound to {@code boolean[]}, {@code char[]}, * {@code short[]}, {@code int[]}, {@code long[]}, {@code float[]}, {@code double[]}, * {@code String[]} and {@code List}. + *

+ * By default, strict parsing is enabled and arguments must conform to be either + * {@code --booleanArgName} or {@code --argName=argValue}. Strict parsing can be disabled with + * {@link Builder#withoutStrictParsing()}. */ public Builder fromArgs(String[] args) { Preconditions.checkNotNull(args, "Arguments should not be null."); - return new Builder(defaultAppName, args, validation); + return new Builder(defaultAppName, args, validation, strictParsing); } /** @@ -191,7 +202,15 @@ public Builder fromArgs(String[] args) { * validation. */ public Builder withValidation() { - return new Builder(defaultAppName, args, true); + return new Builder(defaultAppName, args, true, strictParsing); + } + + /** + * During parsing of the arguments, we will skip over improperly formatted and unknown + * arguments. + */ + public Builder withoutStrictParsing() { + return new Builder(defaultAppName, args, validation, false); } /** @@ -218,9 +237,9 @@ public T as(Class klass) { // Attempt to parse the arguments into the set of initial options to use if (args != null) { - ListMultimap options = parseCommandLine(args); + ListMultimap options = parseCommandLine(args, strictParsing); LOG.debug("Provided Arguments: {}", options); - initialOptions = parseObjects(klass, options); + initialOptions = parseObjects(klass, options, strictParsing); } // Create our proxy @@ -814,20 +833,35 @@ public boolean apply(Method input) { *

List style properties are able to be bound to {@code boolean[]}, {@code char[]}, * {@code short[]}, {@code int[]}, {@code long[]}, {@code float[]}, {@code double[]}, * {@code String[]}, and {@code List}. + * + *

If strict parsing is enabled, options must start with '--', and not have an empty argument + * name or value based upon the positioning of the '='. */ - private static ListMultimap parseCommandLine(String[] args) { + private static ListMultimap parseCommandLine( + String[] args, boolean strictParsing) { ImmutableListMultimap.Builder builder = ImmutableListMultimap.builder(); for (String arg : args) { - Preconditions.checkArgument(arg.startsWith("--"), - "Unknown argument %s in command line %s", arg, Arrays.toString(args)); - int index = arg.indexOf("="); - // Make sure that '=' isn't the first character after '--' or the last character - Preconditions.checkArgument(index != 2 && index != arg.length() - 1, - "Unknown argument %s in command line %s", arg, Arrays.toString(args)); - if (index > 0) { - builder.put(arg.substring(2, index), arg.substring(index + 1, arg.length())); - } else { - builder.put(arg.substring(2), "true"); + try { + Preconditions.checkArgument(arg.startsWith("--"), + "Argument '%s' does not begin with '--'", arg); + int index = arg.indexOf("="); + // Make sure that '=' isn't the first character after '--' or the last character + Preconditions.checkArgument(index != 2, + "Argument '%s' starts with '--=', empty argument name not allowed", arg); + Preconditions.checkArgument(index != arg.length() - 1, + "Argument '%s' ends with '=', empty argument value not allowed", arg); + if (index > 0) { + builder.put(arg.substring(2, index), arg.substring(index + 1, arg.length())); + } else { + builder.put(arg.substring(2), "true"); + } + } catch (IllegalArgumentException e) { + if (strictParsing) { + throw e; + } else { + LOG.warn("Strict parsing is disabled, ignoring option '{}' because {}", + arg, e.getMessage()); + } } } return builder.build(); @@ -842,9 +876,12 @@ private static ListMultimap parseCommandLine(String[] args) { *

* We special case the "runner" option. It is mapped to the class of the {@link PipelineRunner} * based off of the {@link PipelineRunner}s simple class name. + *

+ * If strict parsing is enabled, unknown options or options which can not be converted to + * the expected java type using an {@link ObjectMapper} will be ignored. */ private static Map parseObjects( - Class klass, ListMultimap options) { + Class klass, ListMultimap options, boolean strictParsing) { Map propertyNamesToGetters = Maps.newHashMap(); PipelineOptionsFactory.validateWellFormed(klass, getRegisteredOptions()); @SuppressWarnings("unchecked") @@ -856,33 +893,40 @@ private static Map parseObjects( } Map convertedOptions = Maps.newHashMap(); for (Map.Entry> entry : options.asMap().entrySet()) { - if (!propertyNamesToGetters.containsKey(entry.getKey())) { - LOG.warn("Ignoring argument {}={}", entry.getKey(), entry.getValue()); - continue; - } - - Method method = propertyNamesToGetters.get(entry.getKey()); - JavaType type = MAPPER.getTypeFactory().constructType(method.getGenericReturnType()); - if ("runner".equals(entry.getKey())) { - String runner = Iterables.getOnlyElement(entry.getValue()); - Preconditions.checkArgument(SUPPORTED_PIPELINE_RUNNERS.containsKey(runner), - "Unknown 'runner' specified %s, supported pipeline runners %s", - runner, Sets.newTreeSet(SUPPORTED_PIPELINE_RUNNERS.keySet())); - convertedOptions.put("runner", SUPPORTED_PIPELINE_RUNNERS.get(runner)); - } else if (method.getReturnType().isArray() - || Collection.class.isAssignableFrom(method.getReturnType())) { - // Split any strings with "," - List values = FluentIterable.from(entry.getValue()) - .transformAndConcat(new Function>() { - @Override - public Iterable apply(String input) { - return Arrays.asList(input.split(",")); - } - }).toList(); - convertedOptions.put(entry.getKey(), MAPPER.convertValue(values, type)); - } else { - String value = Iterables.getOnlyElement(entry.getValue()); - convertedOptions.put(entry.getKey(), MAPPER.convertValue(value, type)); + try { + Preconditions.checkArgument(propertyNamesToGetters.containsKey(entry.getKey()), + "Class %s missing a property named '%s'", klass, entry.getKey()); + + Method method = propertyNamesToGetters.get(entry.getKey()); + JavaType type = MAPPER.getTypeFactory().constructType(method.getGenericReturnType()); + if ("runner".equals(entry.getKey())) { + String runner = Iterables.getOnlyElement(entry.getValue()); + Preconditions.checkArgument(SUPPORTED_PIPELINE_RUNNERS.containsKey(runner), + "Unknown 'runner' specified '%s', supported pipeline runners %s", + runner, Sets.newTreeSet(SUPPORTED_PIPELINE_RUNNERS.keySet())); + convertedOptions.put("runner", SUPPORTED_PIPELINE_RUNNERS.get(runner)); + } else if (method.getReturnType().isArray() + || Collection.class.isAssignableFrom(method.getReturnType())) { + // Split any strings with "," + List values = FluentIterable.from(entry.getValue()) + .transformAndConcat(new Function>() { + @Override + public Iterable apply(String input) { + return Arrays.asList(input.split(",")); + } + }).toList(); + convertedOptions.put(entry.getKey(), MAPPER.convertValue(values, type)); + } else { + String value = Iterables.getOnlyElement(entry.getValue()); + convertedOptions.put(entry.getKey(), MAPPER.convertValue(value, type)); + } + } catch (IllegalArgumentException e) { + if (strictParsing) { + throw e; + } else { + LOG.warn("Strict parsing is disabled, ignoring option '{}' with value '{}' because {}", + entry.getKey(), entry.getValue(), e.getMessage()); + } } } return convertedOptions; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java index d5fb27d9203c1..8ec5eb25e125b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java @@ -24,6 +24,7 @@ import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.testing.ExpectedLogs; import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -44,6 +45,7 @@ public class PipelineOptionsFactoryTest { @Rule public ExpectedException expectedException = ExpectedException.none(); @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); + @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(PipelineOptionsFactory.class); @Test public void testAutomaticRegistrationOfPipelineOptions() { @@ -494,6 +496,15 @@ public void testSetASingularAttributeUsingAListThrowsAnError() { PipelineOptionsFactory.fromArgs(args).create(); } + @Test + public void testSetASingularAttributeUsingAListIsIgnoredWithoutStrictParsing() { + String[] args = new String[] { + "--diskSizeGb=100", + "--diskSizeGb=200"}; + expectedLogs.expectWarn("Strict parsing is disabled, ignoring option"); + PipelineOptionsFactory.fromArgs(args).withoutStrictParsing().create(); + } + @Test public void testSettingRunner() { String[] args = new String[] {"--runner=BlockingDataflowPipelineRunner"}; @@ -504,12 +515,71 @@ public void testSettingRunner() { @Test public void testSettingUnknownRunner() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Unknown 'runner' specified UnknownRunner, supported pipeline " - + "runners [BlockingDataflowPipelineRunner, DataflowPipelineRunner, DirectPipelineRunner]"); String[] args = new String[] {"--runner=UnknownRunner"}; + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Unknown 'runner' specified 'UnknownRunner', supported " + + "pipeline runners [BlockingDataflowPipelineRunner, DataflowPipelineRunner, " + + "DirectPipelineRunner]"); + PipelineOptionsFactory.fromArgs(args).create(); + } - PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create(); - options.getRunner(); + @Test + public void testUsingArgumentWithUnknownPropertyIsNotAllowed() { + String[] args = new String[] {"--unknownProperty=value"}; + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("missing a property named 'unknownProperty'"); + PipelineOptionsFactory.fromArgs(args).create(); + } + + @Test + public void testUsingArgumentWithUnknownPropertyIsIgnoredWithoutStrictParsing() { + String[] args = new String[] {"--unknownProperty=value"}; + expectedLogs.expectWarn("missing a property named 'unknownProperty'"); + PipelineOptionsFactory.fromArgs(args).withoutStrictParsing().create(); + } + + @Test + public void testUsingArgumentWithoutValueIsNotAllowed() { + String[] args = new String[] {"--diskSizeGb="}; + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Argument '--diskSizeGb=' ends with '='"); + PipelineOptionsFactory.fromArgs(args).create(); + } + + @Test + public void testUsingArgumentWithoutValueIsIgnoredWithoutStrictParsing() { + String[] args = new String[] {"--diskSizeGb="}; + expectedLogs.expectWarn("Strict parsing is disabled, ignoring option"); + PipelineOptionsFactory.fromArgs(args).withoutStrictParsing().create(); + } + + @Test + public void testUsingArgumentStartingWithIllegalCharacterIsNotAllowed() { + String[] args = new String[] {" --diskSizeGb=100"}; + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Argument ' --diskSizeGb=100' does not begin with '--'"); + PipelineOptionsFactory.fromArgs(args).create(); + } + + @Test + public void testUsingArgumentStartingWithIllegalCharacterIsIgnoredWithoutStrictParsing() { + String[] args = new String[] {" --diskSizeGb=100"}; + expectedLogs.expectWarn("Strict parsing is disabled, ignoring option"); + PipelineOptionsFactory.fromArgs(args).withoutStrictParsing().create(); + } + + @Test + public void testUsingArgumentWithInvalidNameIsNotAllowed() { + String[] args = new String[] {"--=100"}; + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Argument '--=100' starts with '--='"); + PipelineOptionsFactory.fromArgs(args).create(); + } + + @Test + public void testUsingArgumentWithInvalidNameIsIgnoredWithoutStrictParsing() { + String[] args = new String[] {"--=100"}; + expectedLogs.expectWarn("Strict parsing is disabled, ignoring option"); + PipelineOptionsFactory.fromArgs(args).withoutStrictParsing().create(); } } From b7acea01f5c0d99ef096854b5fe18618dd8ca9aa Mon Sep 17 00:00:00 2001 From: amyu Date: Wed, 14 Jan 2015 15:05:07 -0800 Subject: [PATCH 0081/1541] Change join example to use non-static tags. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=83977086 --- .../cloud/dataflow/examples/JoinExamples.java | 41 +++++++++---------- 1 file changed, 19 insertions(+), 22 deletions(-) diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/JoinExamples.java b/examples/src/main/java/com/google/cloud/dataflow/examples/JoinExamples.java index 5f910d6dfbcf4..803de8aad7f71 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/JoinExamples.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/JoinExamples.java @@ -60,27 +60,6 @@ public class JoinExamples { // A table that maps country codes to country names. private static final String COUNTRY_CODES = "gdelt-bq:full.crosswalk_geocountrycodetohuman"; - private static final TupleTag eventInfoTag = new TupleTag(); - private static final TupleTag countryInfoTag = new TupleTag(); - - /** - * Process the CoGbkResult elements generated by the CoGroupByKey transform. - */ - static class ProcessJoinFn extends DoFn, KV> { - - @Override - public void processElement(ProcessContext c) { - KV e = c.element(); - CoGbkResult val = e.getValue(); - String countryCode = e.getKey(); - String countryName = "none"; - countryName = e.getValue().getOnly(countryInfoTag); - for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) { - // Generate a string that combines information from both collection values - c.output(KV.of(countryCode, "Country name: " + countryName + ", Event info: " + eventInfo)); - } - } - } /** * Join two collections, using country code as the key. @@ -88,6 +67,9 @@ public void processElement(ProcessContext c) { static PCollection joinEvents(PCollection eventsTable, PCollection countryCodes) throws Exception { + final TupleTag eventInfoTag = new TupleTag(); + final TupleTag countryInfoTag = new TupleTag(); + // transform both input collections to tuple collections, where the keys are country // codes in both cases. PCollection> eventInfo = eventsTable.apply( @@ -101,9 +83,24 @@ static PCollection joinEvents(PCollection eventsTable, .and(countryInfoTag, countryInfo) .apply(CoGroupByKey.create()); + // Process the CoGbkResult elements generated by the CoGroupByKey transform. // country code 'key' -> string of , PCollection> finalResultCollection = - kvpCollection.apply(ParDo.of(new ProcessJoinFn())); + kvpCollection.apply(ParDo.of(new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) { + KV e = c.element(); + CoGbkResult val = e.getValue(); + String countryCode = e.getKey(); + String countryName = "none"; + countryName = e.getValue().getOnly(countryInfoTag); + for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) { + // Generate a string that combines information from both collection values + c.output(KV.of(countryCode, "Country name: " + countryName + + ", Event info: " + eventInfo)); + } + } + })); // write to GCS PCollection formattedResults = finalResultCollection From 0d62c34e098543838f73c7e0f6474c654fcdfffa Mon Sep 17 00:00:00 2001 From: klk Date: Thu, 15 Jan 2015 10:08:16 -0800 Subject: [PATCH 0082/1541] Undo deprecation of PTransform.getInput and PTransform.getOutput. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84051898 --- .../com/google/cloud/dataflow/sdk/transforms/PTransform.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java index 6fc2618dcc021..d7df4977a3992 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java @@ -237,9 +237,7 @@ public Pipeline getPipeline() { * Returns the input of this transform. * * @throws IllegalStateException if this PTransform hasn't been applied yet - * @deprecated Use pipeline.getInput(transform) */ - @Deprecated public Input getInput() { @SuppressWarnings("unchecked") Input input = (Input) getPipeline().getInput(this); @@ -250,9 +248,7 @@ public Input getInput() { * Returns the output of this transform. * * @throws IllegalStateException if this PTransform hasn't been applied yet - * #deprecated use pipeline.getOutput(transform) */ - @Deprecated public Output getOutput() { @SuppressWarnings("unchecked") Output output = (Output) getPipeline().getOutput(this); From d828500122b3fcd4382ab1bc8f29ec1a7b775665 Mon Sep 17 00:00:00 2001 From: lcwik Date: Thu, 15 Jan 2015 11:23:11 -0800 Subject: [PATCH 0083/1541] Add support for Google API tracing to the SDK. To help debug communication issues between the SDK and Google services, a token may be given to you by a Googler which will record the inputs/outputs of some or all API calls between the SDK and Google services. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84058790 --- .../dataflow/sdk/options/GcpOptions.java | 2 +- .../sdk/options/GoogleApiDebugOptions.java | 103 ++++++++++++++ .../cloud/dataflow/sdk/util/Transport.java | 48 ++++++- .../options/GoogleApiDebugOptionsTest.java | 134 ++++++++++++++++++ 4 files changed, 280 insertions(+), 7 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptionsTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java index 959a4fc77f3fb..52354d92a0c35 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java @@ -56,7 +56,7 @@ * application default credentials falling back to gcloud. The other options can be * used by setting the corresponding properties. */ -public interface GcpOptions extends PipelineOptions { +public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions { /** * Project id to use when launching jobs. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java new file mode 100644 index 0000000000000..72e4bae0032d8 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java @@ -0,0 +1,103 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.api.client.googleapis.services.AbstractGoogleClient; +import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; +import com.google.api.client.googleapis.services.GoogleClientRequestInitializer; +import com.google.common.base.Preconditions; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.IOException; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * These options configure debug settings for Google API clients generated by the Dataflow SDK. + */ +public interface GoogleApiDebugOptions extends PipelineOptions { + /** + * This option enables tracing of API calls to Google services used within the Dataflow SDK. + * A tracing token must be requested from Google to be able to use this option. + * An invalid tracing token will result in 400 errors from Google when the API is invoked. + */ + GoogleApiTracer[] getGoogleApiTrace(); + void setGoogleApiTrace(GoogleApiTracer[] commands); + + /** + * A {@link GoogleClientRequestInitializer} which adds the 'trace' token to Google API calls. + */ + public static class GoogleApiTracer implements GoogleClientRequestInitializer { + private static final Pattern COMMAND_LINE_PATTERN = Pattern.compile("([^#]*)#(.*)"); + /** + * Creates a {@link GoogleApiTracer} which sets the trace {@code token} on all + * calls which match the given client type. + */ + public static GoogleApiTracer create(AbstractGoogleClient client, String token) { + return new GoogleApiTracer(client.getClass().getCanonicalName(), token); + } + + /** + * Creates a {@link GoogleApiTracer} which sets the trace {@code token} on all + * calls which match for the given request type. + */ + public static GoogleApiTracer create(AbstractGoogleClientRequest request, String token) { + return new GoogleApiTracer(request.getClass().getCanonicalName(), token); + } + + /** + * Creates a {@link GoogleClientRequestInitializer} which adds the trace token + * based upon the passed in value. + *

+ * The {@code value} represents a string containing {@code ApiName#TracingToken}. + * The {@code ApiName} is used to match against the request classes + * {@link Class#getCanonicalName() canonical name} for which to add the {@code TracingToken} to. + * For example, to match: + *

    + *
  • all Google API calls: {@code #TracingToken} + *
  • all Dataflow API calls: {@code Dataflow#TracingToken} + *
  • all Dataflow V1B3 API calls: {@code Dataflow.V1b3#TracingToken} + *
  • all Dataflow V1B3 Jobs API calls: {@code Dataflow.V1b3.Projects.Jobs#TracingToken} + *
  • all Dataflow V1B3 Jobs Get calls: {@code Dataflow.V1b3.Projects.Jobs.Get#TracingToken} + *
  • all Job creation calls in any version: {@code Jobs.Create#TracingToken} + *
+ */ + @JsonCreator + public static GoogleApiTracer create(String value) { + Matcher matcher = COMMAND_LINE_PATTERN.matcher(value); + Preconditions.checkArgument(matcher.find() && matcher.groupCount() == 2, + "Unable to parse '%s', expected format 'ClientRequestName#Token'", value); + return new GoogleApiTracer(matcher.group(1), matcher.group(2)); + } + + private final String clientRequestName; + private final String token; + + private GoogleApiTracer(String clientRequestName, String token) { + this.clientRequestName = clientRequestName; + this.token = token; + } + + @Override + public void initialize(AbstractGoogleClientRequest request) throws IOException { + if (request.getClass().getCanonicalName().contains(clientRequestName)) { + request.set("trace", token); + } + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java index d1accb2ac191f..c0c9485247f66 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java @@ -17,6 +17,9 @@ package com.google.cloud.dataflow.sdk.util; import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport; +import com.google.api.client.googleapis.services.AbstractGoogleClient.Builder; +import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; +import com.google.api.client.googleapis.services.GoogleClientRequestInitializer; import com.google.api.client.http.HttpTransport; import com.google.api.client.json.JsonFactory; import com.google.api.client.json.jackson2.JacksonFactory; @@ -31,6 +34,7 @@ import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; import com.google.cloud.dataflow.sdk.options.GcsOptions; import com.google.cloud.dataflow.sdk.options.StreamingOptions; +import com.google.common.base.MoreObjects; import java.io.IOException; import java.net.MalformedURLException; @@ -78,10 +82,12 @@ public static JsonFactory getJsonFactory() { newBigQueryClient(BigQueryOptions options) { return new Bigquery.Builder(getTransport(), getJsonFactory(), new RetryHttpRequestInitializer(options.getGcpCredential())) - .setApplicationName(options.getAppName()); + .setApplicationName(options.getAppName()) + .setGoogleClientRequestInitializer( + new ChainedGoogleClientRequestInitializer(options.getGoogleApiTrace())); } -/** + /** * Returns a Pubsub client builder. *

* Note: this client's endpoint is not modified by the @@ -91,7 +97,9 @@ public static JsonFactory getJsonFactory() { newPubsubClient(StreamingOptions options) { return new Pubsub.Builder(getTransport(), getJsonFactory(), new RetryHttpRequestInitializer(options.getGcpCredential())) - .setApplicationName(options.getAppName()); + .setApplicationName(options.getAppName()) + .setGoogleClientRequestInitializer( + new ChainedGoogleClientRequestInitializer(options.getGoogleApiTrace())); } /** @@ -116,7 +124,9 @@ public static Dataflow.Builder newDataflowClient(DataflowPipelineOptions options new RetryHttpRequestInitializer(options.getGcpCredential())) .setApplicationName(options.getAppName()) .setRootUrl(rootUrl) - .setServicePath(servicePath); + .setServicePath(servicePath) + .setGoogleClientRequestInitializer( + new ChainedGoogleClientRequestInitializer(options.getGoogleApiTrace())); } /** @@ -126,7 +136,9 @@ public static Dataflow.Builder newDataflowClient(DataflowPipelineOptions options public static Dataflow.Builder newRawDataflowClient(DataflowPipelineOptions options) { return newDataflowClient(options) - .setHttpRequestInitializer(options.getGcpCredential()); + .setHttpRequestInitializer(options.getGcpCredential()) + .setGoogleClientRequestInitializer( + new ChainedGoogleClientRequestInitializer(options.getGoogleApiTrace())); } /** @@ -142,6 +154,30 @@ public static Dataflow.Builder newDataflowClient(DataflowPipelineOptions options // Do not log the code 404. Code up the stack will deal with 404's if needed, and // logging it by default clutters the output during file staging. options.getGcpCredential(), NanoClock.SYSTEM, Sleeper.DEFAULT, Arrays.asList(404))) - .setApplicationName(options.getAppName()); + .setApplicationName(options.getAppName()) + .setGoogleClientRequestInitializer( + new ChainedGoogleClientRequestInitializer(options.getGoogleApiTrace())); + } + + /** + * Allows multiple {@link GoogleClientRequestInitializer}s to be chained together for use with + * {@link Builder}. + */ + private static final class ChainedGoogleClientRequestInitializer + implements GoogleClientRequestInitializer { + private static final GoogleClientRequestInitializer[] EMPTY_ARRAY = + new GoogleClientRequestInitializer[]{}; + private final GoogleClientRequestInitializer[] chain; + + private ChainedGoogleClientRequestInitializer(GoogleClientRequestInitializer... initializer) { + this.chain = MoreObjects.firstNonNull(initializer, EMPTY_ARRAY); + } + + @Override + public void initialize(AbstractGoogleClientRequest request) throws IOException { + for (GoogleClientRequestInitializer initializer : chain) { + initializer.initialize(request); + } + } } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptionsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptionsTest.java new file mode 100644 index 0000000000000..147021a710e12 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptionsTest.java @@ -0,0 +1,134 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import com.google.api.services.bigquery.Bigquery.Datasets.Delete; +import com.google.api.services.dataflow.Dataflow.V1b3.Projects.Jobs.Create; +import com.google.api.services.dataflow.Dataflow.V1b3.Projects.Jobs.Get; +import com.google.cloud.dataflow.sdk.options.GoogleApiDebugOptions.GoogleApiTracer; +import com.google.cloud.dataflow.sdk.util.TestCredential; +import com.google.cloud.dataflow.sdk.util.Transport; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link GoogleApiDebugOptions}. */ +@RunWith(JUnit4.class) +public class GoogleApiDebugOptionsTest { + @Test + public void testWhenTracingMatches() throws Exception { + String[] args = new String[] {"--googleApiTrace=Projects.Jobs.Get#GetTestToken"}; + DataflowPipelineOptions options = + PipelineOptionsFactory.fromArgs(args).as(DataflowPipelineOptions.class); + options.setGcpCredential(new TestCredential()); + + assertNotNull(options.getGoogleApiTrace()); + + Get request = + options.getDataflowClient().v1b3().projects().jobs().get("testProjectId", "testJobId"); + assertEquals("GetTestToken", request.get("trace")); + } + + @Test + public void testWhenTracingDoesNotMatch() throws Exception { + String[] args = new String[] {"--googleApiTrace=Projects.Jobs.Create#testToken"}; + DataflowPipelineOptions options = + PipelineOptionsFactory.fromArgs(args).as(DataflowPipelineOptions.class); + options.setGcpCredential(new TestCredential()); + + assertNotNull(options.getGoogleApiTrace()); + + Get request = + options.getDataflowClient().v1b3().projects().jobs().get("testProjectId", "testJobId"); + assertNull(request.get("trace")); + } + + @Test + public void testWithMultipleTraces() throws Exception { + String[] args = new String[] { + "--googleApiTrace=Projects.Jobs.Create#CreateTestToken,Projects.Jobs.Get#GetTestToken"}; + DataflowPipelineOptions options = + PipelineOptionsFactory.fromArgs(args).as(DataflowPipelineOptions.class); + options.setGcpCredential(new TestCredential()); + + assertNotNull(options.getGoogleApiTrace()); + + Get getRequest = + options.getDataflowClient().v1b3().projects().jobs().get("testProjectId", "testJobId"); + assertEquals("GetTestToken", getRequest.get("trace")); + + Create createRequest = + options.getDataflowClient().v1b3().projects().jobs().create("testProjectId", null); + assertEquals("CreateTestToken", createRequest.get("trace")); + } + + @Test + public void testMatchingAllDataflowV1b3Calls() throws Exception { + String[] args = new String[] {"--googleApiTrace=Dataflow.V1b3#TestToken"}; + DataflowPipelineOptions options = + PipelineOptionsFactory.fromArgs(args).as(DataflowPipelineOptions.class); + options.setGcpCredential(new TestCredential()); + + assertNotNull(options.getGoogleApiTrace()); + + Get getRequest = + options.getDataflowClient().v1b3().projects().jobs().get("testProjectId", "testJobId"); + assertEquals("TestToken", getRequest.get("trace")); + + Create createRequest = + options.getDataflowClient().v1b3().projects().jobs().create("testProjectId", null); + assertEquals("TestToken", createRequest.get("trace")); + } + + @Test + public void testMatchingAgainstClient() throws Exception { + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setGcpCredential(new TestCredential()); + options.setGoogleApiTrace(new GoogleApiTracer[] { + GoogleApiTracer.create(Transport.newDataflowClient(options).build(), "TestToken")}); + + Get getRequest = + options.getDataflowClient().v1b3().projects().jobs().get("testProjectId", "testJobId"); + assertEquals("TestToken", getRequest.get("trace")); + + Delete deleteRequest = Transport.newBigQueryClient(options).build().datasets() + .delete("testProjectId", "testDatasetId"); + assertNull(deleteRequest.get("trace")); + } + + @Test + public void testMatchingAgainstRequestType() throws Exception { + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setGcpCredential(new TestCredential()); + options.setGoogleApiTrace(new GoogleApiTracer[] {GoogleApiTracer.create( + Transport.newDataflowClient(options).build().v1b3().projects().jobs() + .get("aProjectId", "aJobId"), "TestToken")}); + + Get getRequest = + options.getDataflowClient().v1b3().projects().jobs().get("testProjectId", "testJobId"); + assertEquals("TestToken", getRequest.get("trace")); + + Create createRequest = + options.getDataflowClient().v1b3().projects().jobs().create("testProjectId", null); + assertNull(createRequest.get("trace")); + } +} From 20d27ee687576cdb046181f2545b7f59cdae2730 Mon Sep 17 00:00:00 2001 From: millsd Date: Thu, 15 Jan 2015 17:34:51 -0800 Subject: [PATCH 0084/1541] Pick windows for elements produced in start/finishBundle based on the WindowingFn of the input PCollection [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84093449 --- .../runners/DataflowPipelineTranslator.java | 12 ++-- .../runners/worker/AssignWindowsParDoFn.java | 3 +- .../sdk/runners/worker/CombineValuesFn.java | 3 +- .../worker/GroupAlsoByWindowsParDoFn.java | 3 +- .../sdk/runners/worker/NormalParDoFn.java | 21 +++--- .../cloud/dataflow/sdk/transforms/DoFn.java | 50 +++++++------ .../dataflow/sdk/transforms/DoFnTester.java | 4 +- .../cloud/dataflow/sdk/transforms/ParDo.java | 3 +- .../sdk/transforms/windowing/Window.java | 3 +- .../cloud/dataflow/sdk/util/DoFnContext.java | 68 +++++++++++++----- .../cloud/dataflow/sdk/util/DoFnInfo.java | 47 ++++++++++++ .../cloud/dataflow/sdk/util/DoFnRunner.java | 16 +++-- .../dataflow/sdk/util/WindowedValue.java | 3 + .../worker/MapTaskExecutorFactoryTest.java | 8 ++- .../sdk/runners/worker/NormalParDoFnTest.java | 14 ++-- .../runners/worker/ParDoFnFactoryTest.java | 7 +- .../dataflow/sdk/transforms/ParDoTest.java | 71 +++++++++++++++++++ .../sdk/util/GroupAlsoByWindowsDoFnTest.java | 4 +- .../StreamingGroupAlsoByWindowsDoFnTest.java | 4 +- 19 files changed, 270 insertions(+), 74 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnInfo.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index f948f5c689864..e116965897d3b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -66,7 +66,9 @@ import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.DoFnInfo; import com.google.cloud.dataflow.sdk.util.OutputReference; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.SerializableUtils; @@ -88,7 +90,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -886,7 +887,7 @@ private void translateMultiHelper( TranslationContext context) { context.addStep(transform, "ParallelDo"); translateInputs(transform.getInput(), transform.getSideInputs(), context); - translateFn(transform.getFn(), context); + translateFn(transform.getFn(), transform.getInput().getWindowingFn(), context); translateOutputs(transform.getOutput(), context); } }); @@ -906,7 +907,7 @@ private void translateSingleHelper( TranslationContext context) { context.addStep(transform, "ParallelDo"); translateInputs(transform.getInput(), transform.getSideInputs(), context); - translateFn(transform.getFn(), context); + translateFn(transform.getFn(), transform.getInput().getWindowingFn(), context); context.addOutput("out", transform.getOutput()); } }); @@ -965,12 +966,13 @@ private static void translateSideInputs( } private static void translateFn( - Serializable fn, + DoFn fn, + WindowingFn windowingFn, TranslationContext context) { context.addInput(PropertyNames.USER_FN, fn.getClass().getName()); context.addInput( PropertyNames.SERIALIZED_FN, - byteArrayToJsonString(serializeToByteArray(fn))); + byteArrayToJsonString(serializeToByteArray(new DoFnInfo(fn, windowingFn)))); if (fn instanceof DoFn.RequiresKeyedState) { context.addInput(PropertyNames.USES_KEYED_STATE, "true"); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java index 00558a306ead6..058fba3d2d4f4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java @@ -25,6 +25,7 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn; import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.DoFnInfo; import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.PTuple; import com.google.cloud.dataflow.sdk.util.PropertyNames; @@ -77,7 +78,7 @@ private AssignWindowsParDoFn( CounterSet.AddCounterMutator addCounterMutator) { super( options, - fn, + new DoFnInfo(fn, null), PTuple.empty(), Arrays.asList("output"), stepName, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java index 63c4089f616a2..79ee768263bc3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java @@ -25,6 +25,7 @@ import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.DoFnInfo; import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.PTuple; import com.google.cloud.dataflow.sdk.util.PropertyNames; @@ -118,7 +119,7 @@ private CombineValuesFn( CounterSet.AddCounterMutator addCounterMutator) { super( options, - doFn, + new DoFnInfo(doFn, null), PTuple.empty(), Arrays.asList("output"), stepName, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java index adf0435e6e981..1903e44a66e5a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java @@ -28,6 +28,7 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.DoFnInfo; import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.PTuple; import com.google.cloud.dataflow.sdk.util.PropertyNames; @@ -109,7 +110,7 @@ private GroupAlsoByWindowsParDoFn( CounterSet.AddCounterMutator addCounterMutator) { super( options, - fn, + new DoFnInfo(fn, null), PTuple.empty(), Arrays.asList("output"), stepName, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java index 79d729b974d87..31bef7542cdbc 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java @@ -21,8 +21,8 @@ import com.google.api.services.dataflow.model.MultiOutputInfo; import com.google.api.services.dataflow.model.SideInputInfo; import com.google.cloud.dataflow.sdk.options.PipelineOptions; -import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.DoFnInfo; import com.google.cloud.dataflow.sdk.util.DoFnRunner; import com.google.cloud.dataflow.sdk.util.DoFnRunner.OutputManager; import com.google.cloud.dataflow.sdk.util.ExecutionContext; @@ -65,10 +65,10 @@ public static NormalParDoFn create( SerializableUtils.deserializeFromByteArray( getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), "serialized user fn"); - if (!(deserializedFn instanceof DoFn)) { - throw new Exception("unexpected kind of DoFn: " + deserializedFn.getClass().getName()); + if (!(deserializedFn instanceof DoFnInfo)) { + throw new Exception("unexpected kind of DoFnInfo: " + deserializedFn.getClass().getName()); } - DoFn fn = (DoFn) deserializedFn; + DoFnInfo fnInfo = (DoFnInfo) deserializedFn; PTuple sideInputValues = PTuple.empty(); if (sideInputInfos != null) { @@ -96,12 +96,12 @@ public static NormalParDoFn create( "unexpected number of outputTags for DoFn"); } - return new NormalParDoFn(options, fn, sideInputValues, outputTags, + return new NormalParDoFn(options, fnInfo, sideInputValues, outputTags, stepName, executionContext, addCounterMutator); } public final PipelineOptions options; - public final DoFn fn; + public final DoFnInfo fnInfo; public final PTuple sideInputValues; public final TupleTag mainOutputTag; public final List> sideOutputTags; @@ -113,14 +113,14 @@ public static NormalParDoFn create( DoFnRunner fnRunner; public NormalParDoFn(PipelineOptions options, - DoFn fn, + DoFnInfo fnInfo, PTuple sideInputValues, List outputTags, String stepName, ExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator) { this.options = options; - this.fn = (DoFn) fn; + this.fnInfo = fnInfo; this.sideInputValues = sideInputValues; if (outputTags.size() < 1) { throw new AssertionError("expected at least one output"); @@ -151,7 +151,7 @@ public void startBundle(final Receiver... receivers) throws Exception { fnRunner = DoFnRunner.create( options, - fn, + fnInfo.getDoFn(), sideInputValues, new OutputManager() { final Map, OutputReceiver> undeclaredOutputs = @@ -197,7 +197,8 @@ public void output(Receiver receiver, WindowedValue output) { mainOutputTag, sideOutputTags, stepContext, - addCounterMutator); + addCounterMutator, + fnInfo.getWindowingFn()); fnRunner.startBundle(); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java index d6057d5b29c65..a62982ceff2db 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java @@ -81,13 +81,13 @@ public abstract class Context { * element will have the same timestamp and be in the same windows * as the input element passed to {@link DoFn#processElement}). * - *

Is is illegal to invoke this from {@link #startBundle} or - * {@link #finishBundle} unless the input {@code PCollection} is - * windowed by the - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. - * If this is the case, the output element will have a timestamp - * of negative infinity and be in the - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. + *

If invoked from {@link #startBundle} or {@link #finishValue}, + * this will attempt to use the + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * of the input {@code PCollection} to determine what windows the element + * should be in, throwing an exception if the {@code WindowingFn} attempts + * to access any information about the input element. The output element + * will have a timestamp of negative infinity. */ public abstract void output(O output); @@ -100,13 +100,13 @@ public abstract class Context { * {@link DoFn#getAllowedTimestampSkew}. The output element will * be in the same windows as the input element. * - *

Is is illegal to invoke this from {@link #startBundle} or - * {@link #finishBundle} unless the input {@code PCollection} is - * windowed by the - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. - * If this is the case, the output element's timestamp will be - * the given timestamp and its window will be the - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. + *

If invoked from {@link #startBundle} or {@link #finishValue}, + * this will attempt to use the + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * of the input {@code PCollection} to determine what windows the element + * should be in, throwing an exception if the {@code WindowingFn} attempts + * to access any information about the input element except for the + * timestamp. */ public abstract void outputWithTimestamp(O output, Instant timestamp); @@ -122,6 +122,14 @@ public abstract class Context { *

The output element will have the same timestamp and be in the same * windows as the input element passed to {@link DoFn#processElement}). * + *

If invoked from {@link #startBundle} or {@link #finishValue}, + * this will attempt to use the + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * of the input {@code PCollection} to determine what windows the element + * should be in, throwing an exception if the {@code WindowingFn} attempts + * to access any information about the input element. The output element + * will have a timestamp of negative infinity. + * * @throws IllegalArgumentException if the number of outputs exceeds * the limit of 1,000 outputs per DoFn * @see ParDo#withOutputTags @@ -137,13 +145,13 @@ public abstract class Context { * {@link DoFn#getAllowedTimestampSkew}. The output element will * be in the same windows as the input element. * - *

Is is illegal to invoke this from {@link #startBundle} or - * {@link #finishBundle} unless the input {@code PCollection} is - * windowed by the - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. - * If this is the case, the output element's timestamp will be - * the given timestamp and its window will be the - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow}. + *

If invoked from {@link #startBundle} or {@link #finishValue}, + * this will attempt to use the + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * of the input {@code PCollection} to determine what windows the element + * should be in, throwing an exception if the {@code WindowingFn} attempts + * to access any information about the input element except for the + * timestamp. * * @throws IllegalArgumentException if the number of outputs exceeds * the limit of 1,000 outputs per DoFn diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java index 70ae664a6fc20..55d0f8b18b964 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java @@ -18,6 +18,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.DoFnRunner; import com.google.cloud.dataflow.sdk.util.PTuple; @@ -353,6 +354,7 @@ void initializeState() { mainOutputTag, sideOutputTags, (new BatchModeExecutionContext()).createStepContext("stepName"), - counterSet.getAddCounterMutator()); + counterSet.getAddCounterMutator(), + new GlobalWindow()); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java index 4ddca9ef7dd12..0af3c3baf8971 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java @@ -1039,7 +1039,8 @@ private static DoFnRunner evaluateHelper( mainOutputTag, sideOutputTags, executionContext.getStepContext(name), - context.getAddCounterMutator()); + context.getAddCounterMutator(), + input.getWindowingFn()); fnRunner.startBundle(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java index 71e8b0be3e36d..702476362614b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java @@ -273,7 +273,8 @@ private static void evaluateHelper( outputTag, new ArrayList>(), executionContext.getStepContext(context.getStepName(transform)), - context.getAddCounterMutator()); + context.getAddCounterMutator(), + transform.fn); addWindowsRunner.startBundle(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java index 7eca7d5f510bf..115b3f15f5540 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java @@ -22,7 +22,8 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn.AssignContext; import com.google.cloud.dataflow.sdk.util.DoFnRunner.OutputManager; import com.google.cloud.dataflow.sdk.util.ExecutionContext.StepContext; import com.google.cloud.dataflow.sdk.util.common.CounterSet; @@ -31,7 +32,6 @@ import org.joda.time.Instant; -import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.List; @@ -56,6 +56,7 @@ class DoFnContext extends DoFn.Context { final TupleTag mainOutputTag; final StepContext stepContext; final CounterSet.AddCounterMutator addCounterMutator; + final WindowingFn windowingFn; public DoFnContext(PipelineOptions options, DoFn fn, @@ -64,7 +65,8 @@ public DoFnContext(PipelineOptions options, TupleTag mainOutputTag, List> sideOutputTags, StepContext stepContext, - CounterSet.AddCounterMutator addCounterMutator) { + CounterSet.AddCounterMutator addCounterMutator, + WindowingFn windowingFn) { fn.super(); this.options = options; this.fn = fn; @@ -78,6 +80,7 @@ public DoFnContext(PipelineOptions options, } this.stepContext = stepContext; this.addCounterMutator = addCounterMutator; + this.windowingFn = windowingFn; } public R getReceiver(TupleTag tag) { @@ -109,11 +112,51 @@ public T sideInput(PCollectionView view) { return view.fromIterableInternal((Iterable>) sideInputs.get(tag)); } + WindowedValue makeWindowedValue( + T output, Instant timestamp, Collection windows) { + final Instant inputTimestamp = timestamp; + + if (timestamp == null) { + timestamp = new Instant(Long.MIN_VALUE); + } + + if (windows == null) { + try { + windows = windowingFn.assignWindows(windowingFn.new AssignContext() { + @Override + public Object element() { + throw new UnsupportedOperationException( + "WindowingFn attemped to access input element when none was available"); + } + + @Override + public Instant timestamp() { + if (inputTimestamp == null) { + throw new UnsupportedOperationException( + "WindowingFn attemped to access input timestamp when none was available"); + } + return inputTimestamp; + } + + @Override + public Collection windows() { + throw new UnsupportedOperationException( + "WindowingFn attemped to access input windows when none were available"); + } + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + return WindowedValue.of(output, timestamp, windows); + } + void outputWindowedValue( O output, Instant timestamp, Collection windows) { - WindowedValue windowedElem = WindowedValue.of(output, timestamp, windows); + WindowedValue windowedElem = makeWindowedValue(output, timestamp, windows); outputManager.output(outputMap.get(mainOutputTag), windowedElem); if (stepContext != null) { stepContext.noteOutput(windowedElem); @@ -143,7 +186,7 @@ protected void sideOutputWindowedValue(TupleTag tag, outputMap.put(tag, receiver); } - WindowedValue windowedElem = WindowedValue.of(output, timestamp, windows); + WindowedValue windowedElem = makeWindowedValue(output, timestamp, windows); outputManager.output(receiver, windowedElem); if (stepContext != null) { stepContext.noteSideOutput(tag, windowedElem); @@ -153,31 +196,24 @@ protected void sideOutputWindowedValue(TupleTag tag, // Following implementations of output, outputWithTimestamp, and sideOutput // are only accessible in DoFn.startBundle and DoFn.finishBundle, and will be shadowed by // ProcessContext's versions in DoFn.processElement. - // TODO: it seems wrong to use Long.MIN_VALUE, since it will violate all our rules about - // DoFns preserving watermarks. @Override public void output(O output) { - outputWindowedValue(output, - new Instant(Long.MIN_VALUE), - Arrays.asList(GlobalWindow.Window.INSTANCE)); + outputWindowedValue(output, null, null); } @Override public void outputWithTimestamp(O output, Instant timestamp) { - outputWindowedValue(output, timestamp, Arrays.asList(GlobalWindow.Window.INSTANCE)); + outputWindowedValue(output, timestamp, null); } @Override public void sideOutput(TupleTag tag, T output) { - sideOutputWindowedValue(tag, - output, - new Instant(Long.MIN_VALUE), - Arrays.asList(GlobalWindow.Window.INSTANCE)); + sideOutputWindowedValue(tag, output, null, null); } @Override public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - sideOutputWindowedValue(tag, output, timestamp, Arrays.asList(GlobalWindow.Window.INSTANCE)); + sideOutputWindowedValue(tag, output, timestamp, null); } private String generateInternalAggregatorName(String userName) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnInfo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnInfo.java new file mode 100644 index 0000000000000..1af14d7a130ea --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnInfo.java @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; + +import java.io.Serializable; + +/** + * Wrapper class holding the necessary information to serialize a DoFn. + * + * @param the type of the (main) input elements of the DoFn + * @param the type of the (main) output elements of the DoFn + */ +public class DoFnInfo implements Serializable { + private static final long serialVersionUID = 0; + private DoFn doFn; + private WindowingFn windowingFn; + + public DoFnInfo(DoFn doFn, WindowingFn windowingFn) { + this.doFn = doFn; + this.windowingFn = windowingFn; + } + + public DoFn getDoFn() { + return doFn; + } + + public WindowingFn getWindowingFn() { + return windowingFn; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java index d4d788d89af7f..09320962a3f2d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java @@ -18,6 +18,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; import com.google.cloud.dataflow.sdk.util.ExecutionContext.StepContext; import com.google.cloud.dataflow.sdk.util.common.CounterSet; import com.google.cloud.dataflow.sdk.values.TupleTag; @@ -58,11 +59,12 @@ private DoFnRunner(PipelineOptions options, TupleTag mainOutputTag, List> sideOutputTags, StepContext stepContext, - CounterSet.AddCounterMutator addCounterMutator) { + CounterSet.AddCounterMutator addCounterMutator, + WindowingFn windowingFn) { this.fn = fn; this.context = new DoFnContext<>(options, fn, sideInputs, outputManager, mainOutputTag, sideOutputTags, stepContext, - addCounterMutator); + addCounterMutator, windowingFn); } public static DoFnRunner create( @@ -73,10 +75,11 @@ public static DoFnRunner create( TupleTag mainOutputTag, List> sideOutputTags, StepContext stepContext, - CounterSet.AddCounterMutator addCounterMutator) { + CounterSet.AddCounterMutator addCounterMutator, + WindowingFn windowingFn) { return new DoFnRunner<>( options, fn, sideInputs, outputManager, - mainOutputTag, sideOutputTags, stepContext, addCounterMutator); + mainOutputTag, sideOutputTags, stepContext, addCounterMutator, windowingFn); } @SuppressWarnings({"rawtypes", "unchecked"}) @@ -87,7 +90,8 @@ public static DoFnRunner createWithListOutputs( TupleTag mainOutputTag, List> sideOutputTags, StepContext stepContext, - CounterSet.AddCounterMutator addCounterMutator) { + CounterSet.AddCounterMutator addCounterMutator, + WindowingFn windowingFn) { return create( options, fn, sideInputs, new OutputManager() { @@ -100,7 +104,7 @@ public void output(List list, WindowedValue output) { list.add(output); } }, - mainOutputTag, sideOutputTags, stepContext, addCounterMutator); + mainOutputTag, sideOutputTags, stepContext, addCounterMutator, windowingFn); } /** Calls {@link DoFn#startBundle}. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java index 67daaf0de246a..29f6078baa277 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java @@ -87,6 +87,9 @@ public static WindowedValue valueInEmptyWindows(V value) { private WindowedValue(V value, Instant timestamp, Collection windows) { + checkNotNull(timestamp); + checkNotNull(windows); + this.value = value; this.timestamp = timestamp; this.windows = windows; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java index 445755130db5d..cfd89466d845d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java @@ -42,9 +42,11 @@ import com.google.cloud.dataflow.sdk.runners.worker.SinkFactoryTest.TestSink; import com.google.cloud.dataflow.sdk.runners.worker.SinkFactoryTest.TestSinkFactory; import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.DoFnInfo; import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.SerializableUtils; @@ -291,7 +293,8 @@ static ParallelInstruction createParDoInstruction( TestDoFn fn = new TestDoFn(); String serializedFn = - StringUtils.byteArrayToJsonString(SerializableUtils.serializeToByteArray(fn)); + StringUtils.byteArrayToJsonString( + SerializableUtils.serializeToByteArray(new DoFnInfo(fn, new GlobalWindow()))); CloudObject cloudUserFn = CloudObject.forClassName("DoFn"); addString(cloudUserFn, PropertyNames.SERIALIZED_FN, serializedFn); @@ -339,7 +342,8 @@ public void testCreateParDoOperation() throws Exception { assertThat(parDoOperation.fn, new IsInstanceOf(NormalParDoFn.class)); NormalParDoFn normalParDoFn = (NormalParDoFn) parDoOperation.fn; - assertThat(normalParDoFn.fn, new IsInstanceOf(TestDoFn.class)); + assertThat(normalParDoFn.fnInfo.getDoFn(), + new IsInstanceOf(TestDoFn.class)); assertSame( parDoOperation, diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java index 2c6811c27b5ea..f4a02679ce838 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java @@ -29,7 +29,9 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.DoFnInfo; import com.google.cloud.dataflow.sdk.util.PTuple; import com.google.cloud.dataflow.sdk.util.UserCodeException; import com.google.cloud.dataflow.sdk.util.WindowedValue; @@ -139,6 +141,7 @@ public void testNormalParDoFn() throws Exception { List sideOutputTags = Arrays.asList("tag1", "tag2", "tag3"); TestDoFn fn = new TestDoFn(sideOutputTags); + DoFnInfo fnInfo = new DoFnInfo(fn, new GlobalWindow()); TestReceiver receiver = new TestReceiver(); TestReceiver receiver1 = new TestReceiver(); TestReceiver receiver2 = new TestReceiver(); @@ -151,7 +154,7 @@ public void testNormalParDoFn() throws Exception { outputTags.addAll(sideOutputTags); NormalParDoFn normalParDoFn = new NormalParDoFn(PipelineOptionsFactory.create(), - fn, sideInputValues, outputTags, "doFn", + fnInfo, sideInputValues, outputTags, "doFn", new BatchModeExecutionContext(), (new CounterSet()).getAddCounterMutator()); @@ -203,13 +206,14 @@ public void testNormalParDoFn() throws Exception { @Test public void testUnexpectedNumberOfReceivers() throws Exception { TestDoFn fn = new TestDoFn(Collections.emptyList()); + DoFnInfo fnInfo = new DoFnInfo(fn, new GlobalWindow()); TestReceiver receiver = new TestReceiver(); PTuple sideInputValues = PTuple.empty(); List outputTags = Arrays.asList("output"); NormalParDoFn normalParDoFn = new NormalParDoFn(PipelineOptionsFactory.create(), - fn, sideInputValues, outputTags, "doFn", + fnInfo, sideInputValues, outputTags, "doFn", new BatchModeExecutionContext(), (new CounterSet()).getAddCounterMutator()); @@ -241,13 +245,14 @@ private List stackTraceFrameStrings(Throwable t) { @Test public void testErrorPropagation() throws Exception { TestErrorDoFn fn = new TestErrorDoFn(); + DoFnInfo fnInfo = new DoFnInfo(fn, new GlobalWindow()); TestReceiver receiver = new TestReceiver(); PTuple sideInputValues = PTuple.empty(); List outputTags = Arrays.asList("output"); NormalParDoFn normalParDoFn = new NormalParDoFn(PipelineOptionsFactory.create(), - fn, sideInputValues, outputTags, "doFn", + fnInfo, sideInputValues, outputTags, "doFn", new BatchModeExecutionContext(), (new CounterSet()).getAddCounterMutator()); @@ -308,9 +313,10 @@ public void testErrorPropagation() throws Exception { @Test public void testUndeclaredSideOutputs() throws Exception { TestDoFn fn = new TestDoFn(Arrays.asList("declared", "undecl1", "undecl2", "undecl3")); + DoFnInfo fnInfo = new DoFnInfo(fn, new GlobalWindow()); CounterSet counters = new CounterSet(); NormalParDoFn normalParDoFn = - new NormalParDoFn(PipelineOptionsFactory.create(), fn, PTuple.empty(), + new NormalParDoFn(PipelineOptionsFactory.create(), fnInfo, PTuple.empty(), Arrays.asList("output", "declared"), "doFn", new BatchModeExecutionContext(), counters.getAddCounterMutator()); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java index 6ceb61f267708..be31ece009e4b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java @@ -21,8 +21,10 @@ import com.google.api.services.dataflow.model.MultiOutputInfo; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.DoFnInfo; import com.google.cloud.dataflow.sdk.util.SerializableUtils; import com.google.cloud.dataflow.sdk.util.StringUtils; import com.google.cloud.dataflow.sdk.util.common.CounterSet; @@ -69,7 +71,7 @@ public void testCreateNormalParDoFn() throws Exception { String serializedFn = StringUtils.byteArrayToJsonString( - SerializableUtils.serializeToByteArray(fn)); + SerializableUtils.serializeToByteArray(new DoFnInfo(fn, new GlobalWindow()))); CloudObject cloudUserFn = CloudObject.forClassName("DoFn"); addString(cloudUserFn, "serialized_fn", serializedFn); @@ -92,8 +94,9 @@ public void testCreateNormalParDoFn() throws Exception { Assert.assertThat(parDoFn, new IsInstanceOf(NormalParDoFn.class)); NormalParDoFn normalParDoFn = (NormalParDoFn) parDoFn; - DoFn actualDoFn = normalParDoFn.fn; + DoFn actualDoFn = normalParDoFn.fnInfo.getDoFn(); Assert.assertThat(actualDoFn, new IsInstanceOf(TestDoFn.class)); + Assert.assertThat(normalParDoFn.fnInfo.getWindowingFn(), new IsInstanceOf(GlobalWindow.class)); TestDoFn actualTestDoFn = (TestDoFn) actualDoFn; Assert.assertEquals(stringState, actualTestDoFn.stringState); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java index c5e699216707b..39494e9647c2a 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java @@ -40,12 +40,15 @@ import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; import com.google.cloud.dataflow.sdk.values.CodedTupleTag; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionTuple; import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.TimestampedValue; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.cloud.dataflow.sdk.values.TupleTagList; @@ -1136,4 +1139,72 @@ public void testParDoShiftTimestampInvalid() { // expected } } + + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testWindowingInStartAndFinishBundle() { + Pipeline p = TestPipeline.create(); + + PCollection output = p + .apply(Create.timestamped(TimestampedValue.of("elem", new Instant(1)))) + .apply(Window.into(FixedWindows.of(Duration.millis(1)))) + .apply(ParDo.of(new DoFn() { + @Override + public void startBundle(Context c) { + c.outputWithTimestamp("start", new Instant(2)); + System.out.println("Start: 2"); + } + + @Override + public void processElement(ProcessContext c) { + c.output(c.element()); + System.out.println("Process: " + c.element() + ":" + c.timestamp().getMillis()); + } + + @Override + public void finishBundle(Context c) { + c.outputWithTimestamp("finish", new Instant(3)); + System.out.println("Finish: 3"); + } + })) + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element() + ":" + c.timestamp().getMillis() + + ":" + c.windows().iterator().next().maxTimestamp().getMillis()); + } + })); + + DataflowAssert.that(output).containsInAnyOrder("elem:1:1", "start:2:2", "finish:3:3"); + + p.run(); + } + + @Test + public void testWindowingInStartBundleException() { + Pipeline p = TestPipeline.create(); + + PCollection output = p + .apply(Create.timestamped(TimestampedValue.of("elem", new Instant(1)))) + .apply(Window.into(FixedWindows.of(Duration.millis(1)))) + .apply(ParDo.of(new DoFn() { + @Override + public void startBundle(Context c) { + c.output("start"); + } + + @Override + public void processElement(ProcessContext c) { + c.output(c.element()); + } + })); + + try { + p.run(); + fail("should have failed"); + } catch (Exception e) { + assertThat(e.toString(), containsString( + "WindowingFn attemped to access input timestamp when none was available")); + } + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java index 37c0f8edbff4a..20f24c2ebcd30 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java @@ -23,6 +23,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; @@ -221,7 +222,8 @@ KV>, List> makeRunner( outputTag, new ArrayList>(), execContext.createStepContext("merge"), - counters.getAddCounterMutator()); + counters.getAddCounterMutator(), + new GlobalWindow()); return runner; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java index f5895037e4fcb..c199b9190cfa6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java @@ -25,6 +25,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; @@ -272,7 +273,8 @@ KV>, List> makeRunner( outputTag, new ArrayList>(), execContext.createStepContext("merge"), - counters.getAddCounterMutator()); + counters.getAddCounterMutator(), + new GlobalWindow()); return runner; } From 6f521f91290f016b46ffbecd60aeaffae54ee8e1 Mon Sep 17 00:00:00 2001 From: millsd Date: Fri, 16 Jan 2015 14:15:02 -0800 Subject: [PATCH 0085/1541] Allow MapTaskExecutor to be run multiple times [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84162247 --- .../runners/worker/AssignWindowsParDoFn.java | 18 ++++--- .../sdk/runners/worker/CombineValuesFn.java | 52 +++++++++++-------- .../worker/GroupAlsoByWindowsParDoFn.java | 24 +++++---- .../sdk/runners/worker/NormalParDoFn.java | 44 ++++++++++------ .../util/common/worker/FlattenOperation.java | 5 ++ .../sdk/util/common/worker/Operation.java | 11 +++- .../util/common/worker/ParDoOperation.java | 5 ++ .../sdk/util/common/worker/ReadOperation.java | 8 ++- .../sdk/util/common/worker/Reader.java | 7 +++ .../dataflow/sdk/util/common/worker/Sink.java | 7 +++ .../util/common/worker/WriteOperation.java | 5 ++ .../worker/MapTaskExecutorFactoryTest.java | 4 +- .../sdk/runners/worker/NormalParDoFnTest.java | 27 +++++++--- .../runners/worker/ParDoFnFactoryTest.java | 6 ++- 14 files changed, 155 insertions(+), 68 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java index 058fba3d2d4f4..2a22793e81c4a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java @@ -21,7 +21,6 @@ import com.google.api.services.dataflow.model.MultiOutputInfo; import com.google.api.services.dataflow.model.SideInputInfo; import com.google.cloud.dataflow.sdk.options.PipelineOptions; -import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn; import com.google.cloud.dataflow.sdk.util.CloudObject; @@ -55,7 +54,7 @@ public static AssignWindowsParDoFn create( CounterSet.AddCounterMutator addCounterMutator, StateSampler sampler /* unused */) throws Exception { - Object windowingFn = + final Object windowingFn = SerializableUtils.deserializeFromByteArray( getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), "serialized window fn"); @@ -64,21 +63,28 @@ public static AssignWindowsParDoFn create( "unexpected kind of WindowingFn: " + windowingFn.getClass().getName()); } - DoFn assignWindowsDoFn = new AssignWindowsDoFn((WindowingFn) windowingFn); + final AssignWindowsDoFn assignFn = new AssignWindowsDoFn((WindowingFn) windowingFn); + + DoFnInfoFactory fnFactory = new DoFnInfoFactory() { + @Override + public DoFnInfo createDoFnInfo() { + return new DoFnInfo(assignFn, null); + } + }; return new AssignWindowsParDoFn( - options, assignWindowsDoFn, stepName, executionContext, addCounterMutator); + options, fnFactory, stepName, executionContext, addCounterMutator); } private AssignWindowsParDoFn( PipelineOptions options, - DoFn fn, + DoFnInfoFactory fnFactory, String stepName, ExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator) { super( options, - new DoFnInfo(fn, null), + fnFactory, PTuple.empty(), Arrays.asList("output"), stepName, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java index 79ee768263bc3..31e87649a5d27 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java @@ -78,11 +78,11 @@ public static CombineValuesFn create( "serialized user fn"); Preconditions.checkArgument( deserializedFn instanceof Combine.KeyedCombineFn); - Combine.KeyedCombineFn combineFn = (Combine.KeyedCombineFn) deserializedFn; + final Combine.KeyedCombineFn combineFn = (Combine.KeyedCombineFn) deserializedFn; // Get the combine phase, default to ALL. (The implementation // doesn't have to split the combiner). - String phase = getString(cloudUserFn, PropertyNames.PHASE, CombinePhase.ALL); + final String phase = getString(cloudUserFn, PropertyNames.PHASE, CombinePhase.ALL); Preconditions.checkArgument( sideInputInfos == null || sideInputInfos.size() == 0, @@ -90,36 +90,42 @@ public static CombineValuesFn create( Preconditions.checkArgument( numOutputs == 1, "expected exactly one output for CombineValuesFn"); - DoFn doFn = null; - switch (phase) { - case CombinePhase.ALL: - doFn = new CombineValuesDoFn(combineFn); - break; - case CombinePhase.ADD: - doFn = new AddInputsDoFn(combineFn); - break; - case CombinePhase.MERGE: - doFn = new MergeAccumulatorsDoFn(combineFn); - break; - case CombinePhase.EXTRACT: - doFn = new ExtractOutputDoFn(combineFn); - break; - default: - throw new IllegalArgumentException( - "phase must be one of 'all', 'add', 'merge', 'extract'"); - } - return new CombineValuesFn(options, doFn, stepName, executionContext, addCounterMutator); + DoFnInfoFactory fnFactory = new DoFnInfoFactory() { + @Override + public DoFnInfo createDoFnInfo() { + DoFn doFn = null; + switch (phase) { + case CombinePhase.ALL: + doFn = new CombineValuesDoFn(combineFn); + break; + case CombinePhase.ADD: + doFn = new AddInputsDoFn(combineFn); + break; + case CombinePhase.MERGE: + doFn = new MergeAccumulatorsDoFn(combineFn); + break; + case CombinePhase.EXTRACT: + doFn = new ExtractOutputDoFn(combineFn); + break; + default: + throw new IllegalArgumentException( + "phase must be one of 'all', 'add', 'merge', 'extract'"); + } + return new DoFnInfo(doFn, null); + } + }; + return new CombineValuesFn(options, fnFactory, stepName, executionContext, addCounterMutator); } private CombineValuesFn( PipelineOptions options, - DoFn doFn, + DoFnInfoFactory fnFactory, String stepName, ExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator) { super( options, - new DoFnInfo(doFn, null), + fnFactory, PTuple.empty(), Arrays.asList("output"), stepName, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java index 1903e44a66e5a..a8ba7e840b4f5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java @@ -25,7 +25,6 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; -import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.DoFnInfo; @@ -61,7 +60,7 @@ public static GroupAlsoByWindowsParDoFn create( CounterSet.AddCounterMutator addCounterMutator, StateSampler sampler /* unused */) throws Exception { - Object windowingFn = + final Object windowingFn = SerializableUtils.deserializeFromByteArray( getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), "serialized window fn"); @@ -88,29 +87,34 @@ public static GroupAlsoByWindowsParDoFn create( "Expected WindowedValueCoder for inputCoder, got: " + inputCoder.getClass().getName()); } - Coder elemCoder = ((WindowedValueCoder) inputCoder).getValueCoder(); + final Coder elemCoder = ((WindowedValueCoder) inputCoder).getValueCoder(); if (!(elemCoder instanceof KvCoder)) { throw new Exception( "Expected KvCoder for inputCoder, got: " + elemCoder.getClass().getName()); } - DoFn windowingDoFn = StreamingGroupAlsoByWindowsDoFn.create( - (WindowingFn) windowingFn, - ((KvCoder) elemCoder).getValueCoder()); - + DoFnInfoFactory fnFactory = new DoFnInfoFactory() { + @Override + public DoFnInfo createDoFnInfo() { + return new DoFnInfo(StreamingGroupAlsoByWindowsDoFn.create( + (WindowingFn) windowingFn, + ((KvCoder) elemCoder).getValueCoder()), + null); + } + }; return new GroupAlsoByWindowsParDoFn( - options, windowingDoFn, stepName, executionContext, addCounterMutator); + options, fnFactory, stepName, executionContext, addCounterMutator); } private GroupAlsoByWindowsParDoFn( PipelineOptions options, - DoFn fn, + DoFnInfoFactory fnFactory, String stepName, ExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator) { super( options, - new DoFnInfo(fn, null), + fnFactory, PTuple.empty(), Arrays.asList("output"), stepName, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java index 31bef7542cdbc..ea9ea4a970d87 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java @@ -50,9 +50,17 @@ */ @SuppressWarnings({"rawtypes", "unchecked"}) public class NormalParDoFn extends ParDoFn { + + /** + * Factory for creating DoFn instances. + */ + protected static interface DoFnInfoFactory { + public DoFnInfo createDoFnInfo() throws Exception; + } + public static NormalParDoFn create( PipelineOptions options, - CloudObject cloudUserFn, + final CloudObject cloudUserFn, String stepName, @Nullable List sideInputInfos, @Nullable List multiOutputInfos, @@ -61,14 +69,20 @@ public static NormalParDoFn create( CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler /* ignored */) throws Exception { - Object deserializedFn = - SerializableUtils.deserializeFromByteArray( - getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), - "serialized user fn"); - if (!(deserializedFn instanceof DoFnInfo)) { - throw new Exception("unexpected kind of DoFnInfo: " + deserializedFn.getClass().getName()); - } - DoFnInfo fnInfo = (DoFnInfo) deserializedFn; + DoFnInfoFactory fnFactory = new DoFnInfoFactory() { + @Override + public DoFnInfo createDoFnInfo() throws Exception { + Object deserializedFn = + SerializableUtils.deserializeFromByteArray( + getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), + "serialized user fn"); + if (!(deserializedFn instanceof DoFnInfo)) { + throw new Exception( + "unexpected kind of DoFnInfo: " + deserializedFn.getClass().getName()); + } + return (DoFnInfo) deserializedFn; + } + }; PTuple sideInputValues = PTuple.empty(); if (sideInputInfos != null) { @@ -96,12 +110,12 @@ public static NormalParDoFn create( "unexpected number of outputTags for DoFn"); } - return new NormalParDoFn(options, fnInfo, sideInputValues, outputTags, + return new NormalParDoFn(options, fnFactory, sideInputValues, outputTags, stepName, executionContext, addCounterMutator); } public final PipelineOptions options; - public final DoFnInfo fnInfo; + public final DoFnInfoFactory fnFactory; public final PTuple sideInputValues; public final TupleTag mainOutputTag; public final List> sideOutputTags; @@ -113,14 +127,14 @@ public static NormalParDoFn create( DoFnRunner fnRunner; public NormalParDoFn(PipelineOptions options, - DoFnInfo fnInfo, + DoFnInfoFactory fnFactory, PTuple sideInputValues, List outputTags, String stepName, ExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator) { this.options = options; - this.fnInfo = fnInfo; + this.fnFactory = fnFactory; this.sideInputValues = sideInputValues; if (outputTags.size() < 1) { throw new AssertionError("expected at least one output"); @@ -151,7 +165,7 @@ public void startBundle(final Receiver... receivers) throws Exception { fnRunner = DoFnRunner.create( options, - fnInfo.getDoFn(), + fnFactory.createDoFnInfo().getDoFn(), sideInputValues, new OutputManager() { final Map, OutputReceiver> undeclaredOutputs = @@ -198,7 +212,7 @@ public void output(Receiver receiver, WindowedValue output) { sideOutputTags, stepContext, addCounterMutator, - fnInfo.getWindowingFn()); + fnFactory.createDoFnInfo().getWindowingFn()); fnRunner.startBundle(); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperation.java index 6325d1ac5cdb8..8076216211425 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/FlattenOperation.java @@ -51,4 +51,9 @@ public void process(Object elem) throws Exception { } } } + + @Override + public boolean supportsRestart() { + return true; + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Operation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Operation.java index bedc081cec99d..207d034d4f3d6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Operation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Operation.java @@ -84,7 +84,9 @@ public Operation(String operationName, * exception otherwise. */ void checkUnstarted() { - if (initializationState != InitializationState.UNSTARTED) { + if (!(initializationState == InitializationState.UNSTARTED + || (initializationState == InitializationState.FINISHED + && supportsRestart()))) { throw new AssertionError( "expecting this instruction to not yet be started"); } @@ -129,4 +131,11 @@ public void finish() throws Exception { checkStarted(); initializationState = InitializationState.FINISHED; } + + /** + * Returns true if this Operation can be started again after it is finished. + */ + public boolean supportsRestart() { + return false; + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperation.java index 7a620983476f0..684d11a9cb23c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ParDoOperation.java @@ -62,4 +62,9 @@ public void finish() throws Exception { super.finish(); } } + + @Override + public boolean supportsRestart() { + return true; + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java index e97cf2acb3daf..b727961692ba6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java @@ -86,6 +86,7 @@ public ReadOperation(String operationName, Reader reader, OutputReceiver[] re this.byteCount = addCounterMutator.addCounter( Counter.longs(bytesCounterName(counterPrefix, operationName), SUM)); readState = stateSampler.stateForName(operationName + "-read"); + reader.addObserver(new ReaderObserver()); } /** Invoked by tests. */ @@ -120,6 +121,11 @@ public void start() throws Exception { } } + @Override + public boolean supportsRestart() { + return reader.supportsRestart(); + } + protected void runReadLoop() throws Exception { Receiver receiver = receivers[0]; if (receiver == null) { @@ -127,8 +133,6 @@ protected void runReadLoop() throws Exception { return; } - reader.addObserver(new ReaderObserver()); - try (StateSampler.ScopedState process = stateSampler.scopedState(processState)) { assert process != null; synchronized (sourceIteratorLock) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java index 2b805c6f1d1fd..7c9b783511c01 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java @@ -153,4 +153,11 @@ protected void notifyElementRead(long byteSize) { setChanged(); notifyObservers(byteSize); } + + /** + * Returns whether this Reader can be restarted. + */ + public boolean supportsRestart() { + return false; + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Sink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Sink.java index 829fd1a391535..3a23982f97c72 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Sink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Sink.java @@ -44,4 +44,11 @@ public interface SinkWriter extends AutoCloseable { @Override public void close() throws IOException; } + + /** + * Returns whether this Sink can be restarted. + */ + public boolean supportsRestart() { + return false; + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperation.java index 3965e0d0a19c5..917d285fab9e0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WriteOperation.java @@ -102,6 +102,11 @@ public void finish() throws Exception { } } + @Override + public boolean supportsRestart() { + return sink.supportsRestart(); + } + public Counter getByteCount() { return byteCount; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java index cfd89466d845d..6667ef0c78c06 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java @@ -342,8 +342,8 @@ public void testCreateParDoOperation() throws Exception { assertThat(parDoOperation.fn, new IsInstanceOf(NormalParDoFn.class)); NormalParDoFn normalParDoFn = (NormalParDoFn) parDoOperation.fn; - assertThat(normalParDoFn.fnInfo.getDoFn(), - new IsInstanceOf(TestDoFn.class)); + assertThat(normalParDoFn.fnFactory.createDoFnInfo().getDoFn(), + new IsInstanceOf(TestDoFn.class)); assertSame( parDoOperation, diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java index f4a02679ce838..a6c4238f4c19b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java @@ -136,6 +136,18 @@ public void process(Object outputElem) { } } + static class TestDoFnInfoFactory implements NormalParDoFn.DoFnInfoFactory { + DoFnInfo fnInfo; + + TestDoFnInfoFactory(DoFnInfo fnInfo) { + this.fnInfo = fnInfo; + } + + public DoFnInfo createDoFnInfo() { + return fnInfo; + } + } + @Test public void testNormalParDoFn() throws Exception { List sideOutputTags = Arrays.asList("tag1", "tag2", "tag3"); @@ -154,7 +166,7 @@ public void testNormalParDoFn() throws Exception { outputTags.addAll(sideOutputTags); NormalParDoFn normalParDoFn = new NormalParDoFn(PipelineOptionsFactory.create(), - fnInfo, sideInputValues, outputTags, "doFn", + new TestDoFnInfoFactory(fnInfo), sideInputValues, outputTags, "doFn", new BatchModeExecutionContext(), (new CounterSet()).getAddCounterMutator()); @@ -213,7 +225,7 @@ public void testUnexpectedNumberOfReceivers() throws Exception { List outputTags = Arrays.asList("output"); NormalParDoFn normalParDoFn = new NormalParDoFn(PipelineOptionsFactory.create(), - fnInfo, sideInputValues, outputTags, "doFn", + new TestDoFnInfoFactory(fnInfo), sideInputValues, outputTags, "doFn", new BatchModeExecutionContext(), (new CounterSet()).getAddCounterMutator()); @@ -252,7 +264,7 @@ public void testErrorPropagation() throws Exception { List outputTags = Arrays.asList("output"); NormalParDoFn normalParDoFn = new NormalParDoFn(PipelineOptionsFactory.create(), - fnInfo, sideInputValues, outputTags, "doFn", + new TestDoFnInfoFactory(fnInfo), sideInputValues, outputTags, "doFn", new BatchModeExecutionContext(), (new CounterSet()).getAddCounterMutator()); @@ -316,10 +328,11 @@ public void testUndeclaredSideOutputs() throws Exception { DoFnInfo fnInfo = new DoFnInfo(fn, new GlobalWindow()); CounterSet counters = new CounterSet(); NormalParDoFn normalParDoFn = - new NormalParDoFn(PipelineOptionsFactory.create(), fnInfo, PTuple.empty(), - Arrays.asList("output", "declared"), "doFn", - new BatchModeExecutionContext(), - counters.getAddCounterMutator()); + new NormalParDoFn( + PipelineOptionsFactory.create(), new TestDoFnInfoFactory(fnInfo), PTuple.empty(), + Arrays.asList("output", "declared"), "doFn", + new BatchModeExecutionContext(), + counters.getAddCounterMutator()); normalParDoFn.startBundle(new TestReceiver(), new TestReceiver()); normalParDoFn.processElement(WindowedValue.valueInGlobalWindow(5)); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java index be31ece009e4b..8ae07b516de95 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java @@ -94,9 +94,11 @@ public void testCreateNormalParDoFn() throws Exception { Assert.assertThat(parDoFn, new IsInstanceOf(NormalParDoFn.class)); NormalParDoFn normalParDoFn = (NormalParDoFn) parDoFn; - DoFn actualDoFn = normalParDoFn.fnInfo.getDoFn(); + DoFn actualDoFn = normalParDoFn.fnFactory.createDoFnInfo().getDoFn(); Assert.assertThat(actualDoFn, new IsInstanceOf(TestDoFn.class)); - Assert.assertThat(normalParDoFn.fnInfo.getWindowingFn(), new IsInstanceOf(GlobalWindow.class)); + Assert.assertThat( + normalParDoFn.fnFactory.createDoFnInfo().getWindowingFn(), + new IsInstanceOf(GlobalWindow.class)); TestDoFn actualTestDoFn = (TestDoFn) actualDoFn; Assert.assertEquals(stringState, actualTestDoFn.stringState); From 6972706246a2eeb0aca47a43324f7e10b64d9d2d Mon Sep 17 00:00:00 2001 From: lcwik Date: Fri, 16 Jan 2015 15:13:33 -0800 Subject: [PATCH 0086/1541] The option machineType is a duplicate of workerMachineType. Remove the redundant machineType option. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84167494 --- .../sdk/options/DataflowPipelineWorkerPoolOptions.java | 7 ------- .../dataflow/sdk/runners/DataflowPipelineTranslator.java | 3 --- 2 files changed, 10 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java index a9be13a16df4a..25653495cc0ea 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java @@ -122,13 +122,6 @@ public String getApiServiceName() { String getWorkerMachineType(); void setWorkerMachineType(String value); - /** - * Machine type to create VMs as. - */ - @Description("Dataflow VM machine type.") - String getMachineType(); - void setMachineType(String value); - /** * The policy for tearing down the workers spun up by the service. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index e116965897d3b..3bcca5e8b3afb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -402,9 +402,6 @@ public Job translate(List packages) { workerPool.setDiskSourceImage(options.getDiskSourceImage()); } - if (options.getMachineType() != null) { - workerPool.setMachineType(options.getMachineType()); - } if (options.isStreaming()) { // Use separate data disk for streaming. Disk disk = new Disk(); From 800a77730ac4eb5fa3a7a7b54f3829ca2ffd16fb Mon Sep 17 00:00:00 2001 From: lcwik Date: Wed, 21 Jan 2015 08:46:23 -0800 Subject: [PATCH 0087/1541] Update PipelineOptionsFactory to improve readability. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84438386 --- .../sdk/options/PipelineOptionsFactory.java | 61 +++++++++++++------ 1 file changed, 42 insertions(+), 19 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java index 5de1fa06cbbfe..b20771a7c45f0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java @@ -31,6 +31,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; import com.google.common.collect.ListMultimap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -56,6 +57,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Queue; @@ -93,7 +95,7 @@ public class PipelineOptionsFactory { * @return An object which implements {@link PipelineOptions}. */ public static PipelineOptions create() { - return new Builder(getAppName(3)).as(PipelineOptions.class); + return new Builder().as(PipelineOptions.class); } /** @@ -107,7 +109,7 @@ public static PipelineOptions create() { * @return An object which implements {@code }. */ public static T as(Class klass) { - return new Builder(getAppName(3)).as(klass); + return new Builder().as(klass); } /** @@ -134,7 +136,7 @@ public static T as(Class klass) { * {@link Builder#withoutStrictParsing()}. */ public static Builder fromArgs(String[] args) { - return new Builder(getAppName(3)).fromArgs(args); + return new Builder().fromArgs(args); } /** @@ -144,7 +146,7 @@ public static Builder fromArgs(String[] args) { * validation. */ public Builder withValidation() { - return new Builder(getAppName(3)).withValidation(); + return new Builder().withValidation(); } /** A fluent PipelineOptions builder. */ @@ -155,13 +157,13 @@ public static class Builder { private final boolean strictParsing; // Do not allow direct instantiation - private Builder(String defaultAppName) { - this(defaultAppName, null, false, true); + private Builder() { + this(null, false, true); } - private Builder(String defaultAppName, String[] args, boolean validation, + private Builder(String[] args, boolean validation, boolean strictParsing) { - this.defaultAppName = defaultAppName; + this.defaultAppName = findCallersClassName(); this.args = args; this.validation = validation; this.strictParsing = strictParsing; @@ -192,7 +194,7 @@ private Builder(String defaultAppName, String[] args, boolean validation, */ public Builder fromArgs(String[] args) { Preconditions.checkNotNull(args, "Arguments should not be null."); - return new Builder(defaultAppName, args, validation, strictParsing); + return new Builder(args, validation, strictParsing); } /** @@ -202,7 +204,7 @@ public Builder fromArgs(String[] args) { * validation. */ public Builder withValidation() { - return new Builder(defaultAppName, args, true, strictParsing); + return new Builder(args, true, strictParsing); } /** @@ -210,7 +212,7 @@ public Builder withValidation() { * arguments. */ public Builder withoutStrictParsing() { - return new Builder(defaultAppName, args, validation, false); + return new Builder(args, validation, false); } /** @@ -260,15 +262,31 @@ public T as(Class klass) { } /** - * Returns the simple name of calling class at the stack trace {@code level}. + * Returns the simple name of the calling class using the current threads stack. */ - private static String getAppName(int level) { - StackTraceElement[] stackTrace = Thread.currentThread().getStackTrace(); - try { - return Class.forName(stackTrace[level].getClassName()).getSimpleName(); - } catch (ClassNotFoundException e) { - return "unknown"; + private static String findCallersClassName() { + Iterator elements = + Iterators.forArray(Thread.currentThread().getStackTrace()); + // First find the PipelineOptionsFactory/Builder class in the stack trace. + while (elements.hasNext()) { + StackTraceElement next = elements.next(); + if (PIPELINE_OPTIONS_FACTORY_CLASSES.contains(next.getClassName())) { + break; + } } + // Then find the first instance after which is not the PipelineOptionsFactory/Builder class. + while (elements.hasNext()) { + StackTraceElement next = elements.next(); + if (!PIPELINE_OPTIONS_FACTORY_CLASSES.contains(next.getClassName())) { + try { + return Class.forName(next.getClassName()).getSimpleName(); + } catch (ClassNotFoundException e) { + break; + } + } + } + + return "unknown"; } /** @@ -301,6 +319,11 @@ Class getProxyClass() { private static final ObjectMapper MAPPER = new ObjectMapper(); private static final Map>> SUPPORTED_PIPELINE_RUNNERS; + /** Classes which are used as the boundary in the stack trace to find the callers class name. */ + private static final Set PIPELINE_OPTIONS_FACTORY_CLASSES = ImmutableSet.of( + PipelineOptionsFactory.class.getName(), + Builder.class.getName()); + /** Methods which are ignored when validating the proxy class. */ private static final Set IGNORED_METHODS; @@ -334,7 +357,7 @@ Class getProxyClass() { // Store the list of all available pipeline runners. ImmutableMap.Builder>> builder = - new ImmutableMap.Builder<>(); + ImmutableMap.builder(); Set pipelineRunnerRegistrars = Sets.newTreeSet(ObjectsClassComparator.INSTANCE); pipelineRunnerRegistrars.addAll( From 24d0020c91086d40f6fa1296b1630156ef5bc0b9 Mon Sep 17 00:00:00 2001 From: bchambers Date: Wed, 21 Jan 2015 14:17:56 -0800 Subject: [PATCH 0088/1541] Update the BigQueryReaderTest to use fake JSON data rather than constructed Java wrappers. Fix two ClassCastExceptions that occurred when dealing with nested RECORDs and with REPEATED fields. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84468071 --- .../sdk/util/BigQueryTableRowIterator.java | 24 +- .../runners/worker/BigQueryReaderTest.java | 381 ++++++++++++++---- 2 files changed, 317 insertions(+), 88 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableRowIterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableRowIterator.java index a6ea658ae3f42..1dde22d8ca0b8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableRowIterator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableRowIterator.java @@ -20,7 +20,6 @@ import com.google.api.client.util.Preconditions; import com.google.api.services.bigquery.Bigquery; import com.google.api.services.bigquery.model.Table; -import com.google.api.services.bigquery.model.TableCell; import com.google.api.services.bigquery.model.TableDataList; import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableReference; @@ -33,6 +32,7 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; @@ -99,16 +99,19 @@ private Object getTypedCellValue(TableFieldSchema fieldSchema, Object v) { if (Objects.equals(fieldSchema.getMode(), "REPEATED")) { TableFieldSchema elementSchema = fieldSchema.clone().setMode("REQUIRED"); - List rawValues = (List) v; + @SuppressWarnings("unchecked") + List> rawValues = (List>) v; List values = new ArrayList(rawValues.size()); - for (Object element : rawValues) { - values.add(getTypedCellValue(elementSchema, element)); + for (Map element : rawValues) { + values.add(getTypedCellValue(elementSchema, element.get("v"))); } return values; } if (fieldSchema.getType().equals("RECORD")) { - return getTypedTableRow(fieldSchema.getFields(), (TableRow) v); + @SuppressWarnings("unchecked") + Map typedV = (Map) v; + return getTypedTableRow(fieldSchema.getFields(), typedV); } if (fieldSchema.getType().equals("FLOAT")) { @@ -122,18 +125,19 @@ private Object getTypedCellValue(TableFieldSchema fieldSchema, Object v) { return v; } - private TableRow getTypedTableRow(List fields, TableRow rawRow) { - List cells = rawRow.getF(); + private TableRow getTypedTableRow(List fields, Map rawRow) { + @SuppressWarnings("unchecked") + List> cells = (List>) rawRow.get("f"); Preconditions.checkState(cells.size() == fields.size()); - Iterator cellIt = cells.iterator(); + Iterator> cellIt = cells.iterator(); Iterator fieldIt = fields.iterator(); TableRow row = new TableRow(); while (cellIt.hasNext()) { - TableCell cell = cellIt.next(); + Map cell = cellIt.next(); TableFieldSchema fieldSchema = fieldIt.next(); - row.set(fieldSchema.getName(), getTypedCellValue(fieldSchema, cell.getV())); + row.set(fieldSchema.getName(), getTypedCellValue(fieldSchema, cell.get("v"))); } return row; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java index f524e74744bfe..5c73190d0b4fe 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java @@ -16,23 +16,24 @@ package com.google.cloud.dataflow.sdk.runners.worker; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.atLeast; +import static org.mockito.Matchers.endsWith; +import static org.mockito.Matchers.eq; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; +import com.google.api.client.http.LowLevelHttpRequest; +import com.google.api.client.json.Json; +import com.google.api.client.testing.http.MockHttpTransport; +import com.google.api.client.testing.http.MockLowLevelHttpRequest; +import com.google.api.client.testing.http.MockLowLevelHttpResponse; import com.google.api.services.bigquery.Bigquery; -import com.google.api.services.bigquery.model.Table; -import com.google.api.services.bigquery.model.TableCell; -import com.google.api.services.bigquery.model.TableDataList; -import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.util.Transport; import com.google.cloud.dataflow.sdk.util.common.worker.Reader; +import com.google.common.collect.Lists; import org.junit.After; import org.junit.Assert; @@ -42,107 +43,308 @@ import org.junit.runners.JUnit4; import org.mockito.Mock; import org.mockito.MockitoAnnotations; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import java.io.IOException; -import java.util.Arrays; -import java.util.LinkedList; import java.util.List; /** * Tests for BigQueryReader. * - *

The tests just make sure a basic scenario of reading works because the class itself is a - * thin wrapper over {@code BigQueryTableRowIterator}. The tests for the wrapped class have - * comprehensive coverage. + *

The tests make sure a simple scenario (reading two rows) work for the various kinds of fields + * and modes. */ @RunWith(JUnit4.class) public class BigQueryReaderTest { + + private static final String PROJECT_ID = "project"; + private static final String DATASET = "dataset"; + private static final String TABLE = "table"; + + private static final String GET_TABLE_REQUEST_PATH = + String.format("projects/%s/datasets/%s/tables/%s", PROJECT_ID, DATASET, TABLE); + + // This is a real response (with some unused fields removed) for the table created from this + // schema: + // [ + // {"name":"name","type":"STRING"}, + // {"name":"integer", "type":"INTEGER"}, + // {"name":"float", "type":"FLOAT"}, + // {"name":"bool", "type":"BOOLEAN"}, + // {"name":"record", "type":"RECORD", "fields":[ + // {"name": "nestedInt","type":"INTEGER"}, + // {"name": "nestedFloat","type":"FLOAT"} + // ]}, + // {"name":"repeatedInt", "type":"INTEGER", "mode":"REPEATED"}, + // {"name":"repeatedFloat", "type":"FLOAT", "mode":"REPEATED"}, + // + // {"name":"repeatedRecord", "type":"RECORD", "mode":"REPEATED", "fields":[ + // {"name": "bool", "type": "BOOLEAN"}, + // {"name": "string", "type": "STRING"} + // ]} + //] + private static final String GET_TABLE_RESPONSE_JSON = "{\n" + + " \"schema\": {\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"name\",\n" + + " \"type\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"name\": \"integer\",\n" + + " \"type\": \"INTEGER\"\n" + + " },\n" + + " {\n" + + " \"name\": \"float\",\n" + + " \"type\": \"FLOAT\"\n" + + " },\n" + + " {\n" + + " \"name\": \"bool\",\n" + + " \"type\": \"BOOLEAN\"\n" + + " },\n" + + " {\n" + + " \"name\": \"record\",\n" + + " \"type\": \"RECORD\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"nestedInt\",\n" + + " \"type\": \"INTEGER\"\n" + + " },\n" + + " {\n" + + " \"name\": \"nestedFloat\",\n" + + " \"type\": \"FLOAT\"\n" + + " }\n" + + " ]\n" + + " },\n" + + " {\n" + + " \"name\": \"repeatedInt\",\n" + + " \"type\": \"INTEGER\",\n" + + " \"mode\": \"REPEATED\"\n" + + " },\n" + + " {\n" + + " \"name\": \"repeatedFloat\",\n" + + " \"type\": \"FLOAT\",\n" + + " \"mode\": \"REPEATED\"\n" + + " },\n" + + " {\n" + + " \"name\": \"repeatedRecord\",\n" + + " \"type\": \"RECORD\",\n" + + " \"mode\": \"REPEATED\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"bool\",\n" + + " \"type\": \"BOOLEAN\"\n" + + " },\n" + + " {\n" + + " \"name\": \"string\",\n" + + " \"type\": \"STRING\"\n" + + " }\n" + + " ]\n" + + " }\n" + + " ]\n" + + " },\n" + + " \"numRows\": \"2\",\n" + + " \"type\": \"TABLE\"\n" + + "}"; + + private static final String LIST_TABLE_DATA_REQUEST_PATH = + String.format("projects/%s/datasets/%s/tables/%s/data", PROJECT_ID, DATASET, TABLE); + + // This is a real response (with some unused fields removed) for the table listed above, populated + // with the following data: + //{"name": "Arthur", "integer": 42, "float": 3.14159, "bool": "false", + // "record": {"nestedInt": 43, "nestedFloat": "4.14159"}, + // "repeatedInt":[42, 43, 79]}, + // + //{"name": "Allison", "integer": 79, "float": 2.71828, "bool": "true", + // "record": {"nestedInt": 80, "nestedFloat": "3.71828"}, + // "repeatedFloat":[3.14159, 2.71828], + // "repeatedRecord":[{"bool":"true","string":"hello"}, + // {"bool":"false","string":"world"}]} + private static final String LIST_TABLEDATA_RESPONSE_JSON = "{\n" + + " \"totalRows\": \"2\",\n" + + " \"rows\": [\n" + + " {\n" + + " \"f\": [\n" + + " {\n" + + " \"v\": \"Arthur\"\n" + + " },\n" + + " {\n" + + " \"v\": \"42\"\n" + + " },\n" + + " {\n" + + " \"v\": \"3.14159\"\n" + + " },\n" + + " {\n" + + " \"v\": \"false\"\n" + + " },\n" + + " {\n" + + " \"v\": {\n" + + " \"f\": [\n" + + " {\n" + + " \"v\": \"43\"\n" + + " },\n" + + " {\n" + + " \"v\": \"4.14159\"\n" + + " }\n" + + " ]\n" + + " }\n" + + " },\n" + + " {\n" + + " \"v\": [\n" + + " {\n" + + " \"v\": \"42\"\n" + + " },\n" + + " {\n" + + " \"v\": \"43\"\n" + + " },\n" + + " {\n" + + " \"v\": \"79\"\n" + + " }\n" + + " ]\n" + + " },\n" + + " {\n" + + " \"v\": [\n" + + " ]\n" + + " },\n" + + " {\n" + + " \"v\": [\n" + + " ]\n" + + " }\n" + + " ]\n" + + " },\n" + + " {\n" + + " \"f\": [\n" + + " {\n" + + " \"v\": \"Allison\"\n" + + " },\n" + + " {\n" + + " \"v\": \"79\"\n" + + " },\n" + + " {\n" + + " \"v\": \"2.71828\"\n" + + " },\n" + + " {\n" + + " \"v\": \"true\"\n" + + " },\n" + + " {\n" + + " \"v\": {\n" + + " \"f\": [\n" + + " {\n" + + " \"v\": \"80\"\n" + + " },\n" + + " {\n" + + " \"v\": \"3.71828\"\n" + + " }\n" + + " ]\n" + + " }\n" + + " },\n" + + " {\n" + + " \"v\": [\n" + + " ]\n" + + " },\n" + + " {\n" + + " \"v\": [\n" + + " {\n" + + " \"v\": \"3.14159\"\n" + + " },\n" + + " {\n" + + " \"v\": \"2.71828\"\n" + + " }\n" + + " ]\n" + + " },\n" + + " {\n" + + " \"v\": [\n" + + " {\n" + + " \"v\": {\n" + + " \"f\": [\n" + + " {\n" + + " \"v\": \"true\"\n" + + " },\n" + + " {\n" + + " \"v\": \"hello\"\n" + + " }\n" + + " ]\n" + + " }\n" + + " },\n" + + " {\n" + + " \"v\": {\n" + + " \"f\": [\n" + + " {\n" + + " \"v\": \"false\"\n" + + " },\n" + + " {\n" + + " \"v\": \"world\"\n" + + " }\n" + + " ]\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + + " ]\n" + + " }\n" + + " ]\n" + + "}"; + @Mock - private Bigquery mockClient; - @Mock - private Bigquery.Tables mockTables; - @Mock - private Bigquery.Tables.Get mockTablesGet; - @Mock - private Bigquery.Tabledata mockTabledata; - @Mock - private Bigquery.Tabledata.List mockTabledataList; + private MockHttpTransport mockTransport; + + private Bigquery bigQueryClient; @Before - public void setUp() { + public void setUp() throws Exception { MockitoAnnotations.initMocks(this); - } + when(mockTransport.buildRequest(eq("GET"), endsWith(GET_TABLE_REQUEST_PATH))) + .thenAnswer(new Answer() { + @Override + public LowLevelHttpRequest answer(InvocationOnMock invocation) throws Throwable { + MockLowLevelHttpResponse response = new MockLowLevelHttpResponse() + .setContentType(Json.MEDIA_TYPE) + .setContent(GET_TABLE_RESPONSE_JSON); + return new MockLowLevelHttpRequest((String) invocation.getArguments()[1]) + .setResponse(response); + } + }); + when(mockTransport.buildRequest(eq("GET"), endsWith(LIST_TABLE_DATA_REQUEST_PATH))) + .thenAnswer(new Answer() { + @Override + public LowLevelHttpRequest answer(InvocationOnMock invocation) throws Throwable { + MockLowLevelHttpResponse response = new MockLowLevelHttpResponse() + .setContentType(Json.MEDIA_TYPE) + .setContent(LIST_TABLEDATA_RESPONSE_JSON); + return new MockLowLevelHttpRequest((String) invocation.getArguments()[1]) + .setResponse(response); + } + }); + when(mockTransport.supportsMethod("GET")).thenReturn(true); - @After - public void tearDown() { - verifyNoMoreInteractions(mockClient); - verifyNoMoreInteractions(mockTables); - verifyNoMoreInteractions(mockTablesGet); - verifyNoMoreInteractions(mockTabledata); - verifyNoMoreInteractions(mockTabledataList); + bigQueryClient = new Bigquery(mockTransport, Transport.getJsonFactory(), null); } - private void onTableGet(Table table) throws IOException { - when(mockClient.tables()).thenReturn(mockTables); - when(mockTables.get(anyString(), anyString(), anyString())).thenReturn(mockTablesGet); - when(mockTablesGet.execute()).thenReturn(table); + @After + public void tearDown() throws IOException { + verify(mockTransport, atLeastOnce()).supportsMethod("GET"); + verifyNoMoreInteractions(mockTransport); } private void verifyTableGet() throws IOException { - verify(mockClient).tables(); - verify(mockTables).get("project", "dataset", "table"); - verify(mockTablesGet).execute(); - } - - private void onTableList(TableDataList result) throws IOException { - when(mockClient.tabledata()).thenReturn(mockTabledata); - when(mockTabledata.list(anyString(), anyString(), anyString())).thenReturn(mockTabledataList); - when(mockTabledataList.execute()).thenReturn(result); + verify(mockTransport).buildRequest(eq("GET"), endsWith(GET_TABLE_REQUEST_PATH)); } private void verifyTabledataList() throws IOException { - verify(mockClient, atLeastOnce()).tabledata(); - verify(mockTabledata, atLeastOnce()).list("project", "dataset", "table"); - verify(mockTabledataList, atLeastOnce()).execute(); - // Max results may be set when testing for an empty table. - verify(mockTabledataList, atLeast(0)).setMaxResults(anyLong()); - } - - private Table basicTableSchema() { - return new Table().setSchema(new TableSchema().setFields(Arrays.asList( - new TableFieldSchema().setName("name").setType("STRING"), - new TableFieldSchema().setName("integer").setType("INTEGER"), - new TableFieldSchema().setName("float").setType("FLOAT"), - new TableFieldSchema().setName("bool").setType("BOOLEAN")))); - } - - private TableRow rawRow(Object... args) { - List cells = new LinkedList<>(); - for (Object a : args) { - cells.add(new TableCell().setV(a)); - } - return new TableRow().setF(cells); - } - - private TableDataList rawDataList(TableRow... rows) { - return new TableDataList().setRows(Arrays.asList(rows)); + verify(mockTransport).buildRequest(eq("GET"), endsWith(LIST_TABLE_DATA_REQUEST_PATH)); } @Test public void testRead() throws Exception { - onTableGet(basicTableSchema()); - - // BQ API data is always encoded as a string - TableDataList dataList = rawDataList( - rawRow("Arthur", "42", "3.14159", "false"), rawRow("Allison", "79", "2.71828", "true")); - onTableList(dataList); - BigQueryReader reader = new BigQueryReader( - mockClient, - new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table")); + bigQueryClient, + new TableReference().setProjectId(PROJECT_ID).setDatasetId(DATASET).setTableId(TABLE)); Reader.ReaderIterator iterator = reader.iterator(); Assert.assertTrue(iterator.hasNext()); + TableRow row = iterator.next(); Assert.assertEquals("Arthur", row.get("name")); @@ -150,6 +352,14 @@ public void testRead() throws Exception { Assert.assertEquals(3.14159, row.get("float")); Assert.assertEquals(false, row.get("bool")); + TableRow nested = (TableRow) row.get("record"); + Assert.assertEquals("43", nested.get("nestedInt")); + Assert.assertEquals(4.14159, nested.get("nestedFloat")); + + Assert.assertEquals(Lists.newArrayList("42", "43", "79"), row.get("repeatedInt")); + Assert.assertTrue(((List) row.get("repeatedFloat")).isEmpty()); + Assert.assertTrue(((List) row.get("repeatedRecord")).isEmpty()); + row = iterator.next(); Assert.assertEquals("Allison", row.get("name")); @@ -157,6 +367,21 @@ public void testRead() throws Exception { Assert.assertEquals(2.71828, row.get("float")); Assert.assertEquals(true, row.get("bool")); + nested = (TableRow) row.get("record"); + Assert.assertEquals("80", nested.get("nestedInt")); + Assert.assertEquals(3.71828, nested.get("nestedFloat")); + + Assert.assertTrue(((List) row.get("repeatedInt")).isEmpty()); + Assert.assertEquals(Lists.newArrayList(3.14159, 2.71828), row.get("repeatedFloat")); + + @SuppressWarnings("unchecked") + List nestedRecords = (List) row.get("repeatedRecord"); + Assert.assertEquals(2, nestedRecords.size()); + Assert.assertEquals("hello", nestedRecords.get(0).get("string")); + Assert.assertEquals(true, nestedRecords.get(0).get("bool")); + Assert.assertEquals("world", nestedRecords.get(1).get("string")); + Assert.assertEquals(false, nestedRecords.get(1).get("bool")); + Assert.assertFalse(iterator.hasNext()); verifyTableGet(); From 6616fd636f289d52f255b1252434099838f7b685 Mon Sep 17 00:00:00 2001 From: vgough Date: Thu, 22 Jan 2015 12:05:35 -0800 Subject: [PATCH 0089/1541] Fixes a randomization bug and removes some unused checks in ProxyInvocationHandler. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84553881 --- .../sdk/options/ProxyInvocationHandler.java | 16 +++++----------- .../sdk/options/ProxyInvocationHandlerTest.java | 13 ++++++++----- 2 files changed, 13 insertions(+), 16 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java index 39ab988182370..e74e5a8f6e820 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java @@ -60,8 +60,8 @@ * Represents and {@link InvocationHandler} for a {@link Proxy}. The invocation handler uses bean * introspection of the proxy class to store and retrieve values based off of the property name. *

- * Unset properties use the {@Default} metadata on the getter to return values. If there - * is no {@Default} annotation on the getter, then a default as * per the Java Language Specification for the expected return type is returned. *

@@ -75,7 +75,7 @@ class ProxyInvocationHandler implements InvocationHandler { * No two instances of this class are considered equivalent hence we generate a random hash code * between 0 and {@link Integer#MAX_VALUE}. */ - private final int hashCode = (int) Math.random() * Integer.MAX_VALUE; + private final int hashCode = (int) (Math.random() * Integer.MAX_VALUE); private final Set> knownInterfaces; private final ClassToInstanceMap interfaceToProxyCache; private final Map options; @@ -214,8 +214,8 @@ private Object getValueFromJson(String propertyName, Method method) { } /** - * Returns a default value for the method based upon {@Default} metadata on the getter - * to return values. If there is no {@Default} annotation on the getter, then a default as * per the Java Language Specification for the expected return type is returned. * @@ -246,12 +246,6 @@ private Object getDefault(PipelineOptions proxy, Method method) { return ((Default.Float) annotation).value(); } else if (annotation instanceof Default.Double) { return ((Default.Double) annotation).value(); - } else if (annotation instanceof Default.String) { - return ((Default.String) annotation).value(); - } else if (annotation instanceof Default.String) { - return ((Default.String) annotation).value(); - } else if (annotation instanceof Default.String) { - return ((Default.String) annotation).value(); } else if (annotation instanceof Default.Enum) { return Enum.valueOf((Class) method.getReturnType(), ((Default.Enum) annotation).value()); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java index 01085322f26e4..d893ba4d08d30 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java @@ -209,14 +209,17 @@ public void testHashCode() throws Exception { ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); Simple proxy = handler.as(Simple.class); JLSDefaults sameAsProxy = proxy.as(JLSDefaults.class); + ProxyInvocationHandler handler2 = new ProxyInvocationHandler(Maps.newHashMap()); - Simple proxy2 = handler.as(Simple.class); - JLSDefaults sameAsProxy2 = proxy.as(JLSDefaults.class); + Simple proxy2 = handler2.as(Simple.class); + JLSDefaults sameAsProxy2 = proxy2.as(JLSDefaults.class); + + // Hashcode comparisons below depend on random numbers, so could fail if seed changes. assertTrue(handler.hashCode() == proxy.hashCode()); assertTrue(proxy.hashCode() == sameAsProxy.hashCode()); - assertFalse(handler.hashCode() != handler2.hashCode()); - assertFalse(proxy.hashCode() != proxy2.hashCode()); - assertFalse(proxy.hashCode() != sameAsProxy2.hashCode()); + assertFalse(handler.hashCode() == handler2.hashCode()); + assertFalse(proxy.hashCode() == proxy2.hashCode()); + assertFalse(proxy.hashCode() == sameAsProxy2.hashCode()); } @Test From 1942ce0a21d65885ce6be91a6746c3e747933bb0 Mon Sep 17 00:00:00 2001 From: robertwb Date: Thu, 22 Jan 2015 12:06:11 -0800 Subject: [PATCH 0090/1541] Use deterministic tag ids for tags created in static initializers. These tags are often assigned to static members variables, which are re-created rather than serialized on remote machines, yielding surprising behavior. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84553936 --- .../cloud/dataflow/sdk/values/TupleTag.java | 33 +++++++-- .../dataflow/sdk/values/TupleTagTest.java | 69 +++++++++++++++++++ 2 files changed, 97 insertions(+), 5 deletions(-) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/values/TupleTagTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java index 4b47e80fdbabc..3a0fa15434216 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java @@ -21,6 +21,8 @@ import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.common.collect.HashMultiset; +import com.google.common.collect.Multiset; import com.google.common.reflect.TypeToken; import com.fasterxml.jackson.annotation.JsonCreator; @@ -125,17 +127,38 @@ public TypeToken getTypeToken() { // Internal details below here. static final Random RANDOM = new Random(0); + private static final Multiset staticInits = HashMultiset.create(); final String id; final boolean generated; /** Generates and returns a fresh unique id for a TupleTag's id. */ - static String genId() { - long randomLong; - synchronized (RANDOM) { - randomLong = RANDOM.nextLong(); + static synchronized String genId() { + // It is a common pattern to store tags that are shared between the main + // program and workers in static variables, but such references are not + // serialized as part of the *Fns state. Fortunately, most such tags + // are constructed in static class initializers, e.g. + // + // static final TupleTag MY_TAG = new TupleTag<>(); + // + // and class initialization order is well defined by the JVM spec, so in + // this case we can assign deterministic ids. + StackTraceElement[] stackTrace = Thread.currentThread().getStackTrace(); + for (StackTraceElement frame : stackTrace) { + if (frame.getMethodName().equals("")) { + int counter = staticInits.add(frame.getClassName(), 1); + return frame.getClassName() + "#" + counter; + } } - return Long.toHexString(randomLong); + // Otherwise, assume it'll be serialized and choose a random value to reduce + // the chance of collision. + String nonce = Long.toHexString(RANDOM.nextLong()); + // [Thread.getStackTrace, TupleTag.getId, TupleTag., caller, ...] + String caller = stackTrace.length >= 4 + ? stackTrace[3].getClassName() + "." + stackTrace[3].getMethodName() + + ":" + stackTrace[3].getLineNumber() + : "unknown"; + return caller + "#" + nonce; } @JsonCreator diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/TupleTagTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/TupleTagTest.java new file mode 100644 index 0000000000000..dfdae4df5907d --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/values/TupleTagTest.java @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.values; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link TupleTag}. + */ +@RunWith(JUnit4.class) +public class TupleTagTest { + + private static TupleTag staticTag = new TupleTag<>(); + private static TupleTag staticBlockTag; + private static TupleTag staticMethodTag = createTag(); + private static TupleTag instanceMethodTag = new AnotherClass().createAnotherTag(); + + static { + staticBlockTag = new TupleTag<>(); + } + + private static TupleTag createTag() { + return new TupleTag<>(); + } + + private static class AnotherClass { + private static TupleTag anotherTag = new TupleTag<>(); + private TupleTag createAnotherTag() { + return new TupleTag<>(); + } + } + + @Test + public void testStaticTupleTag() { + assertEquals("com.google.cloud.dataflow.sdk.values.TupleTagTest#0", staticTag.getId()); + assertEquals("com.google.cloud.dataflow.sdk.values.TupleTagTest#3", staticBlockTag.getId()); + assertEquals("com.google.cloud.dataflow.sdk.values.TupleTagTest#1", staticMethodTag.getId()); + assertEquals("com.google.cloud.dataflow.sdk.values.TupleTagTest#2", instanceMethodTag.getId()); + assertEquals( + "com.google.cloud.dataflow.sdk.values.TupleTagTest$AnotherClass#0", + AnotherClass.anotherTag.getId()); + } + + @Test + public void testNonstaticTupleTag() { + assertEquals("com.google.cloud.dataflow.sdk.values.TupleTagTest.testNonstaticTupleTag:65", + new TupleTag().getId().split("#")[0]); + assertNotEquals(new TupleTag().getId(), new TupleTag().getId()); + } +} From e09e6430c3c0ed584081ef048ab8e899863f42ce Mon Sep 17 00:00:00 2001 From: vgough Date: Thu, 22 Jan 2015 16:45:28 -0800 Subject: [PATCH 0091/1541] Fixes a couple window comparison issues in CalendarWindows. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84579045 --- .../dataflow/sdk/transforms/windowing/CalendarWindows.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java index 70590665be5ac..cdbc1a64232b3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java @@ -212,7 +212,7 @@ public boolean isCompatible(WindowingFn other) { } MonthsWindows that = (MonthsWindows) other; return number == that.number - && dayOfMonth == dayOfMonth + && dayOfMonth == that.dayOfMonth && startDate == that.startDate && timeZone == that.timeZone; } @@ -291,8 +291,8 @@ public boolean isCompatible(WindowingFn other) { } YearsWindows that = (YearsWindows) other; return number == that.number - && monthOfYear == monthOfYear - && dayOfMonth == dayOfMonth + && monthOfYear == that.monthOfYear + && dayOfMonth == that.dayOfMonth && startDate == that.startDate && timeZone == that.timeZone; } From 00dc9415d8b2c7b2ceceab87b7c45a2541736c77 Mon Sep 17 00:00:00 2001 From: davor Date: Fri, 23 Jan 2015 09:39:52 -0800 Subject: [PATCH 0092/1541] Fix parts of flakiness in RateLimitingTest. runWithRate() returns the number of milliseconds that it took at execute a DoFn. The callers are expecting that the execution time is greater than the sum of the delay intervals during execution. This test is flaky, in part by, the system time resolution. If the rest of execution took very little time, say, less than the timer resolution, it is certainly possible that the execution time is equal to the sum of the delay intervals. Now, Thread.sleep() calls will, in most cases, sleep for *at least* the given time. But, this is also subject to precision and accuracy of system timers and schedulers, and not guaranteed by Java. This is possible to cause further issues, but unlikely to actually happen. The problem tends to be bigger on Windows, where timer resolution is 15 or 50 ms, whereas on Linux is usually by an order of magnitude smaller. It was still hit on both platforms, however. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84630029 --- .../cloud/dataflow/sdk/transforms/RateLimitingTest.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java index 5a1c2a8f1340c..ee6e17a66810d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java @@ -18,7 +18,6 @@ import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.lessThan; @@ -132,7 +131,7 @@ public void testRateLimitingMax() { long perElementPause = (long) (1000L / rate); long minDuration = (n - 1) * perElementPause; - Assert.assertThat(duration, greaterThan(minDuration)); + Assert.assertThat(duration, greaterThanOrEqualTo(minDuration)); } @Test(timeout = 5000L) @@ -185,10 +184,10 @@ public void testBacklogLimiter() { long duration = runWithRate(2 * RateLimiting.DEFAULT_MAX_PARALLELISM, -1.0 /* unlimited */, new DelayFn()); - // Should take > 2x the delay interval, since no more than half the elements - // can be scheduled at once. + // Should take >= 2x the delay interval, since no more than half the + // elements can be scheduled at once. Assert.assertThat(duration, - greaterThan(2 * DelayFn.DELAY_MS)); + greaterThanOrEqualTo(2 * DelayFn.DELAY_MS)); } private long runWithRate(int numElements, double rateLimit, From 006401cd8c27a7c02f99001bd6acef5e72efb154 Mon Sep 17 00:00:00 2001 From: ccy Date: Fri, 23 Jan 2015 11:43:20 -0800 Subject: [PATCH 0093/1541] Don't rewrap RuntimeException in NormalParDoFn$1.output for cleaner error reporting. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84640341 --- .../cloud/dataflow/sdk/runners/worker/NormalParDoFn.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java index ea9ea4a970d87..5b522875f1592 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java @@ -37,6 +37,7 @@ import com.google.cloud.dataflow.sdk.util.common.worker.Receiver; import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.base.Throwables; import java.util.ArrayList; import java.util.HashMap; @@ -203,8 +204,8 @@ public Receiver initialize(TupleTag tag) { public void output(Receiver receiver, WindowedValue output) { try { receiver.process(output); - } catch (Exception e) { - throw new RuntimeException(e); + } catch (Throwable t) { + Throwables.propagate(t); } } }, From f8329b20387d455c3b3d9880f43dcdd796101637 Mon Sep 17 00:00:00 2001 From: ccy Date: Fri, 23 Jan 2015 15:59:09 -0800 Subject: [PATCH 0094/1541] Throw return value from Throwables.propagate in NormalParDoFn exception handling. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84661143 --- .../google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java index 5b522875f1592..09e3d247aa6b0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java @@ -205,7 +205,7 @@ public void output(Receiver receiver, WindowedValue output) { try { receiver.process(output); } catch (Throwable t) { - Throwables.propagate(t); + throw Throwables.propagate(t); } } }, From e88a30230017a7e8c04f93236f9e88a8ddb66801 Mon Sep 17 00:00:00 2001 From: ccy Date: Fri, 23 Jan 2015 16:07:23 -0800 Subject: [PATCH 0095/1541] Don't rewrap UserCodeException for cleaner user error messages. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84661933 --- .../java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java index 09320962a3f2d..e8ac2486da6e0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.util.ExecutionContext.StepContext; import com.google.cloud.dataflow.sdk.util.common.CounterSet; import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.base.Throwables; import java.util.ArrayList; import java.util.List; @@ -114,6 +115,7 @@ public void startBundle() { fn.startBundle(context); } catch (Throwable t) { // Exception in user code. + Throwables.propagateIfInstanceOf(t, UserCodeException.class); throw new UserCodeException(t); } } @@ -130,6 +132,7 @@ public void processElement(WindowedValue elem) { fn.processElement(processContext); } catch (Throwable t) { // Exception in user code. + Throwables.propagateIfInstanceOf(t, UserCodeException.class); throw new UserCodeException(t); } } @@ -141,6 +144,7 @@ public void finishBundle() { fn.finishBundle(context); } catch (Throwable t) { // Exception in user code. + Throwables.propagateIfInstanceOf(t, UserCodeException.class); throw new UserCodeException(t); } } From 5a4d6e3e2d59646fbd870d163d30d7e388c22bd8 Mon Sep 17 00:00:00 2001 From: robertwb Date: Sat, 24 Jan 2015 01:47:59 -0800 Subject: [PATCH 0096/1541] Rename WindowingFn to WindowFn for consistency with the other *Fns. This is not backwards compatible, but is unlikely to affect much user code at this point. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84685618 --- .../sdk/runners/DataflowPipelineRunner.java | 2 +- .../runners/DataflowPipelineTranslator.java | 12 ++--- .../runners/worker/AssignWindowsParDoFn.java | 10 ++-- .../worker/GroupAlsoByWindowsParDoFn.java | 10 ++-- .../sdk/runners/worker/NormalParDoFn.java | 2 +- ...nTestUtils.java => WindowFnTestUtils.java} | 34 ++++++------- .../dataflow/sdk/transforms/Combine.java | 18 +++---- .../cloud/dataflow/sdk/transforms/Count.java | 2 +- .../cloud/dataflow/sdk/transforms/DoFn.java | 16 +++--- .../dataflow/sdk/transforms/Flatten.java | 20 ++++---- .../dataflow/sdk/transforms/GroupByKey.java | 50 +++++++++---------- .../cloud/dataflow/sdk/transforms/Keys.java | 2 +- .../cloud/dataflow/sdk/transforms/KvSwap.java | 2 +- .../cloud/dataflow/sdk/transforms/ParDo.java | 8 +-- .../dataflow/sdk/transforms/Partition.java | 2 +- .../sdk/transforms/RemoveDuplicates.java | 2 +- .../cloud/dataflow/sdk/transforms/Values.java | 2 +- .../dataflow/sdk/transforms/WithKeys.java | 2 +- .../transforms/windowing/CalendarWindows.java | 28 +++++------ .../transforms/windowing/FixedWindows.java | 6 +-- .../transforms/windowing/GlobalWindow.java | 6 +-- ...dWindowingFn.java => InvalidWindowFn.java} | 28 +++++------ .../MergeOverlappingIntervalWindows.java | 6 +-- ...ndowingFn.java => NonMergingWindowFn.java} | 8 +-- ...owingFn.java => PartitioningWindowFn.java} | 6 +-- .../sdk/transforms/windowing/Sessions.java | 10 ++-- .../transforms/windowing/SlidingWindows.java | 6 +-- .../sdk/transforms/windowing/Window.java | 40 +++++++-------- .../{WindowingFn.java => WindowFn.java} | 18 +++---- .../transforms/windowing/package-info.java | 2 +- .../dataflow/sdk/util/AbstractWindowSet.java | 14 +++--- .../dataflow/sdk/util/AssignWindowsDoFn.java | 12 ++--- .../dataflow/sdk/util/BufferingWindowSet.java | 16 +++--- .../cloud/dataflow/sdk/util/DoFnContext.java | 18 +++---- .../cloud/dataflow/sdk/util/DoFnInfo.java | 12 ++--- .../cloud/dataflow/sdk/util/DoFnRunner.java | 14 +++--- .../sdk/util/GroupAlsoByWindowsDoFn.java | 28 +++++------ .../sdk/util/PartitionBufferingWindowSet.java | 12 ++--- .../util/StreamingGroupAlsoByWindowsDoFn.java | 30 +++++------ .../dataflow/sdk/values/PCollection.java | 24 ++++----- .../dataflow/sdk/values/PCollectionTuple.java | 6 +-- .../runners/worker/ParDoFnFactoryTest.java | 2 +- .../dataflow/sdk/transforms/FlattenTest.java | 12 ++--- .../sdk/transforms/GroupByKeyTest.java | 16 +++--- .../dataflow/sdk/transforms/ParDoTest.java | 2 +- .../windowing/CalendarWindowsTest.java | 18 +++---- .../windowing/FixedWindowsTest.java | 12 ++--- .../transforms/windowing/SessionsTest.java | 14 +++--- .../windowing/SlidingWindowsTest.java | 16 +++--- .../transforms/windowing/WindowingTest.java | 8 +-- .../sdk/util/GroupAlsoByWindowsDoFnTest.java | 6 +-- .../StreamingGroupAlsoByWindowsDoFnTest.java | 4 +- 52 files changed, 328 insertions(+), 328 deletions(-) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/{WindowingFnTestUtils.java => WindowFnTestUtils.java} (84%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/{InvalidWindowingFn.java => InvalidWindowFn.java} (58%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/{NonMergingWindowingFn.java => NonMergingWindowFn.java} (80%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/{PartitioningWindowingFn.java => PartitioningWindowFn.java} (86%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/{WindowingFn.java => WindowFn.java} (87%) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index 1a0d873307375..b6003b2001064 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -159,7 +159,7 @@ public Output apply( if (transform instanceof Combine.GroupedValues) { // TODO: Redundant with translator registration? return (Output) PCollection.createPrimitiveOutputInternal( - ((PCollection) input).getWindowingFn()); + ((PCollection) input).getWindowFn()); } else if (transform instanceof GroupByKey) { // The DataflowPipelineRunner implementation of GroupByKey will sort values by timestamp, // so no need for an explicit sort transform. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index 3bcca5e8b3afb..54dd9326c67f6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -66,7 +66,7 @@ import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.View; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.DoFnInfo; import com.google.cloud.dataflow.sdk.util.OutputReference; @@ -600,7 +600,7 @@ public void addOutput(String name, PValue value) { // Wrap the PCollection element Coder inside a WindowedValueCoder. coder = WindowedValue.getFullCoder( coder, - ((PCollection) value).getWindowingFn().windowCoder()); + ((PCollection) value).getWindowFn().windowCoder()); } } else { // No output coder to encode. @@ -884,7 +884,7 @@ private void translateMultiHelper( TranslationContext context) { context.addStep(transform, "ParallelDo"); translateInputs(transform.getInput(), transform.getSideInputs(), context); - translateFn(transform.getFn(), transform.getInput().getWindowingFn(), context); + translateFn(transform.getFn(), transform.getInput().getWindowFn(), context); translateOutputs(transform.getOutput(), context); } }); @@ -904,7 +904,7 @@ private void translateSingleHelper( TranslationContext context) { context.addStep(transform, "ParallelDo"); translateInputs(transform.getInput(), transform.getSideInputs(), context); - translateFn(transform.getFn(), transform.getInput().getWindowingFn(), context); + translateFn(transform.getFn(), transform.getInput().getWindowFn(), context); context.addOutput("out", transform.getOutput()); } }); @@ -964,12 +964,12 @@ private static void translateSideInputs( private static void translateFn( DoFn fn, - WindowingFn windowingFn, + WindowFn windowFn, TranslationContext context) { context.addInput(PropertyNames.USER_FN, fn.getClass().getName()); context.addInput( PropertyNames.SERIALIZED_FN, - byteArrayToJsonString(serializeToByteArray(new DoFnInfo(fn, windowingFn)))); + byteArrayToJsonString(serializeToByteArray(new DoFnInfo(fn, windowFn)))); if (fn instanceof DoFn.RequiresKeyedState) { context.addInput(PropertyNames.USES_KEYED_STATE, "true"); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java index 2a22793e81c4a..16564e1a1d467 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java @@ -21,7 +21,7 @@ import com.google.api.services.dataflow.model.MultiOutputInfo; import com.google.api.services.dataflow.model.SideInputInfo; import com.google.cloud.dataflow.sdk.options.PipelineOptions; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.DoFnInfo; @@ -54,16 +54,16 @@ public static AssignWindowsParDoFn create( CounterSet.AddCounterMutator addCounterMutator, StateSampler sampler /* unused */) throws Exception { - final Object windowingFn = + final Object windowFn = SerializableUtils.deserializeFromByteArray( getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), "serialized window fn"); - if (!(windowingFn instanceof WindowingFn)) { + if (!(windowFn instanceof WindowFn)) { throw new Exception( - "unexpected kind of WindowingFn: " + windowingFn.getClass().getName()); + "unexpected kind of WindowFn: " + windowFn.getClass().getName()); } - final AssignWindowsDoFn assignFn = new AssignWindowsDoFn((WindowingFn) windowingFn); + final AssignWindowsDoFn assignFn = new AssignWindowsDoFn((WindowFn) windowFn); DoFnInfoFactory fnFactory = new DoFnInfoFactory() { @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java index a8ba7e840b4f5..7e9e2bd89774e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java @@ -25,7 +25,7 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.DoFnInfo; import com.google.cloud.dataflow.sdk.util.ExecutionContext; @@ -60,13 +60,13 @@ public static GroupAlsoByWindowsParDoFn create( CounterSet.AddCounterMutator addCounterMutator, StateSampler sampler /* unused */) throws Exception { - final Object windowingFn = + final Object windowFn = SerializableUtils.deserializeFromByteArray( getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), "serialized window fn"); - if (!(windowingFn instanceof WindowingFn)) { + if (!(windowFn instanceof WindowFn)) { throw new Exception( - "unexpected kind of WindowingFn: " + windowingFn.getClass().getName()); + "unexpected kind of WindowFn: " + windowFn.getClass().getName()); } byte[] serializedCombineFn = getBytes(cloudUserFn, PropertyNames.COMBINE_FN, null); @@ -97,7 +97,7 @@ public static GroupAlsoByWindowsParDoFn create( @Override public DoFnInfo createDoFnInfo() { return new DoFnInfo(StreamingGroupAlsoByWindowsDoFn.create( - (WindowingFn) windowingFn, + (WindowFn) windowFn, ((KvCoder) elemCoder).getValueCoder()), null); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java index 09e3d247aa6b0..001e63f2a1263 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java @@ -213,7 +213,7 @@ public void output(Receiver receiver, WindowedValue output) { sideOutputTags, stepContext, addCounterMutator, - fnFactory.createDoFnInfo().getWindowingFn()); + fnFactory.createDoFnInfo().getWindowFn()); fnRunner.startBundle(); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowingFnTestUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowFnTestUtils.java similarity index 84% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowingFnTestUtils.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowFnTestUtils.java index 687cb64530efa..0e14da6c4cefc 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowingFnTestUtils.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowFnTestUtils.java @@ -17,7 +17,7 @@ package com.google.cloud.dataflow.sdk.testing; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import org.joda.time.Instant; @@ -30,13 +30,13 @@ import java.util.Set; /** - * A utility class for testing {@link WindowingFn}s. + * A utility class for testing {@link WindowFn}s. */ -public class WindowingFnTestUtils { +public class WindowFnTestUtils { /** * Creates a Set of elements to be used as expected output in - * {@link #runWindowingFn}. + * {@link #runWindowFn}. */ public static Set set(long... timestamps) { Set result = new HashSet<>(); @@ -48,20 +48,20 @@ public static Set set(long... timestamps) { /** - * Runs the {@link WindowingFn} over the provided input, returning a map + * Runs the {@link WindowFn} over the provided input, returning a map * of windows to the timestamps in those windows. */ - public static Map> runWindowingFn( - WindowingFn windowingFn, + public static Map> runWindowFn( + WindowFn windowFn, List timestamps) throws Exception { final TestWindowSet windowSet = new TestWindowSet(); for (final Long timestamp : timestamps) { - for (W window : windowingFn.assignWindows( - new TestAssignContext(new Instant(timestamp), windowingFn))) { + for (W window : windowFn.assignWindows( + new TestAssignContext(new Instant(timestamp), windowFn))) { windowSet.put(window, timestampValue(timestamp)); } - windowingFn.mergeWindows(new TestMergeContext(windowSet, windowingFn)); + windowFn.mergeWindows(new TestMergeContext(windowSet, windowFn)); } Map> actual = new HashMap<>(); for (W window : windowSet.windows()) { @@ -78,11 +78,11 @@ private static String timestampValue(long timestamp) { * Test implementation of AssignContext. */ private static class TestAssignContext - extends WindowingFn.AssignContext { + extends WindowFn.AssignContext { private Instant timestamp; - public TestAssignContext(Instant timestamp, WindowingFn windowingFn) { - windowingFn.super(); + public TestAssignContext(Instant timestamp, WindowFn windowFn) { + windowFn.super(); this.timestamp = timestamp; } @@ -106,12 +106,12 @@ public Collection windows() { * Test implementation of MergeContext. */ private static class TestMergeContext - extends WindowingFn.MergeContext { + extends WindowFn.MergeContext { private TestWindowSet windowSet; public TestMergeContext( - TestWindowSet windowSet, WindowingFn windowingFn) { - windowingFn.super(); + TestWindowSet windowSet, WindowFn windowFn) { + windowFn.super(); this.windowSet = windowSet; } @@ -127,7 +127,7 @@ public void merge(Collection toBeMerged, W mergeResult) { } /** - * A WindowSet useful for testing WindowingFns which simply + * A WindowSet useful for testing WindowFns which simply * collects the placed elements into multisets. */ private static class TestWindowSet { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java index f27308b739589..eba6785b30162 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -90,7 +90,7 @@ public static Globally globally( *

Each output element is in the window by which its corresponding input * was grouped, and has the timestamp of the end of that window. The output * {@code PCollection} has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * as the input. * *

See {@link PerKey Combine.PerKey} for more information. @@ -111,7 +111,7 @@ public static PerKey perKey( *

Each output element is in the window by which its corresponding input * was grouped, and has the timestamp of the end of that window. The output * {@code PCollection} has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * as the input. * *

See {@link PerKey Combine.PerKey} for more information. @@ -132,7 +132,7 @@ public static PerKey perKey( *

Each output element is in the window by which its corresponding input * was grouped, and has the timestamp of the end of that window. The output * {@code PCollection} has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * as the input. * *

See {@link PerKey Combine.PerKey} for more information. @@ -154,7 +154,7 @@ public static PerKey perKey( *

Each output element has the same timestamp and is in the same window * as its corresponding input element, and the output * {@code PCollection} has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * associated with it as the input. * *

See {@link GroupedValues Combine.GroupedValues} for more information. @@ -180,7 +180,7 @@ public static GroupedValues groupedValues( *

Each output element has the same timestamp and is in the same window * as its corresponding input element, and the output * {@code PCollection} has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * associated with it as the input. * *

See {@link GroupedValues Combine.GroupedValues} for more information. @@ -206,7 +206,7 @@ public static GroupedValues groupedValues( *

Each output element has the same timestamp and is in the same window * as its corresponding input element, and the output * {@code PCollection} has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * associated with it as the input. * *

See {@link GroupedValues Combine.GroupedValues} for more information. @@ -770,7 +770,7 @@ public PCollection apply(PCollection input) { .apply(Combine.perKey(fn.asKeyedFn())) .apply(Values.create()); - if (input.getWindowingFn().isCompatible(new GlobalWindow())) { + if (input.getWindowFn().isCompatible(new GlobalWindow())) { return insertDefaultValueIfEmpty(output); } else { return output; @@ -892,7 +892,7 @@ public V extractOutput(List accumulator) { *

Each output element is in the window by which its corresponding input * was grouped, and has the timestamp of the end of that window. The output * {@code PCollection} has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * as the input. * * @param the type of the keys of the input and output @@ -970,7 +970,7 @@ protected String getKindString() { *

Each output element has the same timestamp and is in the same window * as its corresponding input element, and the output * {@code PCollection} has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * associated with it as the input. * *

See also {@link #globally}/{@link Globally Combine.Globally}, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java index 0550b2fff5f40..b6e4561f2dde4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java @@ -39,7 +39,7 @@ *

Each output element is in the window by which its corresponding input * was grouped, and has the timestamp of the end of that window. The output * {@code PCollection} has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * as the input. * *

Example of use: diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java index a62982ceff2db..8b8eb1cfab92b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java @@ -83,9 +83,9 @@ public abstract class Context { * *

If invoked from {@link #startBundle} or {@link #finishValue}, * this will attempt to use the - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * of the input {@code PCollection} to determine what windows the element - * should be in, throwing an exception if the {@code WindowingFn} attempts + * should be in, throwing an exception if the {@code WindowFn} attempts * to access any information about the input element. The output element * will have a timestamp of negative infinity. */ @@ -102,9 +102,9 @@ public abstract class Context { * *

If invoked from {@link #startBundle} or {@link #finishValue}, * this will attempt to use the - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * of the input {@code PCollection} to determine what windows the element - * should be in, throwing an exception if the {@code WindowingFn} attempts + * should be in, throwing an exception if the {@code WindowFn} attempts * to access any information about the input element except for the * timestamp. */ @@ -124,9 +124,9 @@ public abstract class Context { * *

If invoked from {@link #startBundle} or {@link #finishValue}, * this will attempt to use the - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * of the input {@code PCollection} to determine what windows the element - * should be in, throwing an exception if the {@code WindowingFn} attempts + * should be in, throwing an exception if the {@code WindowFn} attempts * to access any information about the input element. The output element * will have a timestamp of negative infinity. * @@ -147,9 +147,9 @@ public abstract class Context { * *

If invoked from {@link #startBundle} or {@link #finishValue}, * this will attempt to use the - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * of the input {@code PCollection} to determine what windows the element - * should be in, throwing an exception if the {@code WindowingFn} attempts + * should be in, throwing an exception if the {@code WindowFn} attempts * to access any information about the input element except for the * timestamp. * diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java index 3ef5e3a369a30..dd8efca82a240 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java @@ -20,7 +20,7 @@ import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionList; @@ -57,11 +57,11 @@ public class Flatten { * the {@link PCollection}s in its input. * *

If any of the inputs to {@code Flatten} require window merging, - * all inputs must have equal {@link WindowingFn}s. + * all inputs must have equal {@link WindowFn}s. * The output elements of {@code Flatten} are in the same windows and * have the same timestamps as their corresponding input elements. The output * {@code PCollection} will have the same - * {@link WindowingFn} as all of the inputs. + * {@link WindowFn} as all of the inputs. * * @param the type of the elements in the input and output * {@code PCollection}s. @@ -113,21 +113,21 @@ private FlattenPCollectionList() { } @Override public PCollection apply(PCollectionList inputs) { - WindowingFn windowingFn; + WindowFn windowFn; if (!getInput().getAll().isEmpty()) { - windowingFn = getInput().get(0).getWindowingFn(); + windowFn = getInput().get(0).getWindowFn(); for (PCollection input : getInput().getAll()) { - if (!windowingFn.isCompatible(input.getWindowingFn())) { + if (!windowFn.isCompatible(input.getWindowFn())) { throw new IllegalStateException( - "Inputs to Flatten had incompatible window windowingFns: " - + windowingFn + ", " + input.getWindowingFn()); + "Inputs to Flatten had incompatible window windowFns: " + + windowFn + ", " + input.getWindowFn()); } } } else { - windowingFn = new GlobalWindow(); + windowFn = new GlobalWindow(); } - return PCollection.createPrimitiveOutputInternal(windowingFn); + return PCollection.createPrimitiveOutputInternal(windowFn); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index c6d0acf4094d4..225fd18dfdabb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -26,9 +26,9 @@ import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindowingFn; -import com.google.cloud.dataflow.sdk.transforms.windowing.NonMergingWindowingFn; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindowFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.NonMergingWindowFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder; @@ -98,17 +98,17 @@ *

See {@link Combine.PerKey} for a common pattern of * {@code GroupByKey} followed by {@link Combine.GroupedValues}. * - *

When grouping, windows that can be merged according to the {@link WindowingFn} + *

When grouping, windows that can be merged according to the {@link WindowFn} * of the input {@code PCollection} will be merged together, and a group * corresponding to the new, merged window will be emitted. * The timestamp for each group is the upper bound of its window, e.g., the most * recent timestamp that can be assigned into the window, and the group will be * in the window that it corresponds to. The output {@code PCollection} will - * have the same {@link WindowingFn} as the input. + * have the same {@link WindowFn} as the input. * - *

If the {@link WindowingFn} of the input requires merging, it is not + *

If the {@link WindowFn} of the input requires merging, it is not * valid to apply another {@code GroupByKey} without first applying a new - * {@link WindowingFn}. + * {@link WindowFn}. * * @param the type of the keys of the input and output * {@code PCollection}s @@ -159,7 +159,7 @@ public PCollection>> apply( Coder keyCoder = inputKvCoder.getKeyCoder(); Coder inputValueCoder = inputKvCoder.getValueCoder(); Coder> outputValueCoder = FullWindowedValueCoder.of( - inputValueCoder, getInput().getWindowingFn().windowCoder()); + inputValueCoder, getInput().getWindowFn().windowCoder()); Coder>> outputKvCoder = KvCoder.of(keyCoder, outputValueCoder); return input.apply(ParDo.of( @@ -228,10 +228,10 @@ public int compare(WindowedValue e1, WindowedValue e2) { public static class GroupAlsoByWindow extends PTransform>>>, PCollection>>> { - private final WindowingFn windowingFn; + private final WindowFn windowFn; - public GroupAlsoByWindow(WindowingFn windowingFn) { - this.windowingFn = windowingFn; + public GroupAlsoByWindow(WindowFn windowFn) { + this.windowFn = windowFn; } @Override @@ -259,7 +259,7 @@ public PCollection>> apply( return input.apply(ParDo.of( new GroupAlsoByWindowsDoFn( - (WindowingFn) windowingFn, inputIterableElementValueCoder))) + (WindowFn) windowFn, inputIterableElementValueCoder))) .setCoder(outputKvCoder); } } @@ -282,16 +282,16 @@ public GroupByKeyOnly() { } @SuppressWarnings({"rawtypes", "unchecked"}) @Override public PCollection>> apply(PCollection> input) { - WindowingFn windowingFn = getInput().getWindowingFn(); - if (!(windowingFn instanceof NonMergingWindowingFn)) { + WindowFn windowFn = getInput().getWindowFn(); + if (!(windowFn instanceof NonMergingWindowFn)) { // Prevent merging windows again, without explicit user // involvement, e.g., by Window.into() or Window.remerge(). - windowingFn = new InvalidWindowingFn( - "WindowingFn has already been consumed by previous GroupByKey", - windowingFn); + windowFn = new InvalidWindowFn( + "WindowFn has already been consumed by previous GroupByKey", + windowFn); } return PCollection.>>createPrimitiveOutputInternal( - windowingFn); + windowFn); } @Override @@ -451,17 +451,17 @@ public PCollection>> applyHelper( // This operation groups by the combination of key and window, // merging windows as needed, using the windows assigned to the // key/value input elements and the window merge operation of the - // windowing function associated with the input PCollection. - WindowingFn windowingFn = getInput().getWindowingFn(); - if (windowingFn instanceof InvalidWindowingFn) { - String cause = ((InvalidWindowingFn) windowingFn).getCause(); + // window function associated with the input PCollection. + WindowFn windowFn = getInput().getWindowFn(); + if (windowFn instanceof InvalidWindowFn) { + String cause = ((InvalidWindowFn) windowFn).getCause(); throw new IllegalStateException( "GroupByKey must have a valid Window merge function. " + "Invalid because: " + cause); } - if (windowingFn.isCompatible(new GlobalWindow())) { + if (windowFn.isCompatible(new GlobalWindow())) { // The input PCollection is using the degenerate default - // windowing function, which uses a single global window for all + // window function, which uses a single global window for all // elements. We can implement this using a more-primitive // non-window-aware GBK transform. return input.apply(new GroupByKeyOnly()); @@ -491,7 +491,7 @@ public PCollection>> applyHelper( return gbkOutput // Group each key's values by window, merging windows as needed. - .apply(new GroupAlsoByWindow(windowingFn)); + .apply(new GroupAlsoByWindow(windowFn)); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Keys.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Keys.java index 0e78f4ad1e840..2b356ad6be549 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Keys.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Keys.java @@ -32,7 +32,7 @@ *

Each output element has the same timestamp and is in the same windows * as its corresponding input element, and the output {@code PCollection} * has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * associated with it as the input. * *

See also {@link Values}. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/KvSwap.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/KvSwap.java index 4b12d5db08c28..c898dff911f70 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/KvSwap.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/KvSwap.java @@ -34,7 +34,7 @@ *

Each output element has the same timestamp and is in the same windows * as its corresponding input element, and the output {@code PCollection} * has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * associated with it as the input. * * @param the type of the keys in the input {@code PCollection} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java index 0af3c3baf8971..f61b197bc58a3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java @@ -96,7 +96,7 @@ *

Each output element has the same timestamp and is in the same windows * as its corresponding input element, and the output {@code PCollection} * has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * associated with it as the input. * *

Naming {@code ParDo}s

@@ -708,7 +708,7 @@ public PCollection apply(PCollection input) { if (sideInputs == null) { sideInputs = Collections.emptyList(); } - return PCollection.createPrimitiveOutputInternal(getInput().getWindowingFn()) + return PCollection.createPrimitiveOutputInternal(getInput().getWindowFn()) .setTypeTokenInternal(fn.getOutputTypeToken()); } @@ -900,7 +900,7 @@ public BoundMulti withSideInputs( public PCollectionTuple apply(PCollection input) { PCollectionTuple outputs = PCollectionTuple.ofPrimitiveOutputsInternal( TupleTagList.of(mainOutputTag).and(sideOutputTags.getAll()), - getInput().getWindowingFn()); + getInput().getWindowFn()); // The fn will likely be an instance of an anonymous subclass // such as DoFn { }, thus will have a high-fidelity @@ -1040,7 +1040,7 @@ private static DoFnRunner evaluateHelper( sideOutputTags, executionContext.getStepContext(name), context.getAddCounterMutator(), - input.getWindowingFn()); + input.getWindowFn()); fnRunner.startBundle(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java index 7e5cc00f62a9f..e7051985c0598 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java @@ -55,7 +55,7 @@ *

Each output element has the same timestamp and is in the same windows * as its corresponding input element, and each output {@code PCollection} * has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * associated with it as the input. * * @param the type of the elements of the input and output diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java index 9540eda6da7d1..8d8eaf1e75811 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java @@ -37,7 +37,7 @@ *

Each output element is in the same window as its corresponding input * element, and has the timestamp of the end of that window. The output * {@code PCollection} has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * as the input. * *

Does not preserve any order the input PCollection might have had. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Values.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Values.java index 4ca84e45c6501..dad265f15211b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Values.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Values.java @@ -32,7 +32,7 @@ *

Each output element has the same timestamp and is in the same windows * as its corresponding input element, and the output {@code PCollection} * has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * associated with it as the input. * *

See also {@link Keys}. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java index 82ce93cdf3274..bd8415f2684aa 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java @@ -40,7 +40,7 @@ *

Each output element has the same timestamp and is in the same windows * as its corresponding input element, and the output {@code PCollection} * has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * associated with it as the input. * * @param the type of the keys in the output {@code PCollection} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java index cdbc1a64232b3..78b3d59ecdd71 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java @@ -26,7 +26,7 @@ import org.joda.time.Years; /** - * A collection of {@link WindowingFn}s that windows values into calendar-based + * A collection of {@link WindowFn}s that windows values into calendar-based * windows such as spans of days, months, or years. * *

For example, to group data into quarters that change on the 15th, use @@ -35,7 +35,7 @@ public class CalendarWindows { /** - * Returns a {@link WindowingFn} that windows elements into periods measured by days. + * Returns a {@link WindowFn} that windows elements into periods measured by days. * *

For example, {@code CalendarWindows.days(1)} will window elements into * separate windows for each day. @@ -45,7 +45,7 @@ public static DaysWindows days(int number) { } /** - * Returns a {@link WindowingFn} that windows elements into periods measured by weeks. + * Returns a {@link WindowFn} that windows elements into periods measured by weeks. * *

For example, {@code CalendarWindows.weeks(1, DateTimeConstants.TUESDAY)} will * window elements into week-long windows starting on Tuesdays. @@ -58,7 +58,7 @@ public static DaysWindows weeks(int number, int startDayOfWeek) { } /** - * Returns a {@link WindowingFn} that windows elements into periods measured by months. + * Returns a {@link WindowFn} that windows elements into periods measured by months. * *

For example, * {@code CalendarWindows.months(8).withStartingMonth(2014, 1).beginningOnDay(10)} @@ -70,7 +70,7 @@ public static MonthsWindows months(int number) { } /** - * Returns a {@link WindowingFn} that windows elements into periods measured by years. + * Returns a {@link WindowFn} that windows elements into periods measured by years. * *

For example, * {@code CalendarWindows.years(1).withTimeZone(DateTimeZone.forId("America/Los_Angeles"))} @@ -82,7 +82,7 @@ public static YearsWindows years(int number) { } /** - * A {@link WindowingFn} that windows elements into periods measured by days. + * A {@link WindowFn} that windows elements into periods measured by days. * *

By default, periods of multiple days are measured starting at the * epoch. This can be overridden with {@link #withStartingDay}. @@ -90,7 +90,7 @@ public static YearsWindows years(int number) { *

The time zone used to determine calendar boundaries is UTC, unless this * is overridden with the {@link #withTimeZone} method. */ - public static class DaysWindows extends PartitioningWindowingFn { + public static class DaysWindows extends PartitioningWindowFn { public DaysWindows withStartingDay(int year, int month, int day) { return new DaysWindows( @@ -132,7 +132,7 @@ public Coder windowCoder() { } @Override - public boolean isCompatible(WindowingFn other) { + public boolean isCompatible(WindowFn other) { if (!(other instanceof DaysWindows)) { return false; } @@ -144,7 +144,7 @@ public boolean isCompatible(WindowingFn other) { } /** - * A {@link WindowingFn} that windows elements into periods measured by months. + * A {@link WindowFn} that windows elements into periods measured by months. * *

By default, periods of multiple months are measured starting at the * epoch. This can be overridden with {@link #withStartingMonth}. @@ -155,7 +155,7 @@ public boolean isCompatible(WindowingFn other) { *

The time zone used to determine calendar boundaries is UTC, unless this * is overridden with the {@link #withTimeZone} method. */ - public static class MonthsWindows extends PartitioningWindowingFn { + public static class MonthsWindows extends PartitioningWindowFn { public MonthsWindows beginningOnDay(int dayOfMonth) { return new MonthsWindows( @@ -206,7 +206,7 @@ public Coder windowCoder() { } @Override - public boolean isCompatible(WindowingFn other) { + public boolean isCompatible(WindowFn other) { if (!(other instanceof MonthsWindows)) { return false; } @@ -219,7 +219,7 @@ public boolean isCompatible(WindowingFn other) { } /** - * A {@link WindowingFn} that windows elements into periods measured by years. + * A {@link WindowFn} that windows elements into periods measured by years. * *

By default, periods of multiple years are measured starting at the * epoch. This can be overridden with {@link #withStartingYear}. @@ -230,7 +230,7 @@ public boolean isCompatible(WindowingFn other) { *

The time zone used to determine calendar boundaries is UTC, unless this * is overridden with the {@link #withTimeZone} method. */ - public static class YearsWindows extends PartitioningWindowingFn { + public static class YearsWindows extends PartitioningWindowFn { public YearsWindows beginningOnDay(int monthOfYear, int dayOfMonth) { return new YearsWindows( @@ -285,7 +285,7 @@ public Coder windowCoder() { } @Override - public boolean isCompatible(WindowingFn other) { + public boolean isCompatible(WindowFn other) { if (!(other instanceof YearsWindows)) { return false; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java index d381a80627b73..138d82399342a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java @@ -22,7 +22,7 @@ import org.joda.time.Instant; /** - * A {@link WindowingFn} that windows values into fixed-size timestamp-based windows. + * A {@link WindowFn} that windows values into fixed-size timestamp-based windows. * *

For example, in order to partition the data into 10 minute windows: *

 {@code
@@ -32,7 +32,7 @@
  * } 
*/ @SuppressWarnings("serial") -public class FixedWindows extends PartitioningWindowingFn { +public class FixedWindows extends PartitioningWindowFn { /** * Size of this window. @@ -86,7 +86,7 @@ public Coder windowCoder() { } @Override - public boolean isCompatible(WindowingFn other) { + public boolean isCompatible(WindowFn other) { return (other instanceof FixedWindows) && (size.equals(((FixedWindows) other).size)) && (offset.equals(((FixedWindows) other).offset)); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java index 507b1cc860783..e3298fbb3607c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java @@ -27,18 +27,18 @@ import java.util.Collection; /** - * Default {@link WindowingFn} where all data is in the same bucket. + * Default {@link WindowFn} where all data is in the same bucket. */ @SuppressWarnings("serial") public class GlobalWindow - extends NonMergingWindowingFn { + extends NonMergingWindowFn { @Override public Collection assignWindows(AssignContext c) { return Arrays.asList(Window.INSTANCE); } @Override - public boolean isCompatible(WindowingFn o) { + public boolean isCompatible(WindowFn o) { return o instanceof GlobalWindow; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowingFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowFn.java similarity index 58% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowingFn.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowFn.java index 53dc93a213945..3a17f4a4cbb9a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowingFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowFn.java @@ -21,31 +21,31 @@ import java.util.Collection; /** - * A {@link WindowingFn} that represents an invalid pipeline state. + * A {@link WindowFn} that represents an invalid pipeline state. * * @param window type */ -public class InvalidWindowingFn extends WindowingFn { +public class InvalidWindowFn extends WindowFn { private String cause; - private WindowingFn originalWindowingFn; + private WindowFn originalWindowFn; - public InvalidWindowingFn(String cause, WindowingFn originalWindowingFn) { - this.originalWindowingFn = originalWindowingFn; + public InvalidWindowFn(String cause, WindowFn originalWindowFn) { + this.originalWindowFn = originalWindowFn; this.cause = cause; } /** - * Returns the reason that this {@code WindowingFn} is invalid. + * Returns the reason that this {@code WindowFn} is invalid. */ public String getCause() { return cause; } /** - * Returns the original windowingFn that this InvalidWindowingFn replaced. + * Returns the original windowFn that this InvalidWindowFn replaced. */ - public WindowingFn getOriginalWindowingFn() { - return originalWindowingFn; + public WindowFn getOriginalWindowFn() { + return originalWindowFn; } @Override @@ -60,16 +60,16 @@ public void mergeWindows(MergeContext c) { @Override public Coder windowCoder() { - return originalWindowingFn.windowCoder(); + return originalWindowFn.windowCoder(); } /** - * {@code InvalidWindowingFn} objects with the same {@code originalWindowingFn} are compatible. + * {@code InvalidWindowFn} objects with the same {@code originalWindowFn} are compatible. */ @Override - public boolean isCompatible(WindowingFn other) { + public boolean isCompatible(WindowFn other) { return getClass() == other.getClass() - && getOriginalWindowingFn().isCompatible( - ((InvalidWindowingFn) other).getOriginalWindowingFn()); + && getOriginalWindowFn().isCompatible( + ((InvalidWindowFn) other).getOriginalWindowFn()); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java index 4d4dd8492684d..e90f8f0897e6d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java @@ -22,14 +22,14 @@ import java.util.List; /** - * A {@link WindowingFn} that merges overlapping {@link IntervalWindow}s. + * A {@link WindowFn} that merges overlapping {@link IntervalWindow}s. */ public class MergeOverlappingIntervalWindows { /** * Merge overlapping intervals. */ - public static void mergeWindows(WindowingFn.MergeContext c) throws Exception { + public static void mergeWindows(WindowFn.MergeContext c) throws Exception { // Merge any overlapping windows into a single window. // Sort the list of existing windows so we only have to // traverse the list once rather than considering all @@ -72,7 +72,7 @@ public void add(IntervalWindow window) { union = union == null ? window : union.span(window); parts.add(window); } - public void apply(WindowingFn.MergeContext c) throws Exception { + public void apply(WindowFn.MergeContext c) throws Exception { if (parts.size() > 1) { c.merge(parts, union); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowingFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowFn.java similarity index 80% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowingFn.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowFn.java index ffeea996d60dc..3cce009e9e728 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowingFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowFn.java @@ -17,14 +17,14 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; /** - * Abstract base class for {@link WindowingFn}s that do not merge windows. + * Abstract base class for {@link WindowFn}s that do not merge windows. * * @param type of elements being windowed * @param {@link BoundedWindow} subclass used to represent the windows used by this - * {@code WindowingFn} + * {@code WindowFn} */ -public abstract class NonMergingWindowingFn - extends WindowingFn { +public abstract class NonMergingWindowFn + extends WindowFn { @Override public final void mergeWindows(MergeContext c) { } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowingFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowFn.java similarity index 86% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowingFn.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowFn.java index 6a65ba134f181..7b9d8d025f760 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowingFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowFn.java @@ -22,14 +22,14 @@ import java.util.Collection; /** - * A {@link WindowingFn} that places each value into exactly one window + * A {@link WindowFn} that places each value into exactly one window * based on its timestamp and never merges windows. * * @param type of elements being windowed * @param window type */ -public abstract class PartitioningWindowingFn - extends NonMergingWindowingFn { +public abstract class PartitioningWindowFn + extends NonMergingWindowFn { /** * Returns the single window to which elements with this timestamp belong. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java index 26744a549ba2a..ff936df120084 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java @@ -24,7 +24,7 @@ import java.util.Collection; /** - * A WindowingFn windowing values into sessions separated by {@link #gapDuration}-long + * A WindowFn windowing values into sessions separated by {@link #gapDuration}-long * periods with no elements. * *

For example, in order to window data into session with at least 10 minute @@ -35,7 +35,7 @@ * Window.by(Sessions.withGapDuration(Duration.standardMinutes(10)))); * } */ -public class Sessions extends WindowingFn { +public class Sessions extends WindowFn { /** * Duration of the gaps between sessions. @@ -43,14 +43,14 @@ public class Sessions extends WindowingFn { private final Duration gapDuration; /** - * Creates a {@code Sessions} {@link WindowingFn} with the specified gap duration. + * Creates a {@code Sessions} {@link WindowFn} with the specified gap duration. */ public static Sessions withGapDuration(Duration gapDuration) { return new Sessions(gapDuration); } /** - * Creates a {@code Sessions} {@link WindowingFn} with the specified gap duration. + * Creates a {@code Sessions} {@link WindowFn} with the specified gap duration. */ private Sessions(Duration gapDuration) { this.gapDuration = gapDuration; @@ -75,7 +75,7 @@ public Coder windowCoder() { } @Override - public boolean isCompatible(WindowingFn other) { + public boolean isCompatible(WindowFn other) { return other instanceof Sessions; } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java index 45b90dd33d1e4..d29062e31ef2e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java @@ -26,7 +26,7 @@ import java.util.List; /** - * A WindowingFn that windows values into possibly overlapping fixed-size + * A WindowFn that windows values into possibly overlapping fixed-size * timestamp-based windows. * *

For example, in order to window data into 10 minute windows that @@ -38,7 +38,7 @@ * } */ @SuppressWarnings("serial") -public class SlidingWindows extends NonMergingWindowingFn { +public class SlidingWindows extends NonMergingWindowFn { /** * Amount of time between generated windows. @@ -119,7 +119,7 @@ public Collection assignWindows(AssignContext c) { } @Override - public boolean isCompatible(WindowingFn other) { + public boolean isCompatible(WindowFn other) { if (other instanceof SlidingWindows) { SlidingWindows that = (SlidingWindows) other; return period.equals(that.period) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java index 702476362614b..28956b1360989 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java @@ -41,7 +41,7 @@ /** * {@code Window} logically divides up or groups the elements of a - * {@link PCollection} into finite windows according to a {@link WindowingFn}. + * {@link PCollection} into finite windows according to a {@link WindowFn}. * The output of {@code Window} contains the same elements as input, but they * have been logically assigned to windows. The next * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}s, including one @@ -80,7 +80,7 @@ * {(KV("foo", 2), 1m), (KV("bar", 1), 1m), (KV("foo", 1), 2m)} * * - *

Several predefined {@link WindowingFn}s are provided: + *

Several predefined {@link WindowFn}s are provided: *

    *
  • {@link FixedWindows} partitions the timestamps into fixed-width intervals. *
  • {@link SlidingWindows} places data into overlapping fixed-width intervals. @@ -88,8 +88,8 @@ * is separated from the next by no more than a specified gap. *
* - * Additionally, custom {@link WindowingFn}s can be created, by creating new - * subclasses of {@link WindowingFn}. + * Additionally, custom {@link WindowFn}s can be created, by creating new + * subclasses of {@link WindowFn}. */ public class Window { /** @@ -100,7 +100,7 @@ public class Window { * *

The resulting {@code PTransform} is incomplete, and its input/output * type is not yet bound. Use {@link Window.Unbound#into} to specify the - * {@link WindowingFn} to use, which will also bind the input/output type of this + * {@link WindowFn} to use, which will also bind the input/output type of this * {@code PTransform}. */ public static Unbound named(String name) { @@ -109,14 +109,14 @@ public static Unbound named(String name) { /** * Creates a {@code Window} {@code PTransform} that uses the given - * {@link WindowingFn} to window the data. + * {@link WindowFn} to window the data. * *

The resulting {@code PTransform}'s types have been bound, with both the * input and output being a {@code PCollection}, inferred from the types of - * the argument {@code WindowingFn}. It is ready to be applied, or further + * the argument {@code WindowFn}. It is ready to be applied, or further * properties can be set on it first. */ - public static Bound into(WindowingFn fn) { + public static Bound into(WindowFn fn) { return new Unbound().into(fn); } @@ -124,7 +124,7 @@ public static Bound into(WindowingFn fn) { * An incomplete {@code Window} transform, with unbound input/output type. * *

Before being applied, {@link Window.Unbound#into} must be - * invoked to specify the {@link WindowingFn} to invoke, which will also + * invoked to specify the {@link WindowFn} to invoke, which will also * bind the input/output type of this {@code PTransform}. */ public static class Unbound { @@ -151,27 +151,27 @@ public Unbound named(String name) { /** * Returns a new {@code Window} {@code PTransform} that's like this - * transform but which will use the given {@link WindowingFn}, and which has + * transform but which will use the given {@link WindowFn}, and which has * its input and output types bound. Does not modify this transform. The * resulting {@code PTransform} is sufficiently specified to be applied, * but more properties can still be specified. */ - public Bound into(WindowingFn fn) { + public Bound into(WindowFn fn) { return new Bound<>(name, fn); } } /** * A {@code PTransform} that windows the elements of a {@code PCollection}, - * into finite windows according to a user-specified {@code WindowingFn}. + * into finite windows according to a user-specified {@code WindowFn}. * * @param The type of elements this {@code Window} is applied to */ @SuppressWarnings("serial") public static class Bound extends PTransform, PCollection> { - WindowingFn fn; + WindowFn fn; - Bound(String name, WindowingFn fn) { + Bound(String name, WindowFn fn) { this.name = name; this.fn = fn; } @@ -226,17 +226,17 @@ public static Remerge remerge() { public static class Remerge extends PTransform, PCollection> { @Override public PCollection apply(PCollection input) { - WindowingFn windowingFn = getInput().getWindowingFn(); - WindowingFn outputWindowingFn = - (windowingFn instanceof InvalidWindowingFn) - ? ((InvalidWindowingFn) windowingFn).getOriginalWindowingFn() - : windowingFn; + WindowFn windowFn = getInput().getWindowFn(); + WindowFn outputWindowFn = + (windowFn instanceof InvalidWindowFn) + ? ((InvalidWindowFn) windowFn).getOriginalWindowFn() + : windowFn; return input.apply(ParDo.named("Identity").of(new DoFn() { @Override public void processElement(ProcessContext c) { c.output(c.element()); } - })).setWindowingFnInternal(outputWindowingFn); + })).setWindowFnInternal(outputWindowFn); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java similarity index 87% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingFn.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java index 935deb09f2b7c..ed5cdb30dac64 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java @@ -26,19 +26,19 @@ /** * The argument to the {@link Window} transform used to assign elements into * windows and to determine how windows are merged. See {@link Window} for more - * information on how {@code WindowingFn}s are used and for a library of - * predefined {@code WindowingFn}s. + * information on how {@code WindowFn}s are used and for a library of + * predefined {@code WindowFn}s. * *

Users will generally want to use the predefined - * {@code WindowingFn}s, but it is also possible to create new + * {@code WindowFn}s, but it is also possible to create new * subclasses. - * TODO: Describe how to properly create {@code WindowingFn}s. + * TODO: Describe how to properly create {@code WindowFn}s. * * @param type of elements being windowed * @param {@link BoundedWindow} subclass used to represent the - * windows used by this {@code WindowingFn} + * windows used by this {@code WindowFn} */ -public abstract class WindowingFn +public abstract class WindowFn implements Serializable { /** @@ -105,13 +105,13 @@ public abstract void merge(Collection toBeMerged, W mergeResult) /** * Returns whether this performs the same merging as the given - * {@code WindowingFn}. + * {@code WindowFn}. */ - public abstract boolean isCompatible(WindowingFn other); + public abstract boolean isCompatible(WindowFn other); /** * Returns the {@link Coder} used for serializing the windows used - * by this windowingFn. + * by this windowFn. */ public abstract Coder windowCoder(); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/package-info.java index cab217d3e9698..eb6ecb3af3833 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/package-info.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/package-info.java @@ -20,7 +20,7 @@ * *

{@code Window} logically divides up or groups the elements of a * {@link com.google.cloud.dataflow.sdk.values.PCollection} into finite windows according to a - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn}. + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn}. * The output of {@code Window} contains the same elements as input, but they * have been logically assigned to windows. The next * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}s, including one diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AbstractWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AbstractWindowSet.java index fa54eee605b9c..9e8306bc25946 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AbstractWindowSet.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AbstractWindowSet.java @@ -18,7 +18,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.values.KV; import java.util.Arrays; @@ -56,14 +56,14 @@ interface ActiveWindowManager { * Wrapper around AbstractWindowSet that provides the MergeContext interface. */ static class WindowMergeContext - extends WindowingFn.MergeContext { + extends WindowFn.MergeContext { private final AbstractWindowSet windowSet; @SuppressWarnings("unchecked") public WindowMergeContext( AbstractWindowSet windowSet, - WindowingFn windowingFn) { - ((WindowingFn) windowingFn).super(); + WindowFn windowFn) { + ((WindowFn) windowFn).super(); this.windowSet = windowSet; } @@ -77,19 +77,19 @@ public WindowMergeContext( } protected final K key; - protected final WindowingFn windowingFn; + protected final WindowFn windowFn; protected final Coder inputCoder; protected final DoFnProcessContext> context; protected final ActiveWindowManager activeWindowManager; protected AbstractWindowSet( K key, - WindowingFn windowingFn, + WindowFn windowFn, Coder inputCoder, DoFnProcessContext> context, ActiveWindowManager activeWindowManager) { this.key = key; - this.windowingFn = windowingFn; + this.windowFn = windowFn; this.inputCoder = inputCoder; this.context = context; this.activeWindowManager = activeWindowManager; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AssignWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AssignWindowsDoFn.java index a0b91b53037d9..27066f8091079 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AssignWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AssignWindowsDoFn.java @@ -18,7 +18,7 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import org.joda.time.Instant; @@ -26,15 +26,15 @@ /** * {@link DoFn} that tags elements of a PCollection with windows, according - * to the provided {@link WindowingFn}. + * to the provided {@link WindowFn}. * @param Type of elements being windowed * @param Window type */ @SuppressWarnings("serial") public class AssignWindowsDoFn extends DoFn { - private WindowingFn fn; + private WindowFn fn; - public AssignWindowsDoFn(WindowingFn fn) { + public AssignWindowsDoFn(WindowFn fn) { this.fn = fn; } @@ -43,8 +43,8 @@ public AssignWindowsDoFn(WindowingFn fn) { public void processElement(ProcessContext c) throws Exception { final DoFnProcessContext context = (DoFnProcessContext) c; Collection windows = - ((WindowingFn) fn).assignWindows( - ((WindowingFn) fn).new AssignContext() { + ((WindowFn) fn).assignWindows( + ((WindowFn) fn).new AssignContext() { @Override public T element() { return context.element(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java index 957434c154421..33cfe80a9231e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java @@ -22,7 +22,7 @@ import com.google.cloud.dataflow.sdk.coders.MapCoder; import com.google.cloud.dataflow.sdk.coders.SetCoder; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.values.CodedTupleTag; import com.google.cloud.dataflow.sdk.values.KV; @@ -49,8 +49,8 @@ class BufferingWindowSet CodedTupleTag.of( "mergeTree", MapCoder.of( - windowingFn.windowCoder(), - SetCoder.of(windowingFn.windowCoder()))); + windowFn.windowCoder(), + SetCoder.of(windowFn.windowCoder()))); /** * A map of live windows to windows that were merged into them. @@ -70,11 +70,11 @@ class BufferingWindowSet protected BufferingWindowSet( K key, - WindowingFn windowingFn, + WindowFn windowFn, Coder inputCoder, DoFnProcessContext>> context, ActiveWindowManager activeWindowManager) throws Exception { - super(key, windowingFn, inputCoder, context, activeWindowManager); + super(key, windowFn, inputCoder, context, activeWindowManager); mergeTree = emptyIfNull( context.context.stepContext.lookup(Arrays.asList(mergeTreeTag)) @@ -86,7 +86,7 @@ protected BufferingWindowSet( @Override public void put(W window, V value) throws Exception { context.context.stepContext.writeToTagList( - bufferTag(window, windowingFn.windowCoder(), inputCoder), + bufferTag(window, windowFn.windowCoder(), inputCoder), value, context.timestamp()); if (!mergeTree.containsKey(window)) { @@ -152,12 +152,12 @@ protected Iterable finalValue(W window) throws Exception { for (W curWindow : curWindows) { Iterable items = context.context.stepContext.readTagList(bufferTag( - curWindow, windowingFn.windowCoder(), inputCoder)); + curWindow, windowFn.windowCoder(), inputCoder)); for (V item : items) { toEmit.add(item); } context.context.stepContext.deleteTagList(bufferTag( - curWindow, windowingFn.windowCoder(), inputCoder)); + curWindow, windowFn.windowCoder(), inputCoder)); } return toEmit; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java index 115b3f15f5540..a900efbbfef67 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java @@ -22,8 +22,8 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn.AssignContext; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn.AssignContext; import com.google.cloud.dataflow.sdk.util.DoFnRunner.OutputManager; import com.google.cloud.dataflow.sdk.util.ExecutionContext.StepContext; import com.google.cloud.dataflow.sdk.util.common.CounterSet; @@ -56,7 +56,7 @@ class DoFnContext extends DoFn.Context { final TupleTag mainOutputTag; final StepContext stepContext; final CounterSet.AddCounterMutator addCounterMutator; - final WindowingFn windowingFn; + final WindowFn windowFn; public DoFnContext(PipelineOptions options, DoFn fn, @@ -66,7 +66,7 @@ public DoFnContext(PipelineOptions options, List> sideOutputTags, StepContext stepContext, CounterSet.AddCounterMutator addCounterMutator, - WindowingFn windowingFn) { + WindowFn windowFn) { fn.super(); this.options = options; this.fn = fn; @@ -80,7 +80,7 @@ public DoFnContext(PipelineOptions options, } this.stepContext = stepContext; this.addCounterMutator = addCounterMutator; - this.windowingFn = windowingFn; + this.windowFn = windowFn; } public R getReceiver(TupleTag tag) { @@ -122,18 +122,18 @@ WindowedValue makeWindowedValue( if (windows == null) { try { - windows = windowingFn.assignWindows(windowingFn.new AssignContext() { + windows = windowFn.assignWindows(windowFn.new AssignContext() { @Override public Object element() { throw new UnsupportedOperationException( - "WindowingFn attemped to access input element when none was available"); + "WindowFn attemped to access input element when none was available"); } @Override public Instant timestamp() { if (inputTimestamp == null) { throw new UnsupportedOperationException( - "WindowingFn attemped to access input timestamp when none was available"); + "WindowFn attemped to access input timestamp when none was available"); } return inputTimestamp; } @@ -141,7 +141,7 @@ public Instant timestamp() { @Override public Collection windows() { throw new UnsupportedOperationException( - "WindowingFn attemped to access input windows when none were available"); + "WindowFn attemped to access input windows when none were available"); } }); } catch (Exception e) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnInfo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnInfo.java index 1af14d7a130ea..9fdb9890fb98f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnInfo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnInfo.java @@ -17,7 +17,7 @@ package com.google.cloud.dataflow.sdk.util; import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import java.io.Serializable; @@ -30,18 +30,18 @@ public class DoFnInfo implements Serializable { private static final long serialVersionUID = 0; private DoFn doFn; - private WindowingFn windowingFn; + private WindowFn windowFn; - public DoFnInfo(DoFn doFn, WindowingFn windowingFn) { + public DoFnInfo(DoFn doFn, WindowFn windowFn) { this.doFn = doFn; - this.windowingFn = windowingFn; + this.windowFn = windowFn; } public DoFn getDoFn() { return doFn; } - public WindowingFn getWindowingFn() { - return windowingFn; + public WindowFn getWindowFn() { + return windowFn; } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java index e8ac2486da6e0..02c80be035278 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java @@ -18,7 +18,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.ExecutionContext.StepContext; import com.google.cloud.dataflow.sdk.util.common.CounterSet; import com.google.cloud.dataflow.sdk.values.TupleTag; @@ -61,11 +61,11 @@ private DoFnRunner(PipelineOptions options, List> sideOutputTags, StepContext stepContext, CounterSet.AddCounterMutator addCounterMutator, - WindowingFn windowingFn) { + WindowFn windowFn) { this.fn = fn; this.context = new DoFnContext<>(options, fn, sideInputs, outputManager, mainOutputTag, sideOutputTags, stepContext, - addCounterMutator, windowingFn); + addCounterMutator, windowFn); } public static DoFnRunner create( @@ -77,10 +77,10 @@ public static DoFnRunner create( List> sideOutputTags, StepContext stepContext, CounterSet.AddCounterMutator addCounterMutator, - WindowingFn windowingFn) { + WindowFn windowFn) { return new DoFnRunner<>( options, fn, sideInputs, outputManager, - mainOutputTag, sideOutputTags, stepContext, addCounterMutator, windowingFn); + mainOutputTag, sideOutputTags, stepContext, addCounterMutator, windowFn); } @SuppressWarnings({"rawtypes", "unchecked"}) @@ -92,7 +92,7 @@ public static DoFnRunner createWithListOutputs( List> sideOutputTags, StepContext stepContext, CounterSet.AddCounterMutator addCounterMutator, - WindowingFn windowingFn) { + WindowFn windowFn) { return create( options, fn, sideInputs, new OutputManager() { @@ -105,7 +105,7 @@ public void output(List list, WindowedValue output) { list.add(output); } }, - mainOutputTag, sideOutputTags, stepContext, addCounterMutator, windowingFn); + mainOutputTag, sideOutputTags, stepContext, addCounterMutator, windowFn); } /** Calls {@link DoFn#startBundle}. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java index f4bd28039686f..26297d0eba974 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java @@ -19,8 +19,8 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.NonMergingWindowingFn; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.NonMergingWindowFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.common.PeekingReiterator; import com.google.cloud.dataflow.sdk.util.common.Reiterable; import com.google.cloud.dataflow.sdk.util.common.Reiterator; @@ -52,13 +52,13 @@ public class GroupAlsoByWindowsDoFn extends DoFn>>, KV>> { // TODO: Add back RequiresKeyed state once that is supported. - protected WindowingFn windowingFn; + protected WindowFn windowFn; protected Coder inputCoder; public GroupAlsoByWindowsDoFn( - WindowingFn windowingFn, + WindowFn windowFn, Coder inputCoder) { - this.windowingFn = windowingFn; + this.windowFn = windowFn; this.inputCoder = inputCoder; } @@ -67,7 +67,7 @@ public void processElement(ProcessContext processContext) throws Exception { DoFnProcessContext>>, KV>> context = (DoFnProcessContext>>, KV>>) processContext; - if (windowingFn instanceof NonMergingWindowingFn) { + if (windowFn instanceof NonMergingWindowFn) { processElementViaIterators(context); } else { processElementViaWindowSet(context); @@ -81,19 +81,19 @@ private void processElementViaWindowSet( K key = context.element().getKey(); BatchActiveWindowManager activeWindowManager = new BatchActiveWindowManager<>(); AbstractWindowSet, W> windowSet = - new BufferingWindowSet(key, windowingFn, inputCoder, context, activeWindowManager); + new BufferingWindowSet(key, windowFn, inputCoder, context, activeWindowManager); for (WindowedValue e : context.element().getValue()) { for (BoundedWindow window : e.getWindows()) { windowSet.put((W) window, e.getValue()); } - ((WindowingFn) windowingFn) - .mergeWindows(new AbstractWindowSet.WindowMergeContext(windowSet, windowingFn)); + ((WindowFn) windowFn) + .mergeWindows(new AbstractWindowSet.WindowMergeContext(windowSet, windowFn)); - maybeOutputWindows(activeWindowManager, windowSet, windowingFn, e.getTimestamp()); + maybeOutputWindows(activeWindowManager, windowSet, windowFn, e.getTimestamp()); } - maybeOutputWindows(activeWindowManager, windowSet, windowingFn, null); + maybeOutputWindows(activeWindowManager, windowSet, windowFn, null); windowSet.flush(); } @@ -105,15 +105,15 @@ private void processElementViaWindowSet( private void maybeOutputWindows( BatchActiveWindowManager activeWindowManager, AbstractWindowSet windowSet, - WindowingFn windowingFn, + WindowFn windowFn, Instant nextTimestamp) throws Exception { if (activeWindowManager.hasMoreWindows() && (nextTimestamp == null || activeWindowManager.nextTimestamp().isBefore(nextTimestamp))) { // There is at least one window ready to emit. Merge now in case that window should be merged // into a not yet completed one. - ((WindowingFn) windowingFn) - .mergeWindows(new AbstractWindowSet.WindowMergeContext(windowSet, windowingFn)); + ((WindowFn) windowFn) + .mergeWindows(new AbstractWindowSet.WindowMergeContext(windowSet, windowFn)); } while (activeWindowManager.hasMoreWindows() diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java index 96b2ece5cf987..f2b96c10434ec 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java @@ -20,7 +20,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.values.CodedTupleTag; import com.google.cloud.dataflow.sdk.values.KV; @@ -38,17 +38,17 @@ class PartitionBufferingWindowSet extends AbstractWindowSet, W> { PartitionBufferingWindowSet( K key, - WindowingFn windowingFn, + WindowFn windowFn, Coder inputCoder, DoFnProcessContext>> context, ActiveWindowManager activeWindowManager) { - super(key, windowingFn, inputCoder, context, activeWindowManager); + super(key, windowFn, inputCoder, context, activeWindowManager); } @Override public void put(W window, V value) throws Exception { context.context.stepContext.writeToTagList( - bufferTag(window, windowingFn.windowCoder(), inputCoder), value, context.timestamp()); + bufferTag(window, windowFn.windowCoder(), inputCoder), value, context.timestamp()); // Adds the window even if it is already present, relying on the streaming backend to // de-deduplicate. activeWindowManager.addWindow(window); @@ -56,7 +56,7 @@ public void put(W window, V value) throws Exception { @Override public void remove(W window) throws Exception { - CodedTupleTag tag = bufferTag(window, windowingFn.windowCoder(), inputCoder); + CodedTupleTag tag = bufferTag(window, windowFn.windowCoder(), inputCoder); context.context.stepContext.deleteTagList(tag); } @@ -77,7 +77,7 @@ public boolean contains(W window) { @Override protected Iterable finalValue(W window) throws Exception { - CodedTupleTag tag = bufferTag(window, windowingFn.windowCoder(), inputCoder); + CodedTupleTag tag = bufferTag(window, windowFn.windowCoder(), inputCoder); Iterable result = context.context.stepContext.readTagList(tag); if (result == null) { throw new IllegalStateException("finalValue called for non-existent window"); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java index 61911a9decd4c..3496bef97953f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java @@ -19,8 +19,8 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.PartitioningWindowingFn; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.PartitioningWindowFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.values.KV; import java.io.IOException; @@ -37,32 +37,32 @@ public class StreamingGroupAlsoByWindowsDoFn extends DoFn>, KV> implements DoFn.RequiresKeyedState { - protected WindowingFn windowingFn; + protected WindowFn windowFn; protected Coder inputCoder; protected StreamingGroupAlsoByWindowsDoFn( - WindowingFn windowingFn, + WindowFn windowFn, Coder inputCoder) { - this.windowingFn = windowingFn; + this.windowFn = windowFn; this.inputCoder = inputCoder; } public static StreamingGroupAlsoByWindowsDoFn create( - WindowingFn windowingFn, + WindowFn windowFn, Coder inputCoder) { - return new StreamingGroupAlsoByWindowsDoFn<>(windowingFn, inputCoder); + return new StreamingGroupAlsoByWindowsDoFn<>(windowFn, inputCoder); } private AbstractWindowSet createWindowSet( K key, DoFnProcessContext> context, AbstractWindowSet.ActiveWindowManager activeWindowManager) throws Exception { - if (windowingFn instanceof PartitioningWindowingFn) { + if (windowFn instanceof PartitioningWindowFn) { return new PartitionBufferingWindowSet( - key, windowingFn, inputCoder, context, activeWindowManager); + key, windowFn, inputCoder, context, activeWindowManager); } else { - return new BufferingWindowSet(key, windowingFn, inputCoder, context, activeWindowManager); + return new BufferingWindowSet(key, windowFn, inputCoder, context, activeWindowManager); } } @@ -75,7 +75,7 @@ public void processElement(ProcessContext processContext) throws Exception { K key = element.getKey(); VI value = element.getValue(); AbstractWindowSet windowSet = createWindowSet( - key, context, new StreamingActiveWindowManager<>(context, windowingFn.windowCoder())); + key, context, new StreamingActiveWindowManager<>(context, windowFn.windowCoder())); for (BoundedWindow window : context.windows()) { windowSet.put((W) window, value); @@ -86,14 +86,14 @@ public void processElement(ProcessContext processContext) throws Exception { TimerOrElement timer = context.element(); AbstractWindowSet windowSet = createWindowSet( (K) timer.key(), context, new StreamingActiveWindowManager<>( - context, windowingFn.windowCoder())); + context, windowFn.windowCoder())); // Attempt to merge windows before emitting; that may remove the current window under // consideration. - ((WindowingFn) windowingFn) - .mergeWindows(new AbstractWindowSet.WindowMergeContext(windowSet, windowingFn)); + ((WindowFn) windowFn) + .mergeWindows(new AbstractWindowSet.WindowMergeContext(windowSet, windowFn)); - W window = WindowUtils.windowFromString(timer.tag(), windowingFn.windowCoder()); + W window = WindowUtils.windowFromString(timer.tag(), windowFn.windowCoder()); boolean windowExists; try { windowExists = windowSet.contains(window); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java index e9a6cbde3773f..12e920d12cd0b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java @@ -19,7 +19,7 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.common.reflect.TypeToken; /** @@ -51,7 +51,7 @@ * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#outputWithTimestamp}. * *

Additionally, a {@code PCollection} has an associated - * {@link WindowingFn} and each element is assigned to a set of windows. + * {@link WindowFn} and each element is assigned to a set of windows. * By default, the windowing function is * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow} * and all elements are assigned into a single default window. @@ -166,10 +166,10 @@ public Output apply(PTransform, } /** - * Returns the {@link WindowingFn} of this {@code PCollection}. + * Returns the {@link WindowFn} of this {@code PCollection}. */ - public WindowingFn getWindowingFn() { - return windowingFn; + public WindowFn getWindowFn() { + return windowFn; } ///////////////////////////////////////////////////////////////////////////// @@ -183,13 +183,13 @@ public Output apply(PTransform, private boolean isOrdered = false; /** - * {@link WindowingFn} that will be used to merge windows in + * {@link WindowFn} that will be used to merge windows in * this {@code PCollection} and subsequent {@code PCollection}s produced * from this one. * *

By default, no merging is performed. */ - private WindowingFn windowingFn; + private WindowFn windowFn; private PCollection() {} @@ -206,12 +206,12 @@ public PCollection setTypeTokenInternal(TypeToken typeToken) { } /** - * Sets the {@link WindowingFn} of this {@code PCollection}. + * Sets the {@link WindowFn} of this {@code PCollection}. * *

For use by primitive transformations only. */ - public PCollection setWindowingFnInternal(WindowingFn windowingFn) { - this.windowingFn = windowingFn; + public PCollection setWindowFnInternal(WindowFn windowFn) { + this.windowFn = windowFn; return this; } @@ -232,7 +232,7 @@ public PCollection setPipelineInternal(Pipeline pipeline) { *

For use by primitive transformations only. */ public static PCollection createPrimitiveOutputInternal( - WindowingFn windowingFn) { - return new PCollection().setWindowingFnInternal(windowingFn); + WindowFn windowFn) { + return new PCollection().setWindowFnInternal(windowFn); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java index 349ec2cc750cc..009ecd93b71e6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java @@ -18,7 +18,7 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.common.collect.ImmutableMap; import com.google.common.reflect.TypeToken; @@ -183,7 +183,7 @@ public Output apply( *

For use by primitive transformations only. */ public static PCollectionTuple ofPrimitiveOutputsInternal( - TupleTagList outputTags, WindowingFn windowingFn) { + TupleTagList outputTags, WindowFn windowFn) { Map, PCollection> pcollectionMap = new LinkedHashMap<>(); for (TupleTag outputTag : outputTags.tupleTags) { if (pcollectionMap.containsKey(outputTag)) { @@ -200,7 +200,7 @@ public static PCollectionTuple ofPrimitiveOutputsInternal( @SuppressWarnings("unchecked") TypeToken token = (TypeToken) outputTag.getTypeToken(); PCollection outputCollection = PCollection - .createPrimitiveOutputInternal(windowingFn) + .createPrimitiveOutputInternal(windowFn) .setTypeTokenInternal(token); pcollectionMap.put(outputTag, outputCollection); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java index 8ae07b516de95..63b8eea648bfc 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java @@ -97,7 +97,7 @@ public void testCreateNormalParDoFn() throws Exception { DoFn actualDoFn = normalParDoFn.fnFactory.createDoFnInfo().getDoFn(); Assert.assertThat(actualDoFn, new IsInstanceOf(TestDoFn.class)); Assert.assertThat( - normalParDoFn.fnFactory.createDoFnInfo().getWindowingFn(), + normalParDoFn.fnFactory.createDoFnInfo().getWindowFn(), new IsInstanceOf(GlobalWindow.class)); TestDoFn actualTestDoFn = (TestDoFn) actualDoFn; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FlattenTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FlattenTest.java index 70cc4f1eaf88d..e5822433b3224 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FlattenTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FlattenTest.java @@ -96,7 +96,7 @@ public void testFlattenPCollectionListEmpty() { } @Test - public void testWindowingFnPropagationFailure() { + public void testWindowFnPropagationFailure() { Pipeline p = TestPipeline.create(); PCollection input1 = @@ -113,12 +113,12 @@ public void testWindowingFnPropagationFailure() { Assert.fail("Exception should have been thrown"); } catch (IllegalStateException e) { Assert.assertTrue(e.getMessage().startsWith( - "Inputs to Flatten had incompatible window windowingFns")); + "Inputs to Flatten had incompatible window windowFns")); } } @Test - public void testWindowingFnPropagation() { + public void testWindowFnPropagation() { Pipeline p = TestPipeline.create(); PCollection input1 = @@ -134,12 +134,12 @@ public void testWindowingFnPropagation() { p.run(); - Assert.assertTrue(output.getWindowingFn().isCompatible( + Assert.assertTrue(output.getWindowFn().isCompatible( FixedWindows.of(Duration.standardMinutes(1)))); } @Test - public void testEqualWindowingFnPropagation() { + public void testEqualWindowFnPropagation() { Pipeline p = TestPipeline.create(); PCollection input1 = @@ -155,7 +155,7 @@ public void testEqualWindowingFnPropagation() { p.run(); - Assert.assertTrue(output.getWindowingFn().isCompatible( + Assert.assertTrue(output.getWindowFn().isCompatible( Sessions.withGapDuration(Duration.standardMinutes(2)))); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java index 0ea40ae263ce9..02d63b9d3fb4d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java @@ -29,7 +29,7 @@ import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; -import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindowFn; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.values.KV; @@ -186,7 +186,7 @@ public void testGroupByKeyNonDeterministic() throws Exception { } @Test - public void testIdentityWindowingFnPropagation() { + public void testIdentityWindowFnPropagation() { Pipeline p = TestPipeline.create(); List> ungroupedPairs = Arrays.asList(); @@ -201,13 +201,13 @@ public void testIdentityWindowingFnPropagation() { p.run(); - Assert.assertTrue(output.getWindowingFn().isCompatible( + Assert.assertTrue(output.getWindowFn().isCompatible( FixedWindows.>of(Duration.standardMinutes(1)))); } @Test - public void testWindowingFnInvalidation() { + public void testWindowFnInvalidation() { Pipeline p = TestPipeline.create(); List> ungroupedPairs = Arrays.asList(); @@ -224,15 +224,15 @@ public void testWindowingFnInvalidation() { p.run(); Assert.assertTrue( - output.getWindowingFn().isCompatible( - new InvalidWindowingFn( + output.getWindowFn().isCompatible( + new InvalidWindowFn( "Invalid", Sessions.>withGapDuration( Duration.standardMinutes(1))))); } @Test - public void testInvalidWindowingFn() { + public void testInvalidWindowFn() { Pipeline p = TestPipeline.create(); List> ungroupedPairs = Arrays.asList(); @@ -275,7 +275,7 @@ public void testRemerge() { p.run(); Assert.assertTrue( - middle.getWindowingFn().isCompatible( + middle.getWindowFn().isCompatible( Sessions.withGapDuration(Duration.standardMinutes(1)))); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java index 39494e9647c2a..22550a37ffa68 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java @@ -1204,7 +1204,7 @@ public void processElement(ProcessContext c) { fail("should have failed"); } catch (Exception e) { assertThat(e.toString(), containsString( - "WindowingFn attemped to access input timestamp when none was available")); + "WindowFn attemped to access input timestamp when none was available")); } } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindowsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindowsTest.java index 36028e493a755..7950c46f566cd 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindowsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindowsTest.java @@ -16,8 +16,8 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; -import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.runWindowingFn; -import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.set; +import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.runWindowFn; +import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.set; import static org.junit.Assert.assertEquals; import org.joda.time.DateTime; @@ -35,7 +35,7 @@ import java.util.Set; /** - * Tests for CalendarWindows WindowingFn. + * Tests for CalendarWindows WindowFn. */ @RunWith(JUnit4.class) public class CalendarWindowsTest { @@ -77,7 +77,7 @@ public void testDays() throws Exception { makeTimestamp(2015, 1, 2, 0, 0)), set(timestamps.get(4), timestamps.get(5))); - assertEquals(expected, runWindowingFn(CalendarWindows.days(1), timestamps)); + assertEquals(expected, runWindowFn(CalendarWindows.days(1), timestamps)); } @Test @@ -114,7 +114,7 @@ public void testWeeks() throws Exception { set(timestamps.get(4), timestamps.get(5))); assertEquals(expected, - runWindowingFn(CalendarWindows.weeks(1, DateTimeConstants.WEDNESDAY), timestamps)); + runWindowFn(CalendarWindows.weeks(1, DateTimeConstants.WEDNESDAY), timestamps)); } @Test @@ -151,7 +151,7 @@ public void testMonths() throws Exception { set(timestamps.get(4), timestamps.get(5))); assertEquals(expected, - runWindowingFn(CalendarWindows.months(1), timestamps)); + runWindowFn(CalendarWindows.months(1), timestamps)); } @Test @@ -187,7 +187,7 @@ public void testMultiMonths() throws Exception { makeTimestamp(2016, 7, 5, 0, 0)), set(timestamps.get(4), timestamps.get(5))); - assertEquals(expected, runWindowingFn( + assertEquals(expected, runWindowFn( CalendarWindows.months(7).withStartingMonth(2014, 3).beginningOnDay(5), timestamps)); } @@ -224,7 +224,7 @@ public void testYears() throws Exception { makeTimestamp(2060, 5, 5, 0, 0)), set(timestamps.get(4), timestamps.get(5))); - assertEquals(expected, runWindowingFn( + assertEquals(expected, runWindowFn( CalendarWindows.years(10).withStartingYear(2000).beginningOnDay(5, 5), timestamps)); } @@ -253,7 +253,7 @@ public void testTimeZone() throws Exception { new DateTime(2014, 1, 3, 0, 0, timeZone).toInstant()), set(timestamps.get(2), timestamps.get(3))); - assertEquals(expected, runWindowingFn( + assertEquals(expected, runWindowFn( CalendarWindows.days(1).withTimeZone(timeZone), timestamps)); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindowsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindowsTest.java index 0a68e72348f73..c59a668989b40 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindowsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindowsTest.java @@ -16,8 +16,8 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; -import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.runWindowingFn; -import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.set; +import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.runWindowFn; +import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.set; import static org.hamcrest.CoreMatchers.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -37,7 +37,7 @@ import java.util.Set; /** - * Tests for FixedWindows WindowingFn. + * Tests for FixedWindows WindowFn. */ @RunWith(JUnit4.class) public class FixedWindowsTest { @@ -50,7 +50,7 @@ public void testSimpleFixedWindow() throws Exception { expected.put(new IntervalWindow(new Instant(100), new Instant(110)), set(100)); assertEquals( expected, - runWindowingFn( + runWindowFn( FixedWindows.of(new Duration(10)), Arrays.asList(1L, 2L, 5L, 9L, 10L, 11L, 100L))); } @@ -63,7 +63,7 @@ public void testFixedOffsetWindow() throws Exception { expected.put(new IntervalWindow(new Instant(95), new Instant(105)), set(100)); assertEquals( expected, - runWindowingFn( + runWindowFn( FixedWindows.of(new Duration(10)).withOffset(new Duration(5)), Arrays.asList(1L, 2L, 5L, 9L, 10L, 11L, 100L))); } @@ -75,7 +75,7 @@ public void testTimeUnit() throws Exception { expected.put(new IntervalWindow(new Instant(5000), new Instant(15000)), set(5000, 5001, 10000)); assertEquals( expected, - runWindowingFn( + runWindowFn( FixedWindows.of(Duration.standardSeconds(10)).withOffset(Duration.standardSeconds(5)), Arrays.asList(1L, 2L, 1000L, 5000L, 5001L, 10000L))); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java index ccb1ddecc4968..c500fcf340001 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java @@ -16,8 +16,8 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; -import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.runWindowingFn; -import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.set; +import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.runWindowFn; +import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.set; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -33,7 +33,7 @@ import java.util.Set; /** - * Tests for Sessions WindowingFn. + * Tests for Sessions WindowFn. */ @RunWith(JUnit4.class) public class SessionsTest { @@ -46,7 +46,7 @@ public void testSimple() throws Exception { expected.put(new IntervalWindow(new Instant(101), new Instant(111)), set(101)); assertEquals( expected, - runWindowingFn( + runWindowFn( Sessions.withGapDuration(new Duration(10)), Arrays.asList(0L, 10L, 101L))); } @@ -58,7 +58,7 @@ public void testConsecutive() throws Exception { expected.put(new IntervalWindow(new Instant(100), new Instant(111)), set(100, 101)); assertEquals( expected, - runWindowingFn( + runWindowFn( Sessions.withGapDuration(new Duration(10)), Arrays.asList(1L, 2L, 5L, 9L, 100L, 101L))); } @@ -70,7 +70,7 @@ public void testMerging() throws Exception { expected.put(new IntervalWindow(new Instant(95), new Instant(111)), set(95, 100, 101)); assertEquals( expected, - runWindowingFn( + runWindowFn( Sessions.withGapDuration(new Duration(10)), Arrays.asList(1L, 15L, 30L, 100L, 101L, 95L, 22L, 10L))); } @@ -83,7 +83,7 @@ public void testTimeUnit() throws Exception { expected.put(new IntervalWindow(new Instant(10000), new Instant(11000)), set(10000)); assertEquals( expected, - runWindowingFn( + runWindowFn( Sessions.withGapDuration(Duration.standardSeconds(1)), Arrays.asList(1L, 2L, 1000L, 5000L, 5001L, 10000L))); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java index f187cb429940e..8af9782fc382c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java @@ -16,8 +16,8 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; -import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.runWindowingFn; -import static com.google.cloud.dataflow.sdk.testing.WindowingFnTestUtils.set; +import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.runWindowFn; +import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.set; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -34,7 +34,7 @@ import java.util.Set; /** - * Tests for the SlidingWindows WindowingFn. + * Tests for the SlidingWindows WindowFn. */ @RunWith(JUnit4.class) public class SlidingWindowsTest { @@ -48,7 +48,7 @@ public void testSimple() throws Exception { expected.put(new IntervalWindow(new Instant(10), new Instant(20)), set(10, 11)); assertEquals( expected, - runWindowingFn( + runWindowFn( SlidingWindows.of(new Duration(10)).every(new Duration(5)), Arrays.asList(1L, 2L, 5L, 9L, 10L, 11L))); } @@ -62,7 +62,7 @@ public void testSlightlyOverlapping() throws Exception { expected.put(new IntervalWindow(new Instant(10), new Instant(17)), set(10, 11)); assertEquals( expected, - runWindowingFn( + runWindowFn( SlidingWindows.of(new Duration(7)).every(new Duration(5)), Arrays.asList(1L, 2L, 5L, 9L, 10L, 11L))); } @@ -75,7 +75,7 @@ public void testElidings() throws Exception { expected.put(new IntervalWindow(new Instant(100), new Instant(103)), set(100)); assertEquals( expected, - runWindowingFn( + runWindowFn( // Only look at the first 3 millisecs of every 10-millisec interval. SlidingWindows.of(new Duration(3)).every(new Duration(10)), Arrays.asList(1L, 2L, 3L, 5L, 9L, 10L, 11L, 100L))); @@ -90,7 +90,7 @@ public void testOffset() throws Exception { expected.put(new IntervalWindow(new Instant(7), new Instant(17)), set(9, 10, 11)); assertEquals( expected, - runWindowingFn( + runWindowFn( SlidingWindows.of(new Duration(10)).every(new Duration(5)).withOffset(new Duration(2)), Arrays.asList(1L, 2L, 5L, 9L, 10L, 11L))); } @@ -105,7 +105,7 @@ public void testTimeUnit() throws Exception { expected.put(new IntervalWindow(new Instant(10000), new Instant(20000)), set(10000)); assertEquals( expected, - runWindowingFn( + runWindowFn( SlidingWindows.of(Duration.standardSeconds(10)).every(Duration.standardSeconds(5)), Arrays.asList(1L, 2L, 1000L, 5000L, 5001L, 10000L))); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java index 9e05a7eb26c4b..34342f7cbc2cf 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowingTest.java @@ -56,14 +56,14 @@ public class WindowingTest implements Serializable { public TemporaryFolder tmpFolder = new TemporaryFolder(); private static class WindowedCount extends PTransform, PCollection> { - private WindowingFn windowingFn; - public WindowedCount(WindowingFn windowingFn) { - this.windowingFn = (WindowingFn) windowingFn; + private WindowFn windowFn; + public WindowedCount(WindowFn windowFn) { + this.windowFn = (WindowFn) windowFn; } @Override public PCollection apply(PCollection in) { return in - .apply(Window.named("Window").into(windowingFn)) + .apply(Window.named("Window").into(windowFn)) .apply(Count.perElement()) .apply(ParDo .named("FormatCounts") diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java index 20f24c2ebcd30..b2c3d32e6c171 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java @@ -27,7 +27,7 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.common.CounterSet; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.TupleTag; @@ -207,11 +207,11 @@ public class GroupAlsoByWindowsDoFnTest { private DoFnRunner>>, KV>, List> makeRunner( - WindowingFn windowingFn) { + WindowFn windowFn) { GroupAlsoByWindowsDoFn fn = new GroupAlsoByWindowsDoFn( - windowingFn, StringUtf8Coder.of()); + windowFn, StringUtf8Coder.of()); DoFnRunner>>, KV>, List> runner = diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java index c199b9190cfa6..3de4a74207546 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java @@ -29,7 +29,7 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowingFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.common.CounterSet; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.TupleTag; @@ -259,7 +259,7 @@ public class StreamingGroupAlsoByWindowsDoFnTest { private DoFnRunner>, KV>, List> makeRunner( - WindowingFn windowingStrategy) { + WindowFn windowingStrategy) { StreamingGroupAlsoByWindowsDoFn, IntervalWindow> fn = StreamingGroupAlsoByWindowsDoFn.create(windowingStrategy, StringUtf8Coder.of()); From c000e5b4c49b2e6c777bf07bb1c3c119e5d30c6c Mon Sep 17 00:00:00 2001 From: ccy Date: Mon, 26 Jan 2015 15:25:09 -0800 Subject: [PATCH 0097/1541] Report GCS path of error-producing write in Dataflow SDK GoogleCloudStorageWriteChannel. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84821763 --- .../gcsio/GoogleCloudStorageWriteChannel.java | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageWriteChannel.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageWriteChannel.java index 11113d0367ea5..a05650d64b69f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageWriteChannel.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageWriteChannel.java @@ -20,7 +20,6 @@ import com.google.api.client.http.InputStreamContent; import com.google.api.client.util.Preconditions; import com.google.api.services.storage.Storage; -import com.google.api.services.storage.model.StorageObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -313,10 +312,6 @@ private void init( ExecutorService threadPool, Storage gcs, String bucketName, String objectName, String contentType) throws IOException { - - // Create object with the given name. - StorageObject object = (new StorageObject()).setName(objectName); - // Create a pipe such that its one end is connected to the input stream used by // the uploader and the other end is the write channel used by the caller. pipeSource = new PipedInputStream(pipeBufferSize); @@ -330,7 +325,8 @@ private void init( objectContentStream.setLength(-1); objectContentStream.setCloseInputStream(false); Storage.Objects.Insert insertObject = - gcs.objects().insert(bucketName, object, objectContentStream); + gcs.objects().insert(bucketName, null, objectContentStream); + insertObject.setName(objectName); insertObject.setDisableGZipContent(true); insertObject.getMediaHttpUploader().setProgressListener( new LoggingMediaHttpUploaderProgressListener(objectName, MIN_LOGGING_INTERVAL_MS)); @@ -373,7 +369,17 @@ private void throwIfUploadFailed() if (uploadOperation.exception() instanceof Error) { throw (Error) uploadOperation.exception(); } - throw new IOException(uploadOperation.exception()); + throw new IOException(String.format("Failed to write to GCS path %s.", getPrintableGCSPath()), + uploadOperation.exception()); } } + + /** + * Gets the printable GCS path of the current channel. + */ + private String getPrintableGCSPath() { + // The bucket and object name are fields stored in the uploadOperation. + return String.format("gs://%s/%s", uploadOperation.insertObject.getBucket(), + uploadOperation.insertObject.getName()); + } } From f4922de1f3c5ba7fd19a062f4108267a4c84e816 Mon Sep 17 00:00:00 2001 From: ananvay Date: Mon, 26 Jan 2015 16:11:02 -0800 Subject: [PATCH 0098/1541] Adding a filter transform. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84825824 --- .../cloud/dataflow/sdk/transforms/Filter.java | 193 ++++++++++++++++++ .../dataflow/sdk/transforms/FilterTest.java | 116 +++++++++++ 2 files changed, 309 insertions(+) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FilterTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java new file mode 100644 index 0000000000000..c870abe43f9c2 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java @@ -0,0 +1,193 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.values.PCollection; + +/** + * {@code PTransform}s for filtering from a {@code PCollection} the + * elements satisfying a predicate, or satisfying an inequality with + * a given value based on the elements' natural ordering. + * + * @param the type of the values in the input {@code PCollection}, + * and the type of the elements in the output {@code PCollection} + */ +@SuppressWarnings("serial") +public class Filter extends PTransform, + PCollection> { + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a {@code PCollection} with + * elements that satisfy the given predicate. The predicate must be + * a {@code SerializableFunction}. + * + *

Example of use: + *

 {@code
+   * PCollection wordList = ...;
+   * PCollection longWords =
+   *     wordList.apply(Filter.by(new MatchIfWordLengthGT(6)));
+   * } 
+ * + *

See also {@link #lessThan}, {@link #lessThanEq}, + * {@link #greaterThan}, {@link #greaterThanEq}, which return elements + * satisfying various inequalities with the specified value based on + * the elements' natural ordering. + */ + public static > + ParDo.Bound by(final C filterPred) { + return ParDo.named("Filter").of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + if (filterPred.apply(c.element()) == true) { + c.output(c.element()); + } + } + }); + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a {@code PCollection} with + * elements that are less than a given value, based on the + * elements' natural ordering. Elements must be {@code Comparable}. + * + *

Example of use: + *

 {@code
+   * PCollection listOfNumbers = ...;
+   * PCollection smallNumbers =
+   *     listOfNumbers.apply(Filter.lessThan(10));
+   * } 
+ * + *

See also {@link #lessThanEq}, {@link #greaterThanEq}, + * {@link #greaterThan} which return elements satisfying various + * inequalities with the specified value based on the elements' + * natural ordering. + * + *

See also {@link #by}, which returns elements + * that satisfy the given predicate. + */ + public static > + ParDo.Bound lessThan(final T value) { + return ParDo.named("Filter.lessThan").of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + if (c.element().compareTo(value) < 0) { + c.output(c.element()); + } + } + }); + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a {@code PCollection} with + * elements that are greater than a given value, based on the + * elements' natural ordering. Elements must be {@code Comparable}. + * + *

Example of use: + *

 {@code
+   * PCollection listOfNumbers = ...;
+   * PCollection largeNumbers =
+   *     listOfNumbers.apply(Filter.greaterThan(1000));
+   * } 
+ * + *

See also {@link #greaterThanEq}, {@link #lessThan}, + * {@link #lessThanEq} which return elements satisfying various + * inequalities with the specified value based on the elements' + * natural ordering. + * + *

See also {@link #by}, which returns elements + * that satisfy the given predicate. + */ + public static > + ParDo.Bound greaterThan(final T value) { + return ParDo.named("Filter.greaterThan").of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + if (c.element().compareTo(value) > 0) { + c.output(c.element()); + } + } + }); + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a {@code PCollection} with + * elements that are less than or equal to a given value, based on the + * elements' natural ordering. Elements must be {@code Comparable}. + * + *

Example of use: + *

 {@code
+   * PCollection listOfNumbers = ...;
+   * PCollection smallOrEqualNumbers =
+   *     listOfNumbers.apply(Filter.lessThanEq(10));
+   * } 
+ * + *

See also {@link #lessThan}, {@link #greaterThanEq}, + * {@link #greaterThan} which return elements satisfying various + * inequalities with the specified value based on the elements' + * natural ordering. + * + *

See also {@link #by}, which returns elements + * that satisfy the given predicate. + */ + public static > + ParDo.Bound lessThanEq(final T value) { + return ParDo.named("Filter.lessThanEq").of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + if (c.element().compareTo(value) <= 0) { + c.output(c.element()); + } + } + }); + } + + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a {@code PCollection} with + * elements that are greater than or equal to a given value, based on + * the elements' natural ordering. Elements must be {@code Comparable}. + * + *

Example of use: + *

 {@code
+   * PCollection listOfNumbers = ...;
+   * PCollection largeOrEqualNumbers =
+   *     listOfNumbers.apply(Filter.greaterThanEq(1000));
+   * } 
+ * + *

See also {@link #greaterThan}, {@link #lessThan}, + * {@link #lessThanEq} which return elements satisfying various + * inequalities with the specified value based on the elements' + * natural ordering. + * + *

See also {@link #by}, which returns elements + * that satisfy the given predicate. + */ + public static > + ParDo.Bound greaterThanEq(final T value) { + return ParDo.named("Filter.greaterThanEq").of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + if (c.element().compareTo(value) >= 0) { + c.output(c.element()); + } + } + }); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FilterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FilterTest.java new file mode 100644 index 0000000000000..d89092299bb70 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FilterTest.java @@ -0,0 +1,116 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static com.google.cloud.dataflow.sdk.TestUtils.createInts; + +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; +import java.util.Arrays; + +/** + * Tests for {@link Filter}. + */ +@RunWith(JUnit4.class) +@SuppressWarnings("serial") +public class FilterTest implements Serializable { + + static class TrivialFn implements SerializableFunction { + private final Boolean returnVal; + + TrivialFn(Boolean returnVal) { + this.returnVal = returnVal; + } + + public Boolean apply(Integer elem) { + return this.returnVal; + } + } + + static class EvenFn implements SerializableFunction { + public Boolean apply(Integer elem) { + return elem % 2 == 0; + } + } + + @Test + public void testIdentityFilterBy() { + TestPipeline p = TestPipeline.create(); + + PCollection input = createInts(p, Arrays.asList(591, 11789, 1257, 24578, 24799, 307)); + + PCollection output = input.apply(Filter.by(new TrivialFn(true))); + + DataflowAssert.that(output).containsInAnyOrder(591, 11789, 1257, 24578, 24799, 307); + p.run(); + } + + @Test + public void testNoFilter() { + TestPipeline p = TestPipeline.create(); + + PCollection input = createInts(p, Arrays.asList(1, 2, 4, 5)); + + PCollection output = input.apply(Filter.by(new TrivialFn(false))); + + DataflowAssert.that(output).containsInAnyOrder(); + p.run(); + } + + @Test + public void testFilterBy() { + TestPipeline p = TestPipeline.create(); + + PCollection input = createInts(p, Arrays.asList(1, 2, 3, 4, 5, 6, 7)); + + PCollection output = input.apply(Filter.by(new EvenFn())); + + DataflowAssert.that(output).containsInAnyOrder(2, 4, 6); + p.run(); + } + + @Test + public void testFilterLessThan() { + TestPipeline p = TestPipeline.create(); + + PCollection input = createInts(p, Arrays.asList(1, 2, 3, 4, 5, 6, 7)); + + PCollection output = input.apply(Filter.lessThan(4)); + + DataflowAssert.that(output).containsInAnyOrder(1, 2, 3); + p.run(); + } + + @Test + public void testFilterGreaterThan() { + TestPipeline p = TestPipeline.create(); + + PCollection input = createInts(p, Arrays.asList(1, 2, 3, 4, 5, 6, 7)); + + PCollection output = input.apply(Filter.greaterThan(4)); + + DataflowAssert.that(output).containsInAnyOrder(5, 6, 7); + p.run(); + } +} From 8d5718af85d98ff766764934a090b392441c8fbf Mon Sep 17 00:00:00 2001 From: vanya Date: Mon, 26 Jan 2015 16:18:28 -0800 Subject: [PATCH 0099/1541] Fix a typo: UserDame->UserName [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84826672 --- .../cloud/dataflow/sdk/options/DataflowPipelineOptions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java index 907d020cad23d..b71949ec9947b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java @@ -69,7 +69,7 @@ public interface DataflowPipelineOptions extends * not be able to be created. */ @Description("Dataflow job name, to uniquely identify active jobs. " - + "Defaults to using the ApplicationName-UserDame-Date.") + + "Defaults to using the ApplicationName-UserName-Date.") @Default.InstanceFactory(JobNameFactory.class) String getJobName(); void setJobName(String value); From caecd62913e9404618e2259ee513b467abd08800 Mon Sep 17 00:00:00 2001 From: ananvay Date: Tue, 27 Jan 2015 19:27:39 -0800 Subject: [PATCH 0100/1541] Updated test for Filter Transform. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84930230 --- .../com/google/cloud/dataflow/sdk/transforms/FilterTest.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FilterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FilterTest.java index d89092299bb70..7d91204edcfa3 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FilterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FilterTest.java @@ -23,6 +23,7 @@ import com.google.cloud.dataflow.sdk.values.PCollection; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -55,6 +56,7 @@ public Boolean apply(Integer elem) { } @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) public void testIdentityFilterBy() { TestPipeline p = TestPipeline.create(); @@ -79,6 +81,7 @@ public void testNoFilter() { } @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) public void testFilterBy() { TestPipeline p = TestPipeline.create(); @@ -91,6 +94,7 @@ public void testFilterBy() { } @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) public void testFilterLessThan() { TestPipeline p = TestPipeline.create(); From 0fdcc1b06daa427b1af0db1f7f6a500a8056b7e7 Mon Sep 17 00:00:00 2001 From: lcwik Date: Wed, 28 Jan 2015 12:05:49 -0800 Subject: [PATCH 0101/1541] Make LogSaver thread-safe so that ExpectedLogs can be used to capture logs which are output from multiple threads. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84990025 --- .../dataflow/sdk/testing/ExpectedLogs.java | 6 +++- .../sdk/testing/ExpectedLogsTest.java | 33 +++++++++++++++++++ 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogs.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogs.java index 3f4e33d63268a..0e2d4722b4aef 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogs.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogs.java @@ -27,11 +27,14 @@ import org.junit.rules.TestRule; import java.util.Collection; +import java.util.concurrent.ConcurrentLinkedDeque; import java.util.logging.Handler; import java.util.logging.Level; import java.util.logging.LogRecord; import java.util.logging.Logger; +import javax.annotation.concurrent.ThreadSafe; + /** * This {@link TestRule} enables the ability to capture JUL logging events during test execution and * assert expectations that they contain certain messages (with or without {@link Throwable}) at @@ -219,8 +222,9 @@ private ExpectedLogs(Class klass) { /** * A JUL logging {@link Handler} that records all logging events which are passed to it. */ + @ThreadSafe private static class LogSaver extends Handler { - Collection logRecords = Lists.newArrayList(); + Collection logRecords = new ConcurrentLinkedDeque<>(); public Collection getLogs() { return logRecords; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java index bcd96cefffe45..2766fa56db5e1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java @@ -25,6 +25,10 @@ import java.io.IOException; import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.Executors; /** Tests for {@link FastNanoClockAndSleeper}. */ @RunWith(JUnit4.class) @@ -89,6 +93,35 @@ public void testLogCaptureOccursAtLowestLogLevel() throws Throwable { expectedLogs.after(); } + @Test + public void testThreadSafetyOfLogSaver() throws Throwable { + expectedLogs.before(); + + CompletionService completionService = + new ExecutorCompletionService<>(Executors.newCachedThreadPool()); + final long scheduledLogTime = System.currentTimeMillis() + 500L; + for (int i = 0; i < 100; i++) { + final String expected = generateRandomString(); + expectedLogs.expectTrace(expected); + completionService.submit(new Callable() { + @Override + public Void call() throws Exception { + // Have all threads started and waiting to log at about the same moment. + Thread.sleep(Math.max(1, scheduledLogTime - System.currentTimeMillis())); + LOG.trace(expected); + return null; + } + }); + } + + // Wait for all the threads to complete. + for (int i = 0; i < 100; i++) { + completionService.take(); + } + + expectedLogs.after(); + } + // Generates a random fake error message. private static String generateRandomString() { Random random = new Random(); From 2ee5d35a42e7a5a3c017ca2fcdb801f738ed67af Mon Sep 17 00:00:00 2001 From: lcwik Date: Wed, 28 Jan 2015 13:35:22 -0800 Subject: [PATCH 0102/1541] Update dependencies to latest versions of Google API services. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=84997980 --- examples/pom.xml | 4 ++-- sdk/pom.xml | 12 ++++++------ 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 6604a90e531ef..81de0a61629b3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -150,7 +150,7 @@ com.google.apis google-api-services-storage - v1-rev11-1.19.0 + v1-rev25-1.19.1 @@ -164,7 +164,7 @@ com.google.apis google-api-services-bigquery - v2-rev167-1.19.0 + v2-rev187-1.19.1 diff --git a/sdk/pom.xml b/sdk/pom.xml index c761197a78b3d..1b3e7c26603a0 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -189,7 +189,7 @@ com.google.apis google-api-services-dataflow - v1beta3-rev1-1.19.0 + v1beta3-rev5-1.19.1 @@ -203,7 +203,7 @@ com.google.apis google-api-services-bigquery - v2-rev167-1.19.0 + v2-rev187-1.19.1 @@ -217,7 +217,7 @@ com.google.apis google-api-services-compute - v1-rev34-1.19.0 + v1-rev46-1.19.1 @@ -231,7 +231,7 @@ com.google.apis google-api-services-pubsub - v1beta1-rev9-1.19.0 + v1beta1-rev18-1.19.1 @@ -245,7 +245,7 @@ com.google.apis google-api-services-storage - v1-rev11-1.19.0 + v1-rev25-1.19.1 @@ -293,7 +293,7 @@ com.google.apis google-api-services-datastore-protobuf - v1beta2-rev1-2.1.0 + v1beta2-rev1-2.1.2 From 84c8ea2be7c4936aa1d222807db7728d87f17992 Mon Sep 17 00:00:00 2001 From: lcwik Date: Wed, 28 Jan 2015 15:07:24 -0800 Subject: [PATCH 0103/1541] Drop the differentiation between user and worker credentials since the getUserCredentials is able to get credentials when executed from the worker for all the use cases the worker got them from. The use cases are to get them from GCE metadata or from a service account keyfile [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85006425 --- .../cloud/dataflow/sdk/io/DatastoreIO.java | 55 ++++--------------- .../dataflow/sdk/options/GcpOptions.java | 4 +- .../runners/worker/DataflowWorkerHarness.java | 15 +---- .../cloud/dataflow/sdk/util/Credentials.java | 40 ++------------ 4 files changed, 21 insertions(+), 93 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java index dfd5041720dd2..6f3e3b8228a6e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java @@ -45,7 +45,6 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.util.Credentials; import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.RetryHttpRequestInitializer; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -57,7 +56,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.security.GeneralSecurityException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -284,7 +282,7 @@ public List splitIntoShards(long desiredShardSizeBytes, PipelineOptions List splitQueries; if (mockSplitter == null) { splitQueries = DatastoreHelper.getQuerySplitter().getSplits( - query, (int) numSplits, getUserDatastore(host, datasetId, options)); + query, (int) numSplits, getDatastore(options)); } else { splitQueries = mockSplitter.getSplits(query, (int) numSplits, null); } @@ -299,11 +297,7 @@ public List splitIntoShards(long desiredShardSizeBytes, PipelineOptions public Reader createBasicReader( PipelineOptions pipelineOptions, Coder coder, ExecutionContext executionContext) throws IOException { - try { - return new DatastoreReader(query, getDatastore(pipelineOptions)); - } catch (GeneralSecurityException e) { - throw new IOException(e); - } + return new DatastoreReader(query, getDatastore(pipelineOptions)); } @Override @@ -313,13 +307,16 @@ public void validate() { Preconditions.checkNotNull(datasetId, "datasetId"); } - private Datastore getDatastore(PipelineOptions pipelineOptions) - throws IOException, GeneralSecurityException { - Datastore datastore = getUserDatastore(host, datasetId, pipelineOptions); - if (datastore == null) { - datastore = getWorkerDatastore(host, datasetId, pipelineOptions); + private Datastore getDatastore(PipelineOptions pipelineOptions) { + DatastoreOptions.Builder builder = + new DatastoreOptions.Builder().host(host).dataset(datasetId).initializer( + new RetryHttpRequestInitializer(null)); + + Credential credential = pipelineOptions.as(GcpOptions.class).getGcpCredential(); + if (credential != null) { + builder.credential(credential); } - return datastore; + return DatastoreFactory.get().create(builder.build()); } /** For testing only. */ @@ -339,36 +336,6 @@ public Source withMockEstimateSizeBytes(Supplier estimateSizeBytes) { } } - public static Datastore getWorkerDatastore( - String host, String datasetId, PipelineOptions options) { - DatastoreOptions.Builder builder = - new DatastoreOptions.Builder().host(host).dataset(datasetId).initializer( - new RetryHttpRequestInitializer(null)); - - try { - Credential credential = - Credentials.getWorkerCredential(options.as(DataflowPipelineOptions.class)); - builder.credential(credential); - } catch (IOException e) { - LOG.warn("IOException: can't get credential for worker.", e); - throw new RuntimeException("Failed on getting credential for worker."); - } - return DatastoreFactory.get().create(builder.build()); - } - - public static Datastore getUserDatastore(String host, String datasetId, PipelineOptions options) - throws IOException, GeneralSecurityException { - DatastoreOptions.Builder builder = - new DatastoreOptions.Builder().host(host).dataset(datasetId).initializer( - new RetryHttpRequestInitializer(null)); - - Credential credential = Credentials.getUserCredential(options.as(GcpOptions.class)); - if (credential != null) { - builder.credential(credential); - } - return DatastoreFactory.get().create(builder.build()); - } - ///////////////////// Write Class ///////////////////////////////// /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java index 52354d92a0c35..114bc0d7c5594 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java @@ -135,13 +135,13 @@ public String create(PipelineOptions options) { /** * Attempts to load the user credentials. See - * {@link Credentials#getUserCredential(GcpOptions)} for more details. + * {@link Credentials#getCredential(GcpOptions)} for more details. */ public static class GcpUserCredentialsFactory implements DefaultValueFactory { @Override public Credential create(PipelineOptions options) { try { - return Credentials.getUserCredential(options.as(GcpOptions.class)); + return Credentials.getCredential(options.as(GcpOptions.class)); } catch (IOException | GeneralSecurityException e) { throw new RuntimeException("Unable to obtain credential", e); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java index ed197b3b15147..f2601a6b5fa02 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java @@ -33,7 +33,6 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingFormatter; import com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingInitializer; -import com.google.cloud.dataflow.sdk.util.Credentials; import com.google.cloud.dataflow.sdk.util.GcsIOChannelFactory; import com.google.cloud.dataflow.sdk.util.IOChannelUtils; import com.google.cloud.dataflow.sdk.util.PropertyNames; @@ -148,18 +147,8 @@ static DataflowWorker create(DataflowWorkerHarnessOptions options) { MDC.put(DataflowWorkerLoggingFormatter.MDC_DATAFLOW_WORKER_ID, options.getWorkerId()); options.setAppName(APPLICATION_NAME); - if (options.getGcpCredential() == null) { - try { - // Load the worker credential, otherwise the default is to load user - // credentials. - options.setGcpCredential(Credentials.getWorkerCredential(options)); - Preconditions.checkState(options.getGcpCredential() != null, - "Failed to obtain worker credential"); - } catch (Throwable e) { - LOG.warn("Unable to obtain any valid credentials. Worker inoperable.", e); - return null; - } - } + Preconditions.checkState(options.getGcpCredential() != null, + "Failed to obtain GCP credential in worker."); // Configure standard IO factories. IOChannelUtils.setIOFactory("gs", new GcsIOChannelFactory(options)); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java index 6d82eed7a7236..e37275cdb936f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java @@ -59,13 +59,7 @@ public class Credentials { * services we access directly (GCS) as opposed to through the backend * (BigQuery, GCE), we need to explicitly request that scope. */ - private static final List WORKER_SCOPES = Arrays.asList( - "https://www.googleapis.com/auth/cloud-platform", - "https://www.googleapis.com/auth/devstorage.full_control", - "https://www.googleapis.com/auth/userinfo.email", - "https://www.googleapis.com/auth/datastore"); - - private static final List USER_SCOPES = Arrays.asList( + private static final List SCOPES = Arrays.asList( "https://www.googleapis.com/auth/cloud-platform", "https://www.googleapis.com/auth/devstorage.full_control", "https://www.googleapis.com/auth/userinfo.email", @@ -79,29 +73,7 @@ public String getRedirectUri() { } /** - * Initializes OAuth2 credential for a worker, using the - * - * application default credentials, or from a local key file when running outside of GCE. - */ - public static Credential getWorkerCredential(GcpOptions options) - throws IOException { - String keyFile = options.getServiceAccountKeyfile(); - String accountName = options.getServiceAccountName(); - - if (keyFile != null && accountName != null) { - try { - return getCredentialFromFile(keyFile, accountName, WORKER_SCOPES); - } catch (GeneralSecurityException e) { - LOG.warn("Unable to obtain credentials from file {}", keyFile); - // Fall through.. - } - } - - return GoogleCredential.getApplicationDefault().createScoped(WORKER_SCOPES); - } - - /** - * Initializes OAuth2 credential for an interactive user program. + * Initializes OAuth2 credentials. * * This can use 4 different mechanisms for obtaining a credential: *

    @@ -132,25 +104,25 @@ public static Credential getWorkerCredential(GcpOptions options) * application default credentials falling back to gcloud. The other options can be * used by providing the corresponding properties. */ - public static Credential getUserCredential(GcpOptions options) + public static Credential getCredential(GcpOptions options) throws IOException, GeneralSecurityException { String keyFile = options.getServiceAccountKeyfile(); String accountName = options.getServiceAccountName(); if (keyFile != null && accountName != null) { try { - return getCredentialFromFile(keyFile, accountName, USER_SCOPES); + return getCredentialFromFile(keyFile, accountName, SCOPES); } catch (GeneralSecurityException e) { throw new IOException("Unable to obtain credentials from file", e); } } if (options.getSecretsFile() != null) { - return getCredentialFromClientSecrets(options, USER_SCOPES); + return getCredentialFromClientSecrets(options, SCOPES); } try { - return GoogleCredential.getApplicationDefault().createScoped(USER_SCOPES); + return GoogleCredential.getApplicationDefault().createScoped(SCOPES); } catch (IOException e) { LOG.debug("Failed to get application default credentials, falling back to gcloud."); } From 33e72de6f4eafaa3183585ecf804ccc0267d23b2 Mon Sep 17 00:00:00 2001 From: robertwb Date: Wed, 28 Jan 2015 15:49:53 -0800 Subject: [PATCH 0104/1541] Rename GlobalWindow (the fn) to GlobalWindows for consistency. Window instances are singular, WindowFns are plural. This change is backwards incompatible, but unlikely to affect much user code. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85010354 --- .../google/cloud/dataflow/sdk/io/AvroIO.java | 4 +-- .../cloud/dataflow/sdk/io/BigQueryIO.java | 4 +-- .../cloud/dataflow/sdk/io/PubsubIO.java | 4 +-- .../cloud/dataflow/sdk/io/ReadSource.java | 4 +-- .../google/cloud/dataflow/sdk/io/TextIO.java | 4 +-- .../dataflow/sdk/transforms/Combine.java | 8 +++--- .../cloud/dataflow/sdk/transforms/Create.java | 4 +-- .../dataflow/sdk/transforms/DoFnTester.java | 4 +-- .../dataflow/sdk/transforms/Flatten.java | 4 +-- .../dataflow/sdk/transforms/GroupByKey.java | 4 +-- .../{GlobalWindow.java => GlobalWindows.java} | 28 +++++++++---------- .../dataflow/sdk/util/WindowedValue.java | 4 +-- .../dataflow/sdk/values/PCollection.java | 2 +- .../runners/DataflowPipelineRunnerTest.java | 4 +-- .../DataflowPipelineTranslatorTest.java | 6 ++-- .../sdk/runners/TransformTreeTest.java | 4 +-- .../worker/MapTaskExecutorFactoryTest.java | 4 +-- .../sdk/runners/worker/NormalParDoFnTest.java | 10 +++---- .../runners/worker/ParDoFnFactoryTest.java | 6 ++-- .../sdk/runners/worker/ShuffleSinkTest.java | 9 +++--- .../sdk/util/GroupAlsoByWindowsDoFnTest.java | 4 +-- .../StreamingGroupAlsoByWindowsDoFnTest.java | 4 +-- 22 files changed, 65 insertions(+), 64 deletions(-) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/{GlobalWindow.java => GlobalWindows.java} (67%) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java index ed8be3a761567..2eaf60343e5f6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java @@ -26,7 +26,7 @@ import com.google.cloud.dataflow.sdk.runners.worker.AvroReader; import com.google.cloud.dataflow.sdk.runners.worker.AvroSink; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.util.ReaderUtils; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.common.worker.Sink; @@ -263,7 +263,7 @@ public PCollection apply(PInput input) { // Force the output's Coder to be what the read is using, and // unchangeable later, to ensure that we read the input in the // format specified by the Read transform. - return PCollection.createPrimitiveOutputInternal(new GlobalWindow()) + return PCollection.createPrimitiveOutputInternal(new GlobalWindows()) .setCoder(getDefaultOutputCoder()); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java index d7ed8c2116a1f..11ad6f71e82f6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java @@ -31,7 +31,7 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.util.BigQueryTableInserter; import com.google.cloud.dataflow.sdk.util.ReaderUtils; import com.google.cloud.dataflow.sdk.util.Transport; @@ -285,7 +285,7 @@ public PCollection apply(PInput input) { throw new IllegalStateException( "must set the table reference of a BigQueryIO.Read transform"); } - return PCollection.createPrimitiveOutputInternal(new GlobalWindow()) + return PCollection.createPrimitiveOutputInternal(new GlobalWindows()) // Force the output's Coder to be what the read is using, and // unchangeable later, to ensure that we read the input in the // format specified by the Read transform. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index f5b57f82fb5dc..4a379920249e7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -20,7 +20,7 @@ import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PDone; import com.google.cloud.dataflow.sdk.values.PInput; @@ -231,7 +231,7 @@ public PCollection apply(PInput input) { + "PubsubIO.Read transform"); } return PCollection.createPrimitiveOutputInternal( - new GlobalWindow()); + new GlobalWindows()); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ReadSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ReadSource.java index fc44e34293382..a4a5943fdbbee 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ReadSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ReadSource.java @@ -20,7 +20,7 @@ import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.runners.dataflow.BasicSerializableSourceFormat; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PInput; import com.google.common.base.Preconditions; @@ -97,7 +97,7 @@ protected Coder getDefaultOutputCoder() { public final PCollection apply(PInput input) { Preconditions.checkNotNull(source, "source must be set"); source.validate(); - return PCollection.createPrimitiveOutputInternal(new GlobalWindow()) + return PCollection.createPrimitiveOutputInternal(new GlobalWindows()) .setCoder(getDefaultOutputCoder()); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java index 14fa60cf42e5c..bef838b2e5dc0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java @@ -24,7 +24,7 @@ import com.google.cloud.dataflow.sdk.runners.worker.TextReader; import com.google.cloud.dataflow.sdk.runners.worker.TextSink; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.util.ReaderUtils; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.common.worker.Sink; @@ -236,7 +236,7 @@ public PCollection apply(PInput input) { // Force the output's Coder to be what the read is using, and // unchangeable later, to ensure that we read the input in the // format specified by the Read transform. - return PCollection.createPrimitiveOutputInternal(new GlobalWindow()).setCoder(coder); + return PCollection.createPrimitiveOutputInternal(new GlobalWindows()).setCoder(coder); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java index eba6785b30162..429d10aa9eb5c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -21,7 +21,7 @@ import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionView; @@ -50,7 +50,7 @@ public class Combine { * *

    If the input {@code PCollection} is empty, the ouput will contain a the * default value of the combining function if the input is windowed into - * the {@link GlobalWindow}; otherwise, the output will be empty. Note: this + * the {@link GlobalWindows}; otherwise, the output will be empty. Note: this * behavior is subject to change. * *

    See {@link Globally Combine.Globally} for more information. @@ -69,7 +69,7 @@ public static Globally globally( * * If the input {@code PCollection} is empty, the ouput will contain a the * default value of the combining function if the input is windowed into - * the {@link GlobalWindow}; otherwise, the output will be empty. Note: this + * the {@link GlobalWindows}; otherwise, the output will be empty. Note: this * behavior is subject to change. * *

    See {@link Globally Combine.Globally} for more information. @@ -770,7 +770,7 @@ public PCollection apply(PCollection input) { .apply(Combine.perKey(fn.asKeyedFn())) .apply(Values.create()); - if (input.getWindowFn().isCompatible(new GlobalWindow())) { + if (input.getWindowFn().isCompatible(new GlobalWindows())) { return insertDefaultValueIfEmpty(output); } else { return output; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java index d28e6a28193f4..97b8f16776841 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java @@ -20,7 +20,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PBegin; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -184,7 +184,7 @@ public static CreateTimestamped timestamped( @Override public PCollection apply(PInput input) { - return PCollection.createPrimitiveOutputInternal(new GlobalWindow()); + return PCollection.createPrimitiveOutputInternal(new GlobalWindows()); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java index 55d0f8b18b964..d00652ce3f6d7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java @@ -18,7 +18,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.DoFnRunner; import com.google.cloud.dataflow.sdk.util.PTuple; @@ -355,6 +355,6 @@ void initializeState() { sideOutputTags, (new BatchModeExecutionContext()).createStepContext("stepName"), counterSet.getAddCounterMutator(), - new GlobalWindow()); + new GlobalWindows()); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java index dd8efca82a240..adf588cbdf90b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java @@ -19,7 +19,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionList; @@ -124,7 +124,7 @@ public PCollection apply(PCollectionList inputs) { } } } else { - windowFn = new GlobalWindow(); + windowFn = new GlobalWindows(); } return PCollection.createPrimitiveOutputInternal(windowFn); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index 225fd18dfdabb..0391c0caf10eb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -25,7 +25,7 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindowFn; import com.google.cloud.dataflow.sdk.transforms.windowing.NonMergingWindowFn; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; @@ -459,7 +459,7 @@ public PCollection>> applyHelper( "GroupByKey must have a valid Window merge function. " + "Invalid because: " + cause); } - if (windowFn.isCompatible(new GlobalWindow())) { + if (windowFn.isCompatible(new GlobalWindows())) { // The input PCollection is using the degenerate default // window function, which uses a single global window for all // elements. We can implement this using a more-primitive diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindows.java similarity index 67% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindows.java index e3298fbb3607c..d4858b85ba5c7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindows.java @@ -30,48 +30,48 @@ * Default {@link WindowFn} where all data is in the same bucket. */ @SuppressWarnings("serial") -public class GlobalWindow - extends NonMergingWindowFn { +public class GlobalWindows + extends NonMergingWindowFn { @Override - public Collection assignWindows(AssignContext c) { - return Arrays.asList(Window.INSTANCE); + public Collection assignWindows(AssignContext c) { + return Arrays.asList(GlobalWindow.INSTANCE); } @Override public boolean isCompatible(WindowFn o) { - return o instanceof GlobalWindow; + return o instanceof GlobalWindows; } @Override - public Coder windowCoder() { - return Window.Coder.INSTANCE; + public Coder windowCoder() { + return GlobalWindow.Coder.INSTANCE; } /** * The default window into which all data is placed. */ - public static class Window extends BoundedWindow { - public static final Window INSTANCE = new Window(); + public static class GlobalWindow extends BoundedWindow { + public static final GlobalWindow INSTANCE = new GlobalWindow(); @Override public Instant maxTimestamp() { return new Instant(Long.MAX_VALUE); } - private Window() {} + private GlobalWindow() {} /** * {@link Coder} for encoding and decoding {@code Window}s. */ - public static class Coder extends AtomicCoder { + public static class Coder extends AtomicCoder { public static final Coder INSTANCE = new Coder(); @Override - public void encode(Window window, OutputStream outStream, Context context) {} + public void encode(GlobalWindow window, OutputStream outStream, Context context) {} @Override - public Window decode(InputStream inStream, Context context) { - return Window.INSTANCE; + public GlobalWindow decode(InputStream inStream, Context context) { + return GlobalWindow.INSTANCE; } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java index 29f6078baa277..0562b58b35c50 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java @@ -26,7 +26,7 @@ import com.google.cloud.dataflow.sdk.coders.InstantCoder; import com.google.cloud.dataflow.sdk.coders.StandardCoder; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; import com.fasterxml.jackson.annotation.JsonCreator; @@ -72,7 +72,7 @@ public static WindowedValue of( public static WindowedValue valueInGlobalWindow(V value) { return new WindowedValue<>(value, new Instant(Long.MIN_VALUE), - Arrays.asList(GlobalWindow.Window.INSTANCE)); + Arrays.asList(GlobalWindows.GlobalWindow.INSTANCE)); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java index 12e920d12cd0b..d7cbe1f734b6d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java @@ -53,7 +53,7 @@ *

    Additionally, a {@code PCollection} has an associated * {@link WindowFn} and each element is assigned to a set of windows. * By default, the windowing function is - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow} + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows} * and all elements are assigned into a single default window. * This default can be overridden with the * {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java index 86a307e1594c5..b3f80891244b3 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java @@ -40,7 +40,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo; import com.google.cloud.dataflow.sdk.util.GcsUtil; import com.google.cloud.dataflow.sdk.util.PackageUtil; @@ -440,7 +440,7 @@ public static class TestTransform @Override public PCollection apply(PCollection input) { - return PCollection.createPrimitiveOutputInternal(new GlobalWindow()); + return PCollection.createPrimitiveOutputInternal(new GlobalWindows()); } @Override diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java index bfa3ed7a21eb8..21a4a02a770a2 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java @@ -46,7 +46,7 @@ import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.Sum; import com.google.cloud.dataflow.sdk.transforms.View; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.util.OutputReference; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.TestCredential; @@ -303,7 +303,7 @@ public EmbeddedTransform(Step step) { @Override public PCollection apply(PCollection input) { - return PCollection.createPrimitiveOutputInternal(new GlobalWindow()); + return PCollection.createPrimitiveOutputInternal(new GlobalWindows()); } @Override @@ -385,7 +385,7 @@ public PCollectionTuple apply(PCollection input) { // Fails here when attempting to construct a tuple with an unbound object. return PCollectionTuple.of(sumTag, sum) .and(doneTag, PCollection.createPrimitiveOutputInternal( - new GlobalWindow())); + new GlobalWindows())); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/TransformTreeTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/TransformTreeTest.java index cceefe90f622e..7305fa20f6b54 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/TransformTreeTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/TransformTreeTest.java @@ -27,7 +27,7 @@ import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.First; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.values.PBegin; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionList; @@ -75,7 +75,7 @@ public PCollectionList apply(PBegin b) { // from within a composite transform. return PCollectionList.of( Arrays.asList(result, PCollection.createPrimitiveOutputInternal( - new GlobalWindow()))); + new GlobalWindows()))); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java index 6667ef0c78c06..69c5859f22ea0 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java @@ -42,7 +42,7 @@ import com.google.cloud.dataflow.sdk.runners.worker.SinkFactoryTest.TestSink; import com.google.cloud.dataflow.sdk.runners.worker.SinkFactoryTest.TestSinkFactory; import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CloudObject; @@ -294,7 +294,7 @@ static ParallelInstruction createParDoInstruction( String serializedFn = StringUtils.byteArrayToJsonString( - SerializableUtils.serializeToByteArray(new DoFnInfo(fn, new GlobalWindow()))); + SerializableUtils.serializeToByteArray(new DoFnInfo(fn, new GlobalWindows()))); CloudObject cloudUserFn = CloudObject.forClassName("DoFn"); addString(cloudUserFn, PropertyNames.SERIALIZED_FN, serializedFn); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java index a6c4238f4c19b..57a17ec2db4e2 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java @@ -29,7 +29,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.DoFnInfo; import com.google.cloud.dataflow.sdk.util.PTuple; @@ -153,7 +153,7 @@ public void testNormalParDoFn() throws Exception { List sideOutputTags = Arrays.asList("tag1", "tag2", "tag3"); TestDoFn fn = new TestDoFn(sideOutputTags); - DoFnInfo fnInfo = new DoFnInfo(fn, new GlobalWindow()); + DoFnInfo fnInfo = new DoFnInfo(fn, new GlobalWindows()); TestReceiver receiver = new TestReceiver(); TestReceiver receiver1 = new TestReceiver(); TestReceiver receiver2 = new TestReceiver(); @@ -218,7 +218,7 @@ public void testNormalParDoFn() throws Exception { @Test public void testUnexpectedNumberOfReceivers() throws Exception { TestDoFn fn = new TestDoFn(Collections.emptyList()); - DoFnInfo fnInfo = new DoFnInfo(fn, new GlobalWindow()); + DoFnInfo fnInfo = new DoFnInfo(fn, new GlobalWindows()); TestReceiver receiver = new TestReceiver(); PTuple sideInputValues = PTuple.empty(); @@ -257,7 +257,7 @@ private List stackTraceFrameStrings(Throwable t) { @Test public void testErrorPropagation() throws Exception { TestErrorDoFn fn = new TestErrorDoFn(); - DoFnInfo fnInfo = new DoFnInfo(fn, new GlobalWindow()); + DoFnInfo fnInfo = new DoFnInfo(fn, new GlobalWindows()); TestReceiver receiver = new TestReceiver(); PTuple sideInputValues = PTuple.empty(); @@ -325,7 +325,7 @@ public void testErrorPropagation() throws Exception { @Test public void testUndeclaredSideOutputs() throws Exception { TestDoFn fn = new TestDoFn(Arrays.asList("declared", "undecl1", "undecl2", "undecl3")); - DoFnInfo fnInfo = new DoFnInfo(fn, new GlobalWindow()); + DoFnInfo fnInfo = new DoFnInfo(fn, new GlobalWindows()); CounterSet counters = new CounterSet(); NormalParDoFn normalParDoFn = new NormalParDoFn( diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java index 63b8eea648bfc..ab69d68965ce9 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java @@ -21,7 +21,7 @@ import com.google.api.services.dataflow.model.MultiOutputInfo; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.DoFnInfo; @@ -71,7 +71,7 @@ public void testCreateNormalParDoFn() throws Exception { String serializedFn = StringUtils.byteArrayToJsonString( - SerializableUtils.serializeToByteArray(new DoFnInfo(fn, new GlobalWindow()))); + SerializableUtils.serializeToByteArray(new DoFnInfo(fn, new GlobalWindows()))); CloudObject cloudUserFn = CloudObject.forClassName("DoFn"); addString(cloudUserFn, "serialized_fn", serializedFn); @@ -98,7 +98,7 @@ public void testCreateNormalParDoFn() throws Exception { Assert.assertThat(actualDoFn, new IsInstanceOf(TestDoFn.class)); Assert.assertThat( normalParDoFn.fnFactory.createDoFnInfo().getWindowFn(), - new IsInstanceOf(GlobalWindow.class)); + new IsInstanceOf(GlobalWindows.class)); TestDoFn actualTestDoFn = (TestDoFn) actualDoFn; Assert.assertEquals(stringState, actualTestDoFn.stringState); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java index b2aa533ce53f2..b58fc38c8f46a 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java @@ -23,7 +23,7 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.WindowedValue; @@ -85,7 +85,7 @@ public class ShuffleSinkTest { private void runTestWriteUngroupingShuffleSink(List expected) throws Exception { Coder> windowedValueCoder = - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), new GlobalWindow().windowCoder()); + WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), new GlobalWindows().windowCoder()); ShuffleSink shuffleSink = new ShuffleSink<>( PipelineOptionsFactory.create(), null, ShuffleSink.ShuffleKind.UNGROUPED, @@ -107,7 +107,8 @@ private void runTestWriteUngroupingShuffleSink(List expected) // Ignore the key. byte[] valueBytes = record.getValue(); WindowedValue value = CoderUtils.decodeFromByteArray(windowedValueCoder, valueBytes); - Assert.assertEquals(Lists.newArrayList(GlobalWindow.Window.INSTANCE), value.getWindows()); + Assert.assertEquals(Lists.newArrayList(GlobalWindows.GlobalWindow.INSTANCE), + value.getWindows()); actual.add(value.getValue()); } @@ -170,7 +171,7 @@ void runTestWriteGroupingSortingShuffleSink( KvCoder.of(BigEndianIntegerCoder.of(), KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())), - new GlobalWindow().windowCoder())); + new GlobalWindows().windowCoder())); TestShuffleWriter shuffleWriter = new TestShuffleWriter(); List actualSizes = new ArrayList<>(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java index b2c3d32e6c171..8ee581b6b78b3 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java @@ -23,7 +23,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; @@ -223,7 +223,7 @@ KV>, List> makeRunner( new ArrayList>(), execContext.createStepContext("merge"), counters.getAddCounterMutator(), - new GlobalWindow()); + new GlobalWindows()); return runner; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java index 3de4a74207546..8b31c51620333 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java @@ -25,7 +25,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; @@ -274,7 +274,7 @@ KV>, List> makeRunner( new ArrayList>(), execContext.createStepContext("merge"), counters.getAddCounterMutator(), - new GlobalWindow()); + new GlobalWindows()); return runner; } From 4c16878ffb0c281e8efae5aebfa70bc84cec5c54 Mon Sep 17 00:00:00 2001 From: sgmc Date: Wed, 28 Jan 2015 20:41:15 -0800 Subject: [PATCH 0105/1541] Fix issue with retrying GCS staging, where we would log the "going to retry" message after actually retrying. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85030057 --- .../sdk/util/AttemptBoundedExponentialBackOff.java | 4 ++++ .../cloud/dataflow/sdk/util/PackageUtil.java | 8 ++++---- .../util/AttemptBoundedExponentialBackOffTest.java | 14 ++++++++++++++ 3 files changed, 22 insertions(+), 4 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOff.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOff.java index 78e8e0538b824..8ac0e1ea8f3a0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOff.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOff.java @@ -79,4 +79,8 @@ public long nextBackOffMillis() { currentAttempt += 1; return Math.round(currentIntervalMillis + randomOffset); } + + public boolean atMaxAttempts() { + return currentAttempt >= maximumNumberOfAttempts; + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java index 5afa04a8af1b3..7226f3ae90772 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PackageUtil.java @@ -19,7 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.Sleeper; import com.google.api.services.dataflow.model.DataflowPackage; @@ -149,7 +148,7 @@ static List stageClasspathElementsToGcs( } // Upload file, retrying on failure. - BackOff backoff = new AttemptBoundedExponentialBackOff( + AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff( MAX_ATTEMPTS, INITIAL_BACKOFF_INTERVAL_MS); while (true) { @@ -161,9 +160,10 @@ static List stageClasspathElementsToGcs( numUploaded++; break; } catch (IOException e) { - if (BackOffUtils.next(retrySleeper, backoff)) { - LOG.warn("Upload attempt failed, will retry staging of classpath: {}", + if (!backoff.atMaxAttempts()) { + LOG.warn("Upload attempt failed, sleeping before retrying staging of classpath: {}", classpathElement, e); + BackOffUtils.next(retrySleeper, backoff); } else { // Rethrow last error, to be included as a cause in the catch below. LOG.error("Upload failed, will NOT retry staging of classpath: {}", diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOffTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOffTest.java index e4f332db12e88..97f5225426faf 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOffTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AttemptBoundedExponentialBackOffTest.java @@ -21,6 +21,9 @@ import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.lessThan; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + import com.google.api.client.util.BackOff; @@ -68,4 +71,15 @@ public void testThatResettingAllowsReuse() throws Exception { assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); } + + @Test + public void testAtMaxAttempts() throws Exception { + AttemptBoundedExponentialBackOff backOff = new AttemptBoundedExponentialBackOff(3, 500); + assertFalse(backOff.atMaxAttempts()); + backOff.nextBackOffMillis(); + assertFalse(backOff.atMaxAttempts()); + backOff.nextBackOffMillis(); + assertTrue(backOff.atMaxAttempts()); + assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); + } } From 02729933fb74c3ac62a6ae01151c8138c32a518f Mon Sep 17 00:00:00 2001 From: wan Date: Wed, 28 Jan 2015 21:55:33 -0800 Subject: [PATCH 0106/1541] Renames InvalidWindowFn to InvalidWindows to conform with the convention that concrete window-fn classes are plural nouns. This is backward-incompatible but unlikely to affect users. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85033832 --- .../cloud/dataflow/sdk/transforms/GroupByKey.java | 8 ++++---- .../{InvalidWindowFn.java => InvalidWindows.java} | 10 +++++----- .../dataflow/sdk/transforms/windowing/Window.java | 4 ++-- .../cloud/dataflow/sdk/transforms/GroupByKeyTest.java | 6 +++--- 4 files changed, 14 insertions(+), 14 deletions(-) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/{InvalidWindowFn.java => InvalidWindows.java} (81%) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index 0391c0caf10eb..9b56585fb9ea9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -26,7 +26,7 @@ import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; -import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindowFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.NonMergingWindowFn; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn; @@ -286,7 +286,7 @@ public PCollection>> apply(PCollection> input) { if (!(windowFn instanceof NonMergingWindowFn)) { // Prevent merging windows again, without explicit user // involvement, e.g., by Window.into() or Window.remerge(). - windowFn = new InvalidWindowFn( + windowFn = new InvalidWindows( "WindowFn has already been consumed by previous GroupByKey", windowFn); } @@ -453,8 +453,8 @@ public PCollection>> applyHelper( // key/value input elements and the window merge operation of the // window function associated with the input PCollection. WindowFn windowFn = getInput().getWindowFn(); - if (windowFn instanceof InvalidWindowFn) { - String cause = ((InvalidWindowFn) windowFn).getCause(); + if (windowFn instanceof InvalidWindows) { + String cause = ((InvalidWindows) windowFn).getCause(); throw new IllegalStateException( "GroupByKey must have a valid Window merge function. " + "Invalid because: " + cause); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindows.java similarity index 81% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowFn.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindows.java index 3a17f4a4cbb9a..57985596dd765 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindowFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/InvalidWindows.java @@ -25,11 +25,11 @@ * * @param window type */ -public class InvalidWindowFn extends WindowFn { +public class InvalidWindows extends WindowFn { private String cause; private WindowFn originalWindowFn; - public InvalidWindowFn(String cause, WindowFn originalWindowFn) { + public InvalidWindows(String cause, WindowFn originalWindowFn) { this.originalWindowFn = originalWindowFn; this.cause = cause; } @@ -42,7 +42,7 @@ public String getCause() { } /** - * Returns the original windowFn that this InvalidWindowFn replaced. + * Returns the original windowFn that this InvalidWindows replaced. */ public WindowFn getOriginalWindowFn() { return originalWindowFn; @@ -64,12 +64,12 @@ public Coder windowCoder() { } /** - * {@code InvalidWindowFn} objects with the same {@code originalWindowFn} are compatible. + * {@code InvalidWindows} objects with the same {@code originalWindowFn} are compatible. */ @Override public boolean isCompatible(WindowFn other) { return getClass() == other.getClass() && getOriginalWindowFn().isCompatible( - ((InvalidWindowFn) other).getOriginalWindowFn()); + ((InvalidWindows) other).getOriginalWindowFn()); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java index 28956b1360989..38e69813ecd54 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java @@ -228,8 +228,8 @@ public static class Remerge extends PTransform, PCollection public PCollection apply(PCollection input) { WindowFn windowFn = getInput().getWindowFn(); WindowFn outputWindowFn = - (windowFn instanceof InvalidWindowFn) - ? ((InvalidWindowFn) windowFn).getOriginalWindowFn() + (windowFn instanceof InvalidWindows) + ? ((InvalidWindows) windowFn).getOriginalWindowFn() : windowFn; return input.apply(ParDo.named("Identity").of(new DoFn() { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java index 02d63b9d3fb4d..5d97a7a657b93 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java @@ -29,7 +29,7 @@ import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; -import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindowFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.values.KV; @@ -225,14 +225,14 @@ public void testWindowFnInvalidation() { Assert.assertTrue( output.getWindowFn().isCompatible( - new InvalidWindowFn( + new InvalidWindows( "Invalid", Sessions.>withGapDuration( Duration.standardMinutes(1))))); } @Test - public void testInvalidWindowFn() { + public void testInvalidWindows() { Pipeline p = TestPipeline.create(); List> ungroupedPairs = Arrays.asList(); From 815fb456404b5a1592aa6981ab9020c44e18a32f Mon Sep 17 00:00:00 2001 From: mattlang Date: Thu, 29 Jan 2015 08:03:17 -0800 Subject: [PATCH 0107/1541] Adds support for compressed (gzip and bzip2) text sources. By default, the compression mode for a text source is determined by examining file extensions (i.e., file names ending in .gz will be processed as gzip files and ending in .bz2 will be processed as bzip2 files). This behavior can be overridden by specifying a compression type: TextIO.Read.from(myFileName).withCompressionType(TextIO.CompressionType.GZIP) GZIP, BZIP2, UNCOMPRESSED, and AUTO compression types are supported. AUTO is the default. GZIP, BZIP2, and UNCOMPRESSED will treat all files as gzipped, bzipped, or uncompressed regardless of their extension or contents. If multiple files are to be read from (by using a glob), the extension of each file will be examined in AUTO mode. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85066281 --- sdk/pom.xml | 6 + .../google/cloud/dataflow/sdk/io/TextIO.java | 115 +++++++++++- .../runners/dataflow/TextIOTranslator.java | 1 + .../sdk/runners/worker/FileBasedReader.java | 62 ++++++- .../sdk/runners/worker/TextReader.java | 33 ++-- .../sdk/runners/worker/TextReaderFactory.java | 6 +- .../dataflow/sdk/util/PropertyNames.java | 1 + .../cloud/dataflow/sdk/io/TextIOTest.java | 37 ++++ .../runners/worker/FileBasedReaderTest.java | 45 +++++ .../runners/worker/TextReaderFactoryTest.java | 11 +- .../sdk/runners/worker/TextReaderTest.java | 169 ++++++++++++++++-- .../dataflow/sdk/util/IOFactoryTest.java | 4 +- 12 files changed, 442 insertions(+), 48 deletions(-) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReaderTest.java diff --git a/sdk/pom.xml b/sdk/pom.xml index 1b3e7c26603a0..1b7992b00d382 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -347,6 +347,12 @@ 1.7.7 + + org.apache.commons + commons-compress + 1.9 + + joda-time joda-time diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java index bef838b2e5dc0..487dcc371af31 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java @@ -21,6 +21,7 @@ import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.worker.FileBasedReader; import com.google.cloud.dataflow.sdk.runners.worker.TextReader; import com.google.cloud.dataflow.sdk.runners.worker.TextSink; import com.google.cloud.dataflow.sdk.transforms.PTransform; @@ -32,9 +33,13 @@ import com.google.cloud.dataflow.sdk.values.PDone; import com.google.cloud.dataflow.sdk.values.PInput; +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; + import java.io.IOException; +import java.io.InputStream; import java.util.List; import java.util.regex.Pattern; +import java.util.zip.GZIPInputStream; import javax.annotation.Nullable; @@ -148,7 +153,20 @@ public static Bound withoutValidation() { return new Bound<>(DEFAULT_TEXT_CODER).withoutValidation(); } - // TODO: strippingNewlines, gzipped, etc. + /** + * Returns a TextIO.Read PTransform that reads from a file with the + * specified compression type. + * + *

    If no compression type is specified, the default is AUTO. In this + * mode, the compression type of the file is determined by its extension + * (e.g., *.gz is gzipped, *.bz2 is bzipped, all other extensions are + * uncompressed). + */ + public static Bound withCompressionType(TextIO.CompressionType compressionType) { + return new Bound<>(DEFAULT_TEXT_CODER).withCompressionType(compressionType); + } + + // TODO: strippingNewlines, etc. /** * A root PTransform that reads from a text file (or multiple text files @@ -173,15 +191,20 @@ public static class Bound extends PTransform> { /** An option to indicate if input validation is desired. Default is true. */ final boolean validate; + /** Option to indicate the input source's compression type. Default is AUTO. */ + final TextIO.CompressionType compressionType; + Bound(Coder coder) { - this(null, null, coder, true); + this(null, null, coder, true, TextIO.CompressionType.AUTO); } - Bound(String name, String filepattern, Coder coder, boolean validate) { + Bound(String name, String filepattern, Coder coder, boolean validate, + TextIO.CompressionType compressionType) { super(name); this.coder = coder; this.filepattern = filepattern; this.validate = validate; + this.compressionType = compressionType; } /** @@ -189,7 +212,7 @@ public static class Bound extends PTransform> { * with the given step name. Does not modify this object. */ public Bound named(String name) { - return new Bound<>(name, filepattern, coder, validate); + return new Bound<>(name, filepattern, coder, validate, compressionType); } /** @@ -199,7 +222,7 @@ public Bound named(String name) { * filepatterns.) Does not modify this object. */ public Bound from(String filepattern) { - return new Bound<>(name, filepattern, coder, validate); + return new Bound<>(name, filepattern, coder, validate, compressionType); } /** @@ -212,7 +235,7 @@ public Bound from(String filepattern) { * elements of the resulting PCollection */ public Bound withCoder(Coder coder) { - return new Bound<>(name, filepattern, coder, validate); + return new Bound<>(name, filepattern, coder, validate, compressionType); } /** @@ -225,7 +248,24 @@ public Bound withCoder(Coder coder) { * available at execution time. */ public Bound withoutValidation() { - return new Bound<>(name, filepattern, coder, false); + return new Bound<>(name, filepattern, coder, false, compressionType); + } + + /** + * Returns a new TextIO.Read PTransform that's like this one but + * reads from input sources using the specified compression type. + * Does not modify this object. + * + *

    If AUTO compression type is specified, a compression type is + * selected on a per-file basis, based on the file's extension (e.g., + * .gz will be processed as a gzipped file, .bz will be processed + * as a bzipped file, other extensions with be treated as uncompressed + * input). + * + *

    If no compression type is specified, the default is AUTO. + */ + public Bound withCompressionType(TextIO.CompressionType compressionType) { + return new Bound<>(name, filepattern, coder, validate, compressionType); } @Override @@ -257,6 +297,10 @@ public boolean needsValidation() { return validate; } + public TextIO.CompressionType getCompressionType() { + return compressionType; + } + static { DirectPipelineRunner.registerDefaultTransformEvaluator( Bound.class, new DirectPipelineRunner.TransformEvaluator() { @@ -367,7 +411,7 @@ public static Bound withoutValidation() { return new Bound<>(DEFAULT_TEXT_CODER).withoutValidation(); } - // TODO: appendingNewlines, gzipped, header, footer, etc. + // TODO: appendingNewlines, header, footer, etc. /** * A PTransform that writes a bounded PCollection to a text file (or @@ -585,6 +629,58 @@ public void evaluate( } } + /** + * Possible text file compression types. + */ + public static enum CompressionType implements FileBasedReader.DecompressingStreamFactory { + /** + * Automatically determine the compression type based on filename extension. + */ + AUTO(""), + /** + * Uncompressed (i.e., may be split). + */ + UNCOMPRESSED(""), + /** + * GZipped. + */ + GZIP(".gz") { + @Override + public InputStream createInputStream(InputStream inputStream) throws IOException { + return new GZIPInputStream(inputStream); + } + }, + /** + * BZipped. + */ + BZIP2(".bz2") { + @Override + public InputStream createInputStream(InputStream inputStream) throws IOException { + return new BZip2CompressorInputStream(inputStream); + } + }; + + private String filenameSuffix; + + private CompressionType(String suffix) { + this.filenameSuffix = suffix; + } + + /** + * Determine if a given filename matches a compression type based on its extension. + * @param filename the filename to match + * @return true iff the filename ends with the compression type's known extension. + */ + public boolean matches(String filename) { + return filename.toLowerCase().endsWith(filenameSuffix.toLowerCase()); + } + + @Override + public InputStream createInputStream(InputStream inputStream) throws IOException { + return inputStream; + } + } + // Pattern which matches old-style shard output patterns, which are now // disallowed. private static final Pattern SHARD_OUTPUT_PATTERN = Pattern.compile("@([0-9]+|\\*)"); @@ -601,7 +697,8 @@ private static void validateOutputComponent(String partialFilePattern) { private static void evaluateReadHelper( Read.Bound transform, DirectPipelineRunner.EvaluationContext context) { TextReader reader = - new TextReader<>(transform.filepattern, true, null, null, transform.coder); + new TextReader<>(transform.filepattern, true, null, null, transform.coder, + transform.getCompressionType()); List elems = ReaderUtils.readElemsFromReader(reader); context.setPCollection(transform.getOutput(), elems); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java index 302cea67a55c5..9cf059bf53126 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java @@ -61,6 +61,7 @@ private void translateReadHelper( context.addInput(PropertyNames.FILEPATTERN, gcsPath); context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); context.addInput(PropertyNames.VALIDATE_SOURCE, transform.needsValidation()); + context.addInput(PropertyNames.COMPRESSION_TYPE, transform.getCompressionType().toString()); // TODO: Orderedness? } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java index de116277fd262..c0c16d88f26bf 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java @@ -23,6 +23,7 @@ import com.google.api.services.dataflow.model.ApproximateProgress; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.IOChannelFactory; import com.google.cloud.dataflow.sdk.util.IOChannelUtils; @@ -35,6 +36,7 @@ import java.io.BufferedInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.InputStream; import java.io.PushbackInputStream; import java.nio.channels.Channels; import java.util.Collection; @@ -122,15 +124,20 @@ protected abstract class FileBasedIterator extends AbstractReaderIterator { protected ByteArrayOutputStream nextElement; protected boolean nextElementComputed = false; protected long offset; + protected FileBasedReader.DecompressingStreamFactory compressionStreamFactory; FileBasedIterator(CopyableSeekableByteChannel seeker, long startOffset, long offset, - @Nullable Long endOffset, ProgressTracker tracker) throws IOException { + @Nullable Long endOffset, ProgressTracker tracker, + FileBasedReader.DecompressingStreamFactory compressionStreamFactory) throws IOException { this.seeker = checkNotNull(seeker); this.seeker.position(startOffset); + this.compressionStreamFactory = compressionStreamFactory; + InputStream inputStream = + compressionStreamFactory.createInputStream(Channels.newInputStream(seeker)); BufferedInputStream bufferedStream = useDefaultBufferSize - ? new BufferedInputStream(Channels.newInputStream(seeker)) - : new BufferedInputStream(Channels.newInputStream(seeker), BUF_SIZE); + ? new BufferedInputStream(inputStream) + : new BufferedInputStream(inputStream, BUF_SIZE); this.stream = new PushbackInputStream(bufferedStream, BUF_SIZE); this.startOffset = startOffset; this.offset = offset; @@ -246,4 +253,53 @@ private void computeNextElement() throws IOException { nextElementComputed = true; } } + + /** + * Factory interface for creating a decompressing {@link InputStream}. + */ + public interface DecompressingStreamFactory { + + /** + * Create a decompressing {@link InputStream} from an existing {@link InputStream}. + * + * @param inputStream the existing stream + * @return a stream that decompresses the contents of the existing stream + * @throws IOException + */ + public InputStream createInputStream(InputStream inputStream) throws IOException; + } + + /** + * Factory for creating decompressing input streams based on a filename and + * a {@link TextIO.CompressionType}. If the compression mode is AUTO, the filename + * is checked against known extensions to determine a compression type to use. + */ + protected static class FilenameBasedStreamFactory + implements FileBasedReader.DecompressingStreamFactory { + private String filename; + private TextIO.CompressionType compressionType; + + public FilenameBasedStreamFactory(String filename, TextIO.CompressionType compressionType) { + this.filename = filename; + this.compressionType = compressionType; + } + + protected TextIO.CompressionType getCompressionTypeForAuto() { + for (TextIO.CompressionType type : TextIO.CompressionType.values()) { + if (type.matches(filename) && type != TextIO.CompressionType.AUTO + && type != TextIO.CompressionType.UNCOMPRESSED) { + return type; + } + } + return TextIO.CompressionType.UNCOMPRESSED; + } + + @Override + public InputStream createInputStream(InputStream inputStream) throws IOException { + if (compressionType == TextIO.CompressionType.AUTO) { + return getCompressionTypeForAuto().createInputStream(inputStream); + } + return compressionType.createInputStream(inputStream); + } + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReader.java index f46eca2427356..afd75fbfd77f9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReader.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.runners.worker; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.util.IOChannelFactory; import com.google.cloud.dataflow.sdk.util.common.worker.ProgressTracker; import com.google.cloud.dataflow.sdk.util.common.worker.ProgressTrackerGroup; @@ -38,16 +39,20 @@ */ public class TextReader extends FileBasedReader { final boolean stripTrailingNewlines; + final TextIO.CompressionType compressionType; public TextReader(String filename, boolean stripTrailingNewlines, @Nullable Long startPosition, - @Nullable Long endPosition, Coder coder) { - this(filename, stripTrailingNewlines, startPosition, endPosition, coder, true); + @Nullable Long endPosition, Coder coder, TextIO.CompressionType compressionType) { + this(filename, stripTrailingNewlines, startPosition, endPosition, coder, true, + compressionType); } protected TextReader(String filename, boolean stripTrailingNewlines, @Nullable Long startPosition, - @Nullable Long endPosition, Coder coder, boolean useDefaultBufferSize) { + @Nullable Long endPosition, Coder coder, boolean useDefaultBufferSize, + TextIO.CompressionType compressionType) { super(filename, startPosition, endPosition, coder, useDefaultBufferSize); this.stripTrailingNewlines = stripTrailingNewlines; + this.compressionType = compressionType; } @Override @@ -99,7 +104,8 @@ private TextFileIterator newReaderIteratorForRangeWithStrictStart(IOChannelFacto SeekableByteChannel seeker = (SeekableByteChannel) reader; return new TextFileIterator( - new CopyableSeekableByteChannel(seeker), stripTrailingNewlines, startOffset, endOffset); + new CopyableSeekableByteChannel(seeker), stripTrailingNewlines, startOffset, endOffset, + new FileBasedReader.FilenameBasedStreamFactory(input, compressionType)); } class TextFileMultiIterator extends LazyMultiReaderIterator { @@ -124,7 +130,8 @@ class TextFileIterator extends FileBasedIterator { private ScanState state; TextFileIterator(CopyableSeekableByteChannel seeker, boolean stripTrailingNewlines, - long startOffset, @Nullable Long endOffset) throws IOException { + long startOffset, @Nullable Long endOffset, + FileBasedReader.DecompressingStreamFactory compressionStreamFactory) throws IOException { this(seeker, stripTrailingNewlines, startOffset, startOffset, endOffset, new ProgressTrackerGroup() { @Override @@ -132,13 +139,15 @@ protected void report(Integer lineLength) { notifyElementRead(lineLength.longValue()); } }.start(), - new ScanState(BUF_SIZE, !stripTrailingNewlines)); + new ScanState(BUF_SIZE, !stripTrailingNewlines), + compressionStreamFactory); } private TextFileIterator(CopyableSeekableByteChannel seeker, boolean stripTrailingNewlines, long startOffset, long offset, @Nullable Long endOffset, ProgressTracker tracker, - ScanState state) throws IOException { - super(seeker, startOffset, offset, endOffset, tracker); + ScanState state, FileBasedReader.DecompressingStreamFactory compressionStreamFactory) + throws IOException { + super(seeker, startOffset, offset, endOffset, tracker, compressionStreamFactory); this.stripTrailingNewlines = stripTrailingNewlines; this.state = state; @@ -149,7 +158,7 @@ private TextFileIterator(TextFileIterator it) throws IOException { // that it may hold bytes that have been read and now reside // in the read buffer (that is copied during cloning). this(it.seeker.copy(), it.stripTrailingNewlines, it.startOffset + it.state.totalBytesRead, - it.offset, it.endOffset, it.tracker.copy(), it.state.copy()); + it.offset, it.endOffset, it.tracker.copy(), it.state.copy(), it.compressionStreamFactory); } @Override @@ -262,7 +271,7 @@ public boolean readBytes(PushbackInputStream stream) throws IOException { * Consumes characters until a separator character is found or the * end of buffer is reached. * - * Updates the state to indicate the position of the separator + *

    Updates the state to indicate the position of the separator * character. If pos==len, no separator was found. * * @return the number of characters consumed. @@ -303,7 +312,7 @@ public int bytesBuffered() { /** * Copies data from the input buffer to the output buffer. * - * If keepNewlines==true, line-termination characters are included in the copy. + *

    If keepNewlines==true, line-termination characters are included in the copy. */ private void copyToOutputBuffer(ByteArrayOutputStream out) { int charsCopied = pos - start; @@ -317,7 +326,7 @@ private void copyToOutputBuffer(ByteArrayOutputStream out) { * Scans the input buffer to determine if a matched carriage return * has an accompanying linefeed and process the input buffer accordingly. * - * If keepNewlines==true and a linefeed character is detected, + *

    If keepNewlines==true and a linefeed character is detected, * it is included in the copy. * * @return the number of characters consumed diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactory.java index 4ec8973b87a12..132206ccbda7e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactory.java @@ -22,6 +22,7 @@ import com.google.api.services.dataflow.model.Source; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.ExecutionContext; @@ -58,6 +59,9 @@ static TextReader create(CloudObject spec, Coder coder, boolean useDef return new TextReader<>(filenameOrPattern, getBoolean(spec, PropertyNames.STRIP_TRAILING_NEWLINES, true), getLong(spec, PropertyNames.START_OFFSET, null), - getLong(spec, PropertyNames.END_OFFSET, null), coder, useDefaultBufferSize); + getLong(spec, PropertyNames.END_OFFSET, null), coder, + useDefaultBufferSize, + Enum.valueOf(TextIO.CompressionType.class, + getString(spec, PropertyNames.COMPRESSION_TYPE, "AUTO"))); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java index 26b260bd50f2d..0afe5ae411901 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java @@ -30,6 +30,7 @@ public class PropertyNames { public static final String CO_GBK_RESULT_SCHEMA = "co_gbk_result_schema"; public static final String COMBINE_FN = "combine_fn"; public static final String COMPONENT_ENCODINGS = "component_encodings"; + public static final String COMPRESSION_TYPE = "compression_type"; public static final String CUSTOM_SOURCE_FORMAT = "custom_source"; public static final String SOURCE_STEP_INPUT = "custom_source_step_input"; public static final String SOURCE_SPEC = "spec"; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java index 1cc3bf64030dd..ead2e779eb026 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java @@ -31,6 +31,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.TextualIntegerCoder; +import com.google.cloud.dataflow.sdk.io.TextIO.CompressionType; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.runners.DirectPipeline; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.EvaluationResults; @@ -64,6 +65,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.zip.GZIPOutputStream; /** * Tests for TextIO Read and Write transforms. @@ -397,4 +399,39 @@ public void testWriteWithoutValidationFlag() throws Exception { assertTrue(write.needsValidation()); assertFalse(write.withoutValidation().needsValidation()); } + + @Test + public void testCompressionTypeIsSet() throws Exception { + TextIO.Read.Bound read = TextIO.Read.from("gs://bucket/test"); + assertEquals(CompressionType.AUTO, read.getCompressionType()); + read = TextIO.Read.from("gs://bucket/test").withCompressionType(CompressionType.GZIP); + assertEquals(CompressionType.GZIP, read.getCompressionType()); + } + + @Test + public void testCompressedRead() throws Exception { + String[] lines = {"Irritable eagle", "Optimistic jay", "Fanciful hawk"}; + File tmpFile = tmpFolder.newFile("test"); + String filename = tmpFile.getPath(); + + List expected = new ArrayList<>(); + try (PrintStream writer = + new PrintStream(new GZIPOutputStream(new FileOutputStream(tmpFile)))) { + for (String line : lines) { + writer.println(line); + expected.add(line); + } + } + + DirectPipeline p = DirectPipeline.createForTest(); + + TextIO.Read.Bound read = + TextIO.Read.from(filename).withCompressionType(CompressionType.GZIP); + PCollection output = p.apply(read); + + EvaluationResults results = p.run(); + + assertThat(results.getPCollection(output), containsInAnyOrder(expected.toArray())); + tmpFile.delete(); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReaderTest.java new file mode 100644 index 0000000000000..3765af8a55901 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReaderTest.java @@ -0,0 +1,45 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.io.TextIO.CompressionType; +import com.google.cloud.dataflow.sdk.runners.worker.FileBasedReader.FilenameBasedStreamFactory; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for FileBasedReader. + */ +@RunWith(JUnit4.class) +public class FileBasedReaderTest { + + private void testGetStreamForAutoHelper(CompressionType expected, String filename) { + FilenameBasedStreamFactory factory = new FilenameBasedStreamFactory(filename, + CompressionType.AUTO); + CompressionType actual = factory.getCompressionTypeForAuto(); + Assert.assertEquals(expected, actual); + } + + @Test + public void testGetStreamForAuto() { + testGetStreamForAutoHelper(CompressionType.UNCOMPRESSED, "test"); + testGetStreamForAutoHelper(CompressionType.UNCOMPRESSED, "test.txt"); + testGetStreamForAutoHelper(CompressionType.GZIP, "test.gz"); + testGetStreamForAutoHelper(CompressionType.BZIP2, "test.bz2"); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactoryTest.java index 160eace961a72..a4a857c7ba8d5 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderFactoryTest.java @@ -25,6 +25,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.TextualIntegerCoder; +import com.google.cloud.dataflow.sdk.io.TextIO.CompressionType; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CloudObject; @@ -44,7 +45,8 @@ @RunWith(JUnit4.class) public class TextReaderFactoryTest { void runTestCreateTextReader(String filename, @Nullable Boolean stripTrailingNewlines, - @Nullable Long start, @Nullable Long end, CloudObject encoding, Coder coder) + @Nullable Long start, @Nullable Long end, CloudObject encoding, Coder coder, + CompressionType compressionType) throws Exception { CloudObject spec = CloudObject.forClassName("TextSource"); addString(spec, "filename", filename); @@ -57,6 +59,7 @@ void runTestCreateTextReader(String filename, @Nullable Boolean stripTrailingNew if (end != null) { addLong(spec, "end_offset", end); } + addString(spec, "compression_type", compressionType.toString()); Source cloudSource = new Source(); cloudSource.setSpec(spec); @@ -73,17 +76,19 @@ void runTestCreateTextReader(String filename, @Nullable Boolean stripTrailingNew Assert.assertEquals(start, textReader.startPosition); Assert.assertEquals(end, textReader.endPosition); Assert.assertEquals(coder, textReader.coder); + Assert.assertEquals(compressionType, textReader.compressionType); } @Test public void testCreatePlainTextReader() throws Exception { runTestCreateTextReader("/path/to/file.txt", null, null, null, - makeCloudEncoding("StringUtf8Coder"), StringUtf8Coder.of()); + makeCloudEncoding("StringUtf8Coder"), StringUtf8Coder.of(), CompressionType.UNCOMPRESSED); } @Test public void testCreateRichTextReader() throws Exception { runTestCreateTextReader("gs://bucket/path/to/file2.txt", false, 200L, 500L, - makeCloudEncoding("TextualIntegerCoder"), TextualIntegerCoder.of()); + makeCloudEncoding("TextualIntegerCoder"), TextualIntegerCoder.of(), + CompressionType.UNCOMPRESSED); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java index 5d2934c4535fc..95981a7eecccf 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java @@ -19,16 +19,20 @@ import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.greaterThan; import com.google.api.services.dataflow.model.ApproximateProgress; import com.google.cloud.dataflow.sdk.TestUtils; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.TextualIntegerCoder; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.io.TextIO.CompressionType; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; import com.google.cloud.dataflow.sdk.util.common.worker.Reader; +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -39,11 +43,13 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.io.OutputStream; import java.io.PrintStream; import java.util.ArrayList; import java.util.Arrays; import java.util.LinkedList; import java.util.List; +import java.util.zip.GZIPOutputStream; /** * Tests for TextReader. @@ -78,7 +84,8 @@ private File initTestFile() throws IOException { @Test public void testReadEmptyFile() throws Exception { TextReader textReader = - new TextReader<>(tmpFolder.newFile().getPath(), true, null, null, StringUtf8Coder.of()); + new TextReader<>(tmpFolder.newFile().getPath(), true, null, null, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); try (Reader.ReaderIterator iterator = textReader.iterator()) { Assert.assertFalse(iterator.hasNext()); } @@ -130,7 +137,8 @@ public void testStartPosition() throws Exception { { TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, 11L, null, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), false, 11L, null, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -146,7 +154,8 @@ public void testStartPosition() throws Exception { { TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, 20L, null, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), false, 20L, null, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -161,7 +170,8 @@ public void testStartPosition() throws Exception { { TextReader textReader = - new TextReader<>(tmpFile.getPath(), true, 0L, 20L, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), true, 0L, 20L, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -175,7 +185,8 @@ public void testStartPosition() throws Exception { { TextReader textReader = - new TextReader<>(tmpFile.getPath(), true, 1L, 20L, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), true, 1L, 20L, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -204,7 +215,8 @@ public void testUtf8Handling() throws Exception { // the first line if counting chars. So correct behavior is to return // just one line, since offsets are in chars, not codepoints. TextReader textReader = - new TextReader<>(tmpFile.getPath(), true, 0L, 3L, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), true, 0L, 3L, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -219,7 +231,8 @@ public void testUtf8Handling() throws Exception { // Starting location is mid-way into a codepoint. // Ensures we don't fail when skipping over an incomplete codepoint. TextReader textReader = - new TextReader<>(tmpFile.getPath(), true, 2L, null, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), true, 2L, null, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -246,7 +259,8 @@ private void testNewlineHandling(String separator, boolean stripNewlines) throws writer.close(); TextReader textReader = - new TextReader<>(tmpFile.getPath(), stripNewlines, null, null, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), stripNewlines, null, null, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -283,7 +297,8 @@ private void testStringPayload(String[] lines, String separator, boolean stripNe writer.close(); TextReader textReader = - new TextReader<>(tmpFile.getPath(), stripNewlines, null, null, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), stripNewlines, null, null, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -313,7 +328,8 @@ public void testCloneIteratorWithEndPositionAndFinalBytesInBuffer() throws Excep Long fileSize = tmpFile.length(); TextReader textReader = - new TextReader<>(tmpFile.getPath(), stripNewlines, null, fileSize, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), stripNewlines, null, fileSize, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); List actual = new ArrayList<>(); Reader.ReaderIterator iterator = textReader.iterator(); @@ -339,7 +355,8 @@ public void testNonStringCoders() throws Exception { writer.close(); TextReader textReader = - new TextReader<>(tmpFile.getPath(), true, null, null, TextualIntegerCoder.of()); + new TextReader<>(tmpFile.getPath(), true, null, null, TextualIntegerCoder.of(), + TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -358,7 +375,8 @@ public void testNonStringCoders() throws Exception { public void testGetApproximatePosition() throws Exception { File tmpFile = initTestFile(); TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, 0L, null, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), false, 0L, null, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); try (Reader.ReaderIterator iterator = textReader.iterator()) { ApproximateProgress progress = sourceProgressToCloudProgress(iterator.getProgress()); @@ -388,7 +406,8 @@ public void testUpdateStopPosition() throws Exception { // Illegal proposed stop position, no update. { TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, null, null, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), false, null, null, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -404,7 +423,8 @@ public void testUpdateStopPosition() throws Exception { // Successful update. { TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, null, null, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), false, null, null, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -431,7 +451,8 @@ public void testUpdateStopPosition() throws Exception { // Proposed stop position is before the current position, no update. { TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, null, null, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), false, null, null, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -457,7 +478,8 @@ public void testUpdateStopPosition() throws Exception { // Proposed stop position is after the current stop (end) position, no update. { TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, null, end, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), false, null, end, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -502,7 +524,8 @@ private void stopPositionTestInternal( // Read from source without split attempts. TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, startOffset, endOffset, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), false, startOffset, endOffset, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); try (TextReader.TextFileIterator iterator = (TextReader.TextFileIterator) textReader.iterator()) { @@ -514,7 +537,8 @@ private void stopPositionTestInternal( // Read the first half of the split. textReader = - new TextReader<>(tmpFile.getPath(), false, startOffset, stopOffset, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), false, startOffset, stopOffset, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); accumulatedRead = new StringBuilder(); try (TextReader.TextFileIterator iterator = @@ -527,7 +551,8 @@ private void stopPositionTestInternal( // Read the second half of the split. textReader = - new TextReader<>(tmpFile.getPath(), false, stopOffset, endOffset, StringUtf8Coder.of()); + new TextReader<>(tmpFile.getPath(), false, stopOffset, endOffset, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); accumulatedRead = new StringBuilder(); try (TextReader.TextFileIterator iterator = @@ -546,6 +571,112 @@ private ApproximateProgress createApproximateProgress( return new ApproximateProgress().setPosition(position); } + private OutputStream getOutputStreamForCompressionType(OutputStream stream, + CompressionType compressionType) throws IOException { + switch (compressionType) { + case GZIP: + return new GZIPOutputStream(stream); + case BZIP2: + return new BZip2CompressorOutputStream(stream); + case UNCOMPRESSED: + case AUTO: + return stream; + default: + Assert.fail("Unrecognized stream type"); + } + return stream; + } + + private File createFileWithCompressionType(String[] lines, String filename, + CompressionType compressionType) throws IOException { + File tmpFile = tmpFolder.newFile(filename); + PrintStream writer = new PrintStream( + getOutputStreamForCompressionType(new FileOutputStream(tmpFile), compressionType)); + for (String line : lines) { + writer.println(line); + } + writer.close(); + return tmpFile; + } + + private void testCompressionTypeHelper(String[] lines, String filename, + CompressionType outputCompressionType, CompressionType inputCompressionType) + throws IOException { + File tmpFile = createFileWithCompressionType(lines, filename, outputCompressionType); + + List expected = new ArrayList<>(); + for (String line : lines) { + expected.add(line); + } + + TextReader textReader = + new TextReader<>(tmpFile.getPath(), true, null, null, StringUtf8Coder.of(), + inputCompressionType); + + List actual = new ArrayList<>(); + try (Reader.ReaderIterator iterator = textReader.iterator()) { + while (iterator.hasNext()) { + actual.add(iterator.next()); + } + } + Assert.assertEquals(expected, actual); + tmpFile.delete(); + } + + @Test + public void testCompressionTypeOneFile() throws IOException { + String[] contents = {"Miserable pigeon", "Vulnerable sparrow", "Brazen crow"}; + // test AUTO compression type with different extensions + testCompressionTypeHelper(contents, "test.gz", CompressionType.GZIP, CompressionType.AUTO); + testCompressionTypeHelper(contents, "test.bz2", CompressionType.BZIP2, CompressionType.AUTO); + testCompressionTypeHelper(contents, "test.txt", CompressionType.UNCOMPRESSED, + CompressionType.AUTO); + testCompressionTypeHelper(contents, "test", CompressionType.UNCOMPRESSED, + CompressionType.AUTO); + // test GZIP, BZIP2, and UNCOMPRESSED + testCompressionTypeHelper(contents, "test.txt", CompressionType.GZIP, CompressionType.GZIP); + testCompressionTypeHelper(contents, "test.txt", CompressionType.BZIP2, CompressionType.BZIP2); + testCompressionTypeHelper(contents, "test.gz", CompressionType.UNCOMPRESSED, + CompressionType.UNCOMPRESSED); + } + + @Test + public void testCompressionTypeFileGlob() throws IOException { + String[][] contents = { + {"Miserable pigeon", "Vulnerable sparrow", "Brazen crow"}, + {"Timid osprey", "Lazy vulture"}, + {"Erratic finch", "Impressible parakeet"}, + }; + File[] files = { + createFileWithCompressionType(contents[0], "test.gz", CompressionType.GZIP), + createFileWithCompressionType(contents[1], "test.bz2", CompressionType.BZIP2), + createFileWithCompressionType(contents[2], "test.txt", CompressionType.UNCOMPRESSED), + }; + + List expected = new ArrayList<>(); + for (String[] fileContents : contents) { + for (String line : fileContents) { + expected.add(line); + } + } + + String path = tmpFolder.getRoot().getPath() + System.getProperty("file.separator") + "*"; + + TextReader textReader = + new TextReader<>(path, true, null, null, StringUtf8Coder.of(), CompressionType.AUTO); + + List actual = new ArrayList<>(); + try (Reader.ReaderIterator iterator = textReader.iterator()) { + while (iterator.hasNext()) { + actual.add(iterator.next()); + } + } + Assert.assertThat(actual, containsInAnyOrder(expected.toArray())); + for (File file : files) { + file.delete(); + } + } + // TODO: sharded filenames // TODO: reading from GCS } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java index b3cebad3e21e5..898d3ad1891af 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/IOFactoryTest.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.util; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.runners.worker.TextReader; import com.google.cloud.dataflow.sdk.util.common.worker.Reader; @@ -77,7 +78,8 @@ public void testMultiFileRead() throws Exception { TextReader reader = new TextReader<>( - tmpFolder.getRoot() + "/file*", true/* strip newlines */, null, null, StringUtf8Coder.of()); + tmpFolder.getRoot() + "/file*", true/* strip newlines */, null, null, StringUtf8Coder.of(), + TextIO.CompressionType.UNCOMPRESSED); Set records = new TreeSet<>(); try (Reader.ReaderIterator iterator = reader.iterator()) { From a741a31cc6455db62097121da4b5bb250690418f Mon Sep 17 00:00:00 2001 From: dhuo Date: Thu, 29 Jan 2015 14:30:37 -0800 Subject: [PATCH 0108/1541] Explicitly close the underlying readChannel before retries in GoogleCloudStorageReadChannel, catching and ignoring (but logging) SSLException since these indicate close() cannot complete normally, but the underlying resources have already been cleaned up so it's safe for us to proceed with the retry and simply abandon the existing readChannel. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85102060 --- .../gcsio/GoogleCloudStorageReadChannel.java | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java index a3d9b65347b2a..61b931561c370 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java @@ -38,6 +38,7 @@ import java.nio.channels.ReadableByteChannel; import java.nio.channels.SeekableByteChannel; import java.util.regex.Pattern; +import javax.net.ssl.SSLException; /** * Provides seekable read access to GCS. @@ -317,6 +318,27 @@ public int read(ByteBuffer buffer) long newPosition = currentPosition; currentPosition = -1; position(newPosition); + + // Before performing lazy seek, explicitly close the underlying channel if necessary, + // catching and ignoring SSLException since the retry indicates an error occurred, so + // there's a high probability that SSL connections would be broken in a way that + // causes close() itself to throw an exception, even though underlying sockets have + // already been cleaned up; close() on an SSLSocketImpl requires a shutdown handshake + // in order to shutdown cleanly, and if the connection has been broken already, then + // this is not possible, and the SSLSocketImpl was already responsible for performing + // local cleanup at the time the exception was raised. + if (lazySeekPending && readChannel != null) { + try { + readChannel.close(); + readChannel = null; + } catch (SSLException ssle) { + LOG.warn("Got SSLException on readChannel.close() before retry; ignoring it.", ssle); + readChannel = null; + } + // For "other" exceptions, we'll let it propagate out without setting readChannel to + // null, in case the caller is able to handle it and then properly try to close() + // again. + } performLazySeek(); } } From e3c4ded230309572b0a2fbba36ff7882c7fbf23e Mon Sep 17 00:00:00 2001 From: peihe Date: Thu, 29 Jan 2015 15:36:54 -0800 Subject: [PATCH 0109/1541] Windowing: Worker code for ReifyTsAndWindowsParDoFn and GroupAlsoByWindowsParDoFn. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85107960 --- .../worker/GroupAlsoByWindowsParDoFn.java | 22 ++++- .../sdk/runners/worker/ParDoFnFactory.java | 2 + .../ReifyTimestampAndWindowsParDoFn.java | 81 +++++++++++++++++++ .../dataflow/sdk/transforms/GroupByKey.java | 17 +--- .../util/ReifyTimestampAndWindowsDoFn.java | 41 ++++++++++ 5 files changed, 149 insertions(+), 14 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReifyTimestampAndWindowsParDoFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReifyTimestampAndWindowsDoFn.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java index 7e9e2bd89774e..9b924994b7958 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java @@ -24,11 +24,13 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.StreamingOptions; import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.DoFnInfo; import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn; import com.google.cloud.dataflow.sdk.util.PTuple; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.SerializableUtils; @@ -49,6 +51,7 @@ * NormalParDoFn, except that it gets deserialized differently. */ class GroupAlsoByWindowsParDoFn extends NormalParDoFn { + public static GroupAlsoByWindowsParDoFn create( PipelineOptions options, CloudObject cloudUserFn, @@ -93,7 +96,14 @@ public static GroupAlsoByWindowsParDoFn create( "Expected KvCoder for inputCoder, got: " + elemCoder.getClass().getName()); } - DoFnInfoFactory fnFactory = new DoFnInfoFactory() { + boolean isStreamingPipeline = false; + if (options instanceof StreamingOptions) { + isStreamingPipeline = ((StreamingOptions) options).isStreaming(); + } + + DoFnInfoFactory fnFactory; + if (isStreamingPipeline) { + fnFactory = new DoFnInfoFactory() { @Override public DoFnInfo createDoFnInfo() { return new DoFnInfo(StreamingGroupAlsoByWindowsDoFn.create( @@ -102,6 +112,16 @@ public DoFnInfo createDoFnInfo() { null); } }; + } else { + fnFactory = new DoFnInfoFactory() { + @Override + public DoFnInfo createDoFnInfo() { + return new DoFnInfo( + new GroupAlsoByWindowsDoFn((WindowFn) windowFn, elemCoder), + null); + } + }; + } return new GroupAlsoByWindowsParDoFn( options, fnFactory, stepName, executionContext, addCounterMutator); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactory.java index 23d4040685bfd..e2ae0841fd8e6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactory.java @@ -68,6 +68,8 @@ private ParDoFnFactory() {} GroupAlsoByWindowsParDoFn.class.getName()); predefinedParDoFnFactories.put("AssignWindowsDoFn", AssignWindowsParDoFn.class.getName()); + predefinedParDoFnFactories.put("ReifyTimestampAndWindowsDoFn", + ReifyTimestampAndWindowsParDoFn.class.getName()); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReifyTimestampAndWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReifyTimestampAndWindowsParDoFn.java new file mode 100644 index 0000000000000..4cab266e05068 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReifyTimestampAndWindowsParDoFn.java @@ -0,0 +1,81 @@ +/******************************************************************************* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.api.services.dataflow.model.MultiOutputInfo; +import com.google.api.services.dataflow.model.SideInputInfo; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.DoFnInfo; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.PTuple; +import com.google.cloud.dataflow.sdk.util.ReifyTimestampAndWindowsDoFn; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.CounterSet.AddCounterMutator; +import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; + +import java.util.Arrays; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * A wrapper around a ReifyTimestampAndWindowsDoFn. This class is the same as NormalParDoFn, except + * that it gets deserialized differently. + */ +@SuppressWarnings({"rawtypes", "unchecked"}) +public class ReifyTimestampAndWindowsParDoFn extends NormalParDoFn { + + public static ReifyTimestampAndWindowsParDoFn create( + PipelineOptions options, + CloudObject cloudUserFn, + String stepName, + @Nullable List sideInputInfos, + @Nullable List multiOutputInfos, + Integer numOutputs, + ExecutionContext executionContext, + CounterSet.AddCounterMutator addCounterMutator, + StateSampler sampler /* unused */) + throws Exception { + + final DoFn doFn = new ReifyTimestampAndWindowsDoFn(); + + DoFnInfoFactory fnFactory = new DoFnInfoFactory() { + @Override + public DoFnInfo createDoFnInfo() { + return new DoFnInfo(doFn, null); + } + }; + return new ReifyTimestampAndWindowsParDoFn( + options, fnFactory, stepName, executionContext, addCounterMutator); + } + private ReifyTimestampAndWindowsParDoFn( + PipelineOptions options, + DoFnInfoFactory fnFactory, + String stepName, + ExecutionContext executionContext, + AddCounterMutator addCounterMutator) { + super(options, + fnFactory, + PTuple.empty(), + Arrays.asList("output"), + stepName, + executionContext, + addCounterMutator); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index 9b56585fb9ea9..4d11723c99638 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -16,7 +16,6 @@ package com.google.cloud.dataflow.sdk.transforms; -import static com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.ValueWithMetadata; import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray; import com.google.cloud.dataflow.sdk.coders.Coder; @@ -24,12 +23,14 @@ import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.ValueWithMetadata; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.NonMergingWindowFn; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn; +import com.google.cloud.dataflow.sdk.util.ReifyTimestampAndWindowsDoFn; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder; import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; @@ -162,17 +163,7 @@ public PCollection>> apply( inputValueCoder, getInput().getWindowFn().windowCoder()); Coder>> outputKvCoder = KvCoder.of(keyCoder, outputValueCoder); - return input.apply(ParDo.of( - new DoFn, KV>>() { - @Override - public void processElement(ProcessContext c) { - KV kv = c.element(); - K key = kv.getKey(); - V value = kv.getValue(); - c.output(KV.of( - key, - WindowedValue.of(value, c.timestamp(), c.windows()))); - }})) + return input.apply(ParDo.of(new ReifyTimestampAndWindowsDoFn())) .setCoder(outputKvCoder); } } @@ -484,7 +475,7 @@ public PCollection>> applyHelper( .apply(new GroupByKeyOnly>()); if (!runnerSortsByTimestamp) { - // Sort each key's values by timestamp. GroupAlsoByWindow requires + // Sort each key's values by timestamp. GroupAlsoByWindow requires // its input to be sorted by timestamp. gbkOutput = gbkOutput.apply(new SortValuesByTimestamp()); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReifyTimestampAndWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReifyTimestampAndWindowsDoFn.java new file mode 100644 index 0000000000000..d3d0947f44215 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReifyTimestampAndWindowsDoFn.java @@ -0,0 +1,41 @@ +/******************************************************************************* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.values.KV; + +/** + * DoFn that makes timestamps and window assignments explicit in the value part of each key/value + * pair. + * + * @param the type of the keys of the input and output {@code PCollection}s + * @param the type of the values of the input {@code PCollection} + */ +public class ReifyTimestampAndWindowsDoFn + extends DoFn, KV>> { + @Override + public void processElement(ProcessContext c) + throws Exception { + KV kv = c.element(); + K key = kv.getKey(); + V value = kv.getValue(); + c.output(KV.of( + key, + WindowedValue.of(value, c.timestamp(), c.windows()))); + } +} From fbdd33e2675ef8b79f5534ebd15cf58b00a826e2 Mon Sep 17 00:00:00 2001 From: andersjohnson Date: Thu, 29 Jan 2015 16:21:28 -0800 Subject: [PATCH 0110/1541] Modify .travis.yml file also to test on osx, and add test_wordcount.sh, which it uses. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85112006 --- .travis.yml | 10 ++-- test_wordcount.sh | 119 ++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 124 insertions(+), 5 deletions(-) create mode 100644 test_wordcount.sh diff --git a/.travis.yml b/.travis.yml index 8cdc4ca197fe3..b9b15456657df 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,5 +1,9 @@ language: java +os: + - linux + - osx + notifications: email: recipients: @@ -7,13 +11,9 @@ notifications: on_success: change on_failure: always -jdk: - - oraclejdk8 - - oraclejdk7 - - openjdk7 - install: - mvn install clean -U -DskipTests=true script: - mvn verify -U + - ./test_wordcount.sh diff --git a/test_wordcount.sh b/test_wordcount.sh new file mode 100644 index 0000000000000..81e2e2a6baa8c --- /dev/null +++ b/test_wordcount.sh @@ -0,0 +1,119 @@ +#!/bin/bash + +# This script runs WordCount example locally in a few different ways. +# Specifically, all combinations of: +# a) using mvn exec, or java -cp with a bundled jar file; +# b) input filename with no directory component, with a relative directory, or +# with an absolute directory; AND +# c) input filename containing wildcards or not. +# +# The one optional parameter is a path from the directory containing the script +# to the directory containing the top-level (parent) pom.xml. If no parameter +# is provided, the script assumes that directory is equal to the directory +# containing the script itself. +# +# The exit-code of the script indicates success or a failure. + +set -e +set -o pipefail + +cd $(dirname $0) + +TOPDIR="." +if [[ $# -gt 0 ]] +then + TOPDIR="$1" +fi + +PASS=1 +JAR_FILE=$TOPDIR/examples/target/google-cloud-dataflow-java-examples-all-bundled-manual_build.jar + +function check_result_hash { + local name=$1 + local outfile_prefix=$2 + local expected=$3 + + local actual=$(md5sum $outfile_prefix-* | awk '{print $1}') + if [[ "$actual" != "$expected" ]] + then + echo "FAIL $name: Output hash mismatch. Got $actual, expected $expected." + PASS="" + else + echo "pass $name" + # Output files are left behind in /tmp + fi +} + +function get_outfile_prefix { + local name=$1 + mktemp --tmpdir=/tmp -u "$name.out.XXXXXXXXXX" +} + +function run_via_mvn { + local name=$1 + local input=$2 + local expected_hash=$3 + + local outfile_prefix="$(get_outfile_prefix "$name")" + local cmd='mvn exec:java -f '"$TOPDIR"'/pom.xml -pl examples \ + -Dexec.mainClass=com.google.cloud.dataflow.examples.WordCount \ + -Dexec.args="--runner=DirectPipelineRunner --input='"$input"' --output='"$outfile_prefix"'"' + echo "$name: Running $cmd" >&2 + sh -c "$cmd" + check_result_hash "$name" "$outfile_prefix" "$expected_hash" +} + +function run_bundled { + local name=$1 + local input=$2 + local expected_hash=$3 + + local outfile_prefix="$(get_outfile_prefix "$name")" + local cmd='java -cp '"$JAR_FILE"' \ + com.google.cloud.dataflow.examples.WordCount \ + --runner=DirectPipelineRunner \ + --input='"$input"' \ + --output='"$outfile_prefix" + echo "$name: Running $cmd" >&2 + sh -c "$cmd" + check_result_hash "$name" "$outfile_prefix" "$expected_hash" +} + +function run_all_ways { + local name=$1 + local input=$2 + local expected_hash=$3 + + run_via_mvn ${name}a $input $expected_hash + check_for_jar_file + run_bundled ${name}b $input $expected_hash +} + +function check_for_jar_file { + if [[ ! -f $JAR_FILE ]] + then + echo "Jar file $JAR_FILE not created" >&2 + exit 2 + fi +} + +# NOTE: We could still test via mvn exec if this fails for some reason. Perhaps +# we ought to do that. +echo "Generating bundled JAR file" >&2 +# NOTE: If this fails, run "mvn clean install" and try again. +mvn bundle:bundle -f $TOPDIR/pom.xml -pl examples +check_for_jar_file + +run_all_ways wordcount1 "LICENSE" f4af56cd6f6f127536d586a6adcefba1 +run_all_ways wordcount2 "./LICENSE" f4af56cd6f6f127536d586a6adcefba1 +run_all_ways wordcount3 "$PWD/LICENSE" f4af56cd6f6f127536d586a6adcefba1 +run_all_ways wordcount4 "L*N?E*" f4af56cd6f6f127536d586a6adcefba1 +run_all_ways wordcount5 "./LICE*N?E" f4af56cd6f6f127536d586a6adcefba1 +run_all_ways wordcount6 "$PWD/*LIC?NSE" f4af56cd6f6f127536d586a6adcefba1 + +if [[ ! "$PASS" ]] +then + echo "One or more tests FAILED." + exit 1 +fi +echo "All tests PASS" From 8b9f2b8bec35f02003a2da3d24929f3156cc0732 Mon Sep 17 00:00:00 2001 From: relax Date: Fri, 30 Jan 2015 14:19:55 -0800 Subject: [PATCH 0111/1541] Move path validation, credential validation, and file staging into separate classes that can be overridden. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85191128 --- .../options/DataflowPipelineDebugOptions.java | 70 ++++++++++++++++ .../DataflowPipelineWorkerPoolOptions.java | 2 +- .../dataflow/sdk/options/GcpOptions.java | 22 +++++- .../sdk/options/PipelineOptionsFactory.java | 16 ++++ .../sdk/runners/DataflowPipelineRunner.java | 46 +++-------- .../runners/DataflowPipelineTranslator.java | 15 ---- .../runners/dataflow/AvroIOTranslator.java | 15 ++-- .../runners/dataflow/TextIOTranslator.java | 18 ++--- .../runners/worker/DataflowWorkerHarness.java | 4 - .../dataflow/sdk/util/CredentialFactory.java | 29 +++++++ .../sdk/util/DataflowPathValidator.java | 79 +++++++++++++++++++ .../sdk/util/GcpCredentialFactory.java | 45 +++++++++++ .../cloud/dataflow/sdk/util/GcsStager.java | 48 +++++++++++ .../sdk/util/NoopCredentialFactory.java | 38 +++++++++ .../dataflow/sdk/util/NoopPathValidator.java | 52 ++++++++++++ .../cloud/dataflow/sdk/util/NoopStager.java | 37 +++++++++ .../dataflow/sdk/util/PathValidator.java | 54 +++++++++++++ .../cloud/dataflow/sdk/util/Stager.java | 29 +++++++ .../runners/DataflowPipelineRunnerTest.java | 5 +- 19 files changed, 544 insertions(+), 80 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CredentialFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowPathValidator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcpCredentialFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsStager.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NoopCredentialFactory.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NoopPathValidator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NoopStager.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PathValidator.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Stager.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java index 76de6e6dd8bf6..ce536e69a93d9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java @@ -16,6 +16,12 @@ package com.google.cloud.dataflow.sdk.options; +import com.google.cloud.dataflow.sdk.util.DataflowPathValidator; +import com.google.cloud.dataflow.sdk.util.GcsStager; +import com.google.cloud.dataflow.sdk.util.InstanceBuilder; +import com.google.cloud.dataflow.sdk.util.PathValidator; +import com.google.cloud.dataflow.sdk.util.Stager; + import java.util.List; /** @@ -64,4 +70,68 @@ public interface DataflowPipelineDebugOptions extends PipelineOptions { @Description("File for writing dataflow job descriptions") String getDataflowJobFile(); void setDataflowJobFile(String value); + + /** + * The name of the validator class used to validate path names. + */ + @Description("The validator class used to validate path names.") + @Default.Class(DataflowPathValidator.class) + Class getPathValidatorClass(); + void setPathValidatorClass(Class validatorClass); + + /** + * The validator class used to validate path names. + */ + @Description("The validator class used to validate path names.") + @Default.InstanceFactory(PathValidatorFactory.class) + PathValidator getPathValidator(); + void setPathValidator(PathValidator validator); + + /** + * The class used to stage files. + */ + @Description("The class used to stage files.") + @Default.Class(GcsStager.class) + Class getStagerClass(); + void setStagerClass(Class stagerClass); + + /** + * The stager instance used to stage files. + */ + @Description("The class use to stage packages.") + @Default.InstanceFactory(StagerFactory.class) + Stager getStager(); + void setStager(Stager stager); + + /** + * Creates a {@link PathValidator} object using the class specified in + * {@link #getPathValidatorClass()}. + */ + public static class PathValidatorFactory implements DefaultValueFactory { + @Override + public PathValidator create(PipelineOptions options) { + DataflowPipelineDebugOptions debugOptions = options.as(DataflowPipelineDebugOptions.class); + return InstanceBuilder.ofType(PathValidator.class) + .fromClass(debugOptions.getPathValidatorClass()) + .fromFactoryMethod("fromOptions") + .withArg(PipelineOptions.class, options) + .build(); + } + } + + /** + * Creates a {@link Stager} object using the class specified in + * {@link #getStagerClass()}. + */ + public static class StagerFactory implements DefaultValueFactory { + @Override + public Stager create(PipelineOptions options) { + DataflowPipelineDebugOptions debugOptions = options.as(DataflowPipelineDebugOptions.class); + return InstanceBuilder.ofType(Stager.class) + .fromClass(debugOptions.getStagerClass()) + .fromFactoryMethod("fromOptions") + .withArg(PipelineOptions.class, options) + .build(); + } + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java index 25653495cc0ea..ddcdc695d23e1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java @@ -21,7 +21,7 @@ /** * Options which are used to configure the Dataflow pipeline worker pool. */ -public interface DataflowPipelineWorkerPoolOptions { +public interface DataflowPipelineWorkerPoolOptions extends PipelineOptions { /** * Disk source image to use by VMs for jobs. * @see Compute Engine Images diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java index 114bc0d7c5594..0d824405b9075 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java @@ -17,7 +17,9 @@ package com.google.cloud.dataflow.sdk.options; import com.google.api.client.auth.oauth2.Credential; -import com.google.cloud.dataflow.sdk.util.Credentials; +import com.google.cloud.dataflow.sdk.util.CredentialFactory; +import com.google.cloud.dataflow.sdk.util.GcpCredentialFactory; +import com.google.cloud.dataflow.sdk.util.InstanceBuilder; import com.fasterxml.jackson.annotation.JsonIgnore; @@ -126,6 +128,12 @@ public String create(PipelineOptions options) { String getCredentialId(); void setCredentialId(String value); + @Description("The factory class used to create oauth credentials") + @Default.Class(GcpCredentialFactory.class) + Class getCredentialFactoryClass(); + void setCredentialFactoryClass( + Class credentialFactoryClass); + /** Alternative Google Cloud Platform Credential. */ @JsonIgnore @Description("Google Cloud Platform user credentials.") @@ -134,14 +142,20 @@ public String create(PipelineOptions options) { void setGcpCredential(Credential value); /** - * Attempts to load the user credentials. See - * {@link Credentials#getCredential(GcpOptions)} for more details. + * Attempts to load the GCP credentials. See + * {@link CredentialFactory#getCredential()} for more details. */ public static class GcpUserCredentialsFactory implements DefaultValueFactory { @Override public Credential create(PipelineOptions options) { + GcpOptions gcpOptions = options.as(GcpOptions.class); try { - return Credentials.getCredential(options.as(GcpOptions.class)); + CredentialFactory factory = InstanceBuilder.ofType(CredentialFactory.class) + .fromClass(gcpOptions.getCredentialFactoryClass()) + .fromFactoryMethod("fromOptions") + .withArg(PipelineOptions.class, options) + .build(); + return factory.getCredential(); } catch (IOException | GeneralSecurityException e) { throw new RuntimeException("Unable to obtain credential", e); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java index b20771a7c45f0..24f8e15069108 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java @@ -522,6 +522,22 @@ public static DataflowWorkerHarnessOptions createFromSystemProperties() { if (System.getProperties().containsKey("job_id")) { options.setJobId(System.getProperty("job_id")); } + if (System.getProperties().containsKey("path_validator_class")) { + try { + options.setPathValidatorClass((Class) Class.forName( + System.getProperty("path_validator_class"))); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Unable to find validator class", e); + } + } + if (System.getProperties().containsKey("credential_factory_class")) { + try { + options.setCredentialFactoryClass((Class) Class.forName( + System.getProperty("credential_factory_class"))); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Unable to find credential factory class", e); + } + } return options; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index b6003b2001064..4da69ac460e5d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -29,13 +29,11 @@ import com.google.cloud.dataflow.sdk.transforms.GroupByKey; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo; -import com.google.cloud.dataflow.sdk.util.GcsUtil; import com.google.cloud.dataflow.sdk.util.IOChannelUtils; import com.google.cloud.dataflow.sdk.util.MonitoringUtil; -import com.google.cloud.dataflow.sdk.util.PackageUtil; +import com.google.cloud.dataflow.sdk.util.PathValidator; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.Transport; -import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PInput; import com.google.cloud.dataflow.sdk.values.POutput; @@ -71,12 +69,6 @@ public class DataflowPipelineRunner extends PipelineRunner /** Provided configuration options. */ private final DataflowPipelineOptions options; - /** The directory on GCS where files should be uploaded. */ - private final GcsPath gcsStaging; - - /** The directory on GCS where temporary files are stored. */ - private final GcsPath gcsTemp; - /** Client for the Dataflow service. This is used to actually submit jobs. */ private final Dataflow dataflowClient; @@ -111,15 +103,8 @@ public static DataflowPipelineRunner fromOptions(PipelineOptions options) { "Missing required values: " + Joiner.on(',').join(missing)); } - Preconditions.checkArgument(!(Strings.isNullOrEmpty(dataflowOptions.getTempLocation()) - && Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())), - "Missing required value: at least one of tempLocation or stagingLocation must be set."); - if (Strings.isNullOrEmpty(dataflowOptions.getTempLocation())) { - dataflowOptions.setTempLocation(dataflowOptions.getStagingLocation()); - } else if (Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())) { - dataflowOptions.setStagingLocation( - GcsPath.fromUri(dataflowOptions.getTempLocation()).resolve("staging").toString()); - } + PathValidator validator = dataflowOptions.getPathValidator(); + validator.validateAndUpdateOptions(); if (dataflowOptions.getFilesToStage() == null) { dataflowOptions.setFilesToStage(detectClassPathResourcesToStage( @@ -144,8 +129,6 @@ public static DataflowPipelineRunner fromOptions(PipelineOptions options) { private DataflowPipelineRunner(DataflowPipelineOptions options) { this.options = options; this.dataflowClient = options.getDataflowClient(); - this.gcsTemp = GcsPath.fromUri(options.getTempLocation()); - this.gcsStaging = GcsPath.fromUri(options.getStagingLocation()); this.translator = DataflowPipelineTranslator.fromOptions(options); // (Re-)register standard IO factories. Clobbers any prior credentials. @@ -176,10 +159,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { LOG.info("Executing pipeline on the Dataflow Service, which will have billing implications " + "related to Google Compute Engine usage and other Google Cloud Services."); - GcsUtil gcsUtil = options.getGcsUtil(); - List packages = - PackageUtil.stageClasspathElementsToGcs(gcsUtil, options.getFilesToStage(), gcsStaging); - + List packages = options.getStager().stageFiles(); Job newJob = translator.translate(pipeline, packages); String version = DataflowReleaseInfo.getReleaseInfo().getVersion(); @@ -188,7 +168,11 @@ public DataflowPipelineJob run(Pipeline pipeline) { newJob.getEnvironment().setUserAgent(DataflowReleaseInfo.getReleaseInfo()); // The Dataflow Service may write to the temporary directory directly, so // must be verified. - newJob.getEnvironment().setTempStoragePrefix(verifyGcsPath(gcsTemp).toResourceName()); + if (!Strings.isNullOrEmpty(options.getTempLocation())) { + DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); + newJob.getEnvironment().setTempStoragePrefix( + dataflowOptions.getPathValidator().verifyGcsPath(options.getTempLocation())); + } newJob.getEnvironment().setDataset(options.getTempDatasetId()); newJob.getEnvironment().setClusterManagerApiService( options.getClusterManagerApi().getApiServiceName()); @@ -269,18 +253,6 @@ public void setHooks(DataflowPipelineRunnerHooks hooks) { @Override public String toString() { return "DataflowPipelineRunner#" + hashCode(); } - /** - * Verifies that a path can be used by the Dataflow Service API. - * @return the supplied path - */ - public static GcsPath verifyGcsPath(GcsPath path) { - Preconditions.checkArgument(path.isAbsolute(), - "Must provide absolute paths for Dataflow"); - Preconditions.checkArgument(!path.getObject().contains("//"), - "Dataflow Service does not allow objects with consecutive slashes"); - return path; - } - /** * Attempts to detect all the resources the class loader has access to. This does not recurse * to class loader parents stopping it from pulling in resources from the system class loader. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index 54dd9326c67f6..b072f64fd576b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -73,7 +73,6 @@ import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.SerializableUtils; import com.google.cloud.dataflow.sdk.util.WindowedValue; -import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionTuple; import com.google.cloud.dataflow.sdk.values.PCollectionView; @@ -243,15 +242,6 @@ public interface TranslationContext { */ public void addInput(String name, PInput value); - /** - * Adds an input with the given name and value to the current - * Dataflow step. - * - *

    This applies any verification of paths required by the Dataflow - * service. - */ - public void addInput(String name, GcsPath path); - /** * Adds an input which is a dictionary of strings to objects. */ @@ -586,11 +576,6 @@ public void addInput(String name, PInput value) { } } - @Override - public void addInput(String name, GcsPath path) { - addInput(name, DataflowPipelineRunner.verifyGcsPath(path).toResourceName()); - } - @Override public void addOutput(String name, PValue value) { Coder coder; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java index f191b25ba86ee..e23afad8357b9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java @@ -22,9 +22,9 @@ import com.google.cloud.dataflow.sdk.io.ShardNameTemplate; import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator; import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext; +import com.google.cloud.dataflow.sdk.util.PathValidator; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.WindowedValue; -import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; /** * Avro transform support code for the Dataflow backend. @@ -51,11 +51,11 @@ private void translateReadHelper( throw new IllegalArgumentException("AvroIO not supported in streaming mode."); } - // Only GCS paths are permitted for filepatterns in the DataflowPipelineRunner. - GcsPath gcsPath = GcsPath.fromUri(transform.getFilepattern()); + PathValidator validator = context.getPipelineOptions().getPathValidator(); + String filepattern = validator.validateInputFilePatternSupported(transform.getFilepattern()); context.addStep(transform, "ParallelRead"); context.addInput(PropertyNames.FORMAT, "avro"); - context.addInput(PropertyNames.FILEPATTERN, gcsPath); + context.addInput(PropertyNames.FILEPATTERN, filepattern); context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); // TODO: Orderedness? } @@ -77,8 +77,9 @@ public void translate( private void translateWriteHelper( AvroIO.Write.Bound transform, TranslationContext context) { - // Only GCS paths are permitted for filepatterns in the DataflowPipelineRunner. - GcsPath gcsPath = GcsPath.fromUri(transform.getFilenamePrefix()); + PathValidator validator = context.getPipelineOptions().getPathValidator(); + String filenamePrefix = validator.validateOutputFilePrefixSupported( + transform.getFilenamePrefix()); context.addStep(transform, "ParallelWrite"); context.addInput(PropertyNames.PARALLEL_INPUT, transform.getInput()); @@ -98,7 +99,7 @@ private void translateWriteHelper( } context.addInput(PropertyNames.FORMAT, "avro"); - context.addInput(PropertyNames.FILENAME_PREFIX, gcsPath); + context.addInput(PropertyNames.FILENAME_PREFIX, filenamePrefix); context.addInput(PropertyNames.SHARD_NAME_TEMPLATE, transform.getShardTemplate()); context.addInput(PropertyNames.FILENAME_SUFFIX, transform.getFilenameSuffix()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java index 9cf059bf53126..aa80248f3fcba 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java @@ -21,9 +21,9 @@ import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TransformTranslator; import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext; +import com.google.cloud.dataflow.sdk.util.PathValidator; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.WindowedValue; -import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; /** * TextIO transform support code for the Dataflow backend. @@ -49,16 +49,14 @@ private void translateReadHelper( throw new IllegalArgumentException("TextIO not supported in streaming mode."); } - // Validate the provided GCS path. - GcsPath gcsPath = GcsPath.fromUri(transform.getFilepattern()); - Preconditions.checkArgument( - context.getPipelineOptions().getGcsUtil().isGcsPatternSupported(gcsPath.getObject())); + PathValidator validator = context.getPipelineOptions().getPathValidator(); + String filepattern = validator.validateInputFilePatternSupported(transform.getFilepattern()); context.addStep(transform, "ParallelRead"); // TODO: How do we want to specify format and // format-specific properties? context.addInput(PropertyNames.FORMAT, "text"); - context.addInput(PropertyNames.FILEPATTERN, gcsPath); + context.addInput(PropertyNames.FILEPATTERN, filepattern); context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); context.addInput(PropertyNames.VALIDATE_SOURCE, transform.needsValidation()); context.addInput(PropertyNames.COMPRESSION_TYPE, transform.getCompressionType().toString()); @@ -86,8 +84,10 @@ private void translateWriteHelper( throw new IllegalArgumentException("TextIO not supported in streaming mode."); } - // Only GCS paths are permitted for filepatterns in the DataflowPipelineRunner. - GcsPath gcsPath = GcsPath.fromUri(transform.getFilenamePrefix()); + PathValidator validator = context.getPipelineOptions().getPathValidator(); + String filenamePrefix = validator.validateOutputFilePrefixSupported( + transform.getFilenamePrefix()); + context.addStep(transform, "ParallelWrite"); context.addInput(PropertyNames.PARALLEL_INPUT, transform.getInput()); @@ -109,7 +109,7 @@ private void translateWriteHelper( // TODO: How do we want to specify format and // format-specific properties? context.addInput(PropertyNames.FORMAT, "text"); - context.addInput(PropertyNames.FILENAME_PREFIX, gcsPath); + context.addInput(PropertyNames.FILENAME_PREFIX, filenamePrefix); context.addInput(PropertyNames.SHARD_NAME_TEMPLATE, transform.getShardNameTemplate()); context.addInput(PropertyNames.FILENAME_SUFFIX, transform.getFilenameSuffix()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java index f2601a6b5fa02..d069b63d67a6a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java @@ -20,7 +20,6 @@ import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudTime; import com.google.api.client.util.Lists; -import com.google.api.client.util.Preconditions; import com.google.api.services.dataflow.Dataflow; import com.google.api.services.dataflow.model.LeaseWorkItemRequest; import com.google.api.services.dataflow.model.LeaseWorkItemResponse; @@ -147,9 +146,6 @@ static DataflowWorker create(DataflowWorkerHarnessOptions options) { MDC.put(DataflowWorkerLoggingFormatter.MDC_DATAFLOW_WORKER_ID, options.getWorkerId()); options.setAppName(APPLICATION_NAME); - Preconditions.checkState(options.getGcpCredential() != null, - "Failed to obtain GCP credential in worker."); - // Configure standard IO factories. IOChannelUtils.setIOFactory("gs", new GcsIOChannelFactory(options)); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CredentialFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CredentialFactory.java new file mode 100644 index 0000000000000..0bc3012a5f202 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CredentialFactory.java @@ -0,0 +1,29 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.auth.oauth2.Credential; + +import java.io.IOException; +import java.security.GeneralSecurityException; + +/** + * Construct an oauth credential to be used by the SDK and the SDK workers. + */ +public interface CredentialFactory { + public Credential getCredential() throws IOException, GeneralSecurityException; +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowPathValidator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowPathValidator.java new file mode 100644 index 0000000000000..ab79906eb77d5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DataflowPathValidator.java @@ -0,0 +1,79 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.common.base.Strings; + +/** + * GCP implementation of {@link PathValidator}. Only GCS paths are allowed. + */ +public class DataflowPathValidator implements PathValidator { + + private DataflowPipelineOptions dataflowOptions; + + private DataflowPathValidator(DataflowPipelineOptions options) { + this.dataflowOptions = options; + } + + public static DataflowPathValidator fromOptions(PipelineOptions options) { + return new DataflowPathValidator(options.as(DataflowPipelineOptions.class)); + } + + @Override + public void validateAndUpdateOptions() { + Preconditions.checkArgument(!(Strings.isNullOrEmpty(dataflowOptions.getTempLocation()) + && Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())), + "Missing required value: at least one of tempLocation or stagingLocation must be set."); + if (Strings.isNullOrEmpty(dataflowOptions.getTempLocation())) { + dataflowOptions.setTempLocation(dataflowOptions.getStagingLocation()); + } else if (Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())) { + dataflowOptions.setStagingLocation( + GcsPath.fromUri(dataflowOptions.getTempLocation()).resolve("staging").toString()); + } + } + + @Override + public String validateInputFilePatternSupported(String filepattern) { + GcsPath gcsPath = GcsPath.fromUri(filepattern); + Preconditions.checkArgument( + dataflowOptions.getGcsUtil().isGcsPatternSupported(gcsPath.getObject())); + return verifyGcsPath(filepattern); + } + + @Override + public String validateOutputFilePrefixSupported(String filePrefix) { + return verifyGcsPath(filePrefix); + } + + /** + * Verifies that a path can be used by the Dataflow Service API. + * @return the supplied path + */ + @Override + public String verifyGcsPath(String path) { + GcsPath gcsPath = GcsPath.fromUri(path); + Preconditions.checkArgument(gcsPath.isAbsolute(), + "Must provide absolute paths for Dataflow"); + Preconditions.checkArgument(!gcsPath.getObject().contains("//"), + "Dataflow Service does not allow objects with consecutive slashes"); + return gcsPath.toResourceName(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcpCredentialFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcpCredentialFactory.java new file mode 100644 index 0000000000000..2999a56e4368b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcpCredentialFactory.java @@ -0,0 +1,45 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.auth.oauth2.Credential; +import com.google.cloud.dataflow.sdk.options.GcpOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; + +import java.io.IOException; +import java.security.GeneralSecurityException; + +/** + * Construct an oauth credential to be used by the SDK and the SDK workers. + * Returns a GCP credential. + */ +public class GcpCredentialFactory implements CredentialFactory { + private GcpOptions options; + + private GcpCredentialFactory(GcpOptions options) { + this.options = options; + } + + public static GcpCredentialFactory fromOptions(PipelineOptions options) { + return new GcpCredentialFactory(options.as(GcpOptions.class)); + } + + @Override + public Credential getCredential() throws IOException, GeneralSecurityException { + return Credentials.getCredential(options); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsStager.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsStager.java new file mode 100644 index 0000000000000..718071d2afab1 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsStager.java @@ -0,0 +1,48 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.common.base.Preconditions; + +import java.util.List; + +/** + * Utility class for staging files to GCS. + */ +public class GcsStager implements Stager { + private DataflowPipelineOptions options; + + private GcsStager(DataflowPipelineOptions options) { + this.options = options; + } + + public static GcsStager fromOptions(PipelineOptions options) { + return new GcsStager(options.as(DataflowPipelineOptions.class)); + } + + @Override + public List stageFiles() { + Preconditions.checkNotNull(options.getStagingLocation()); + return PackageUtil.stageClasspathElementsToGcs( + options.getGcsUtil(), options.getFilesToStage(), + GcsPath.fromUri(options.getStagingLocation())); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NoopCredentialFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NoopCredentialFactory.java new file mode 100644 index 0000000000000..5292f8e5c2d53 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NoopCredentialFactory.java @@ -0,0 +1,38 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.auth.oauth2.Credential; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; + +import java.io.IOException; +import java.security.GeneralSecurityException; + +/** + * Construct an oauth credential to be used by the SDK and the SDK workers. + * Always returns a null Credential object. + */ +public class NoopCredentialFactory implements CredentialFactory { + public static NoopCredentialFactory fromOptions(PipelineOptions options) { + return new NoopCredentialFactory(); + } + + @Override + public Credential getCredential() throws IOException, GeneralSecurityException { + return null; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NoopPathValidator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NoopPathValidator.java new file mode 100644 index 0000000000000..9920126b92c63 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NoopPathValidator.java @@ -0,0 +1,52 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.options.PipelineOptions; + +/** + * Noop implementation of {@link PathValidator}. All paths are allowed and returned unchanged. + */ +public class NoopPathValidator implements PathValidator { + + private NoopPathValidator() { + } + + public static PathValidator fromOptions(PipelineOptions options) { + return new NoopPathValidator(); + } + + @Override + public void validateAndUpdateOptions() { + return; + } + + @Override + public String validateInputFilePatternSupported(String filepattern) { + return filepattern; + } + + @Override + public String validateOutputFilePrefixSupported(String filePrefix) { + return filePrefix; + } + + @Override + public String verifyGcsPath(String path) { + return path; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NoopStager.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NoopStager.java new file mode 100644 index 0000000000000..b54018c49c4d9 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NoopStager.java @@ -0,0 +1,37 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; + +import java.util.ArrayList; +import java.util.List; + +/** + * Do-nothing stager class. stageFiles() does nothing and returns an empty list of packages. + */ +class NoopStager implements Stager { + public static NoopStager fromOptions(PipelineOptions options) { + return new NoopStager(); + } + + @Override + public List stageFiles() { + return new ArrayList(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PathValidator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PathValidator.java new file mode 100644 index 0000000000000..c1a65ab7b57dd --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PathValidator.java @@ -0,0 +1,54 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.options.PipelineOptions; + +/** + * Interface for controlling validation of paths. + */ +public interface PathValidator { + /** + * Validates paths in the current {@link PipelineOptions} object. May modify the + * options object. + */ + public void validateAndUpdateOptions(); + + /** + * Validate that a file pattern is conforming. + * + * @param filepattern The file pattern to verify. + * @return The post-validation filepattern. + */ + public String validateInputFilePatternSupported(String filepattern); + + /** + * Validate that an output file prefix is conforming. + * + * @param filePrefix the file prefix to verify. + * @return The post-validation filePrefix. + */ + public String validateOutputFilePrefixSupported(String filePrefix); + + /** + * Validate that a GCS path is conforming. + * + * @param path The GCS path to verify. + * @return The post-validation path. + */ + public String verifyGcsPath(String path); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Stager.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Stager.java new file mode 100644 index 0000000000000..f4a1a6035a121 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Stager.java @@ -0,0 +1,29 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.services.dataflow.model.DataflowPackage; + +import java.util.List; + +/** + * Interface for staging files needed for running a Dataflow pipeline. + */ +public interface Stager { + /* Stage files and return a list of packages. */ + public List stageFiles(); +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java index b3f80891244b3..cd5902d3b12ce 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java @@ -133,8 +133,7 @@ private DataflowPipelineOptions buildPipelineOptions( ArgumentCaptor jobCaptor) throws IOException { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setProject("someProject"); - options.setTempLocation(DataflowPipelineRunner.verifyGcsPath( - GcsPath.fromComponents("somebucket", "some/path")).toString()); + options.setTempLocation(GcsPath.fromComponents("somebucket", "some/path").toString()); // Set FILES_PROPERTY to empty to prevent a default value calculated from classpath. options.setFilesToStage(new LinkedList()); options.setDataflowClient(buildMockDataflow(jobCaptor)); @@ -483,7 +482,7 @@ public void testTransformTranslator() throws IOException { DataflowPipelineTranslator translator = DataflowPipelineRunner .fromOptions(options).getTranslator(); - translator.registerTransformTranslator( + DataflowPipelineTranslator.registerTransformTranslator( TestTransform.class, new DataflowPipelineTranslator.TransformTranslator() { @SuppressWarnings("unchecked") From 39a821051ffc296d02dd88c9f4cd571016224f58 Mon Sep 17 00:00:00 2001 From: peihe Date: Fri, 30 Jan 2015 14:46:22 -0800 Subject: [PATCH 0112/1541] FIX: 1. not depends on UnsupportedOperationException. 2. removeWindow from activeWindowManager in PartitionBufferingWindowSet. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85193656 --- .../cloud/dataflow/sdk/util/BufferingWindowSet.java | 12 +++++++++--- .../sdk/util/PartitionBufferingWindowSet.java | 4 ++-- .../sdk/util/StreamingGroupAlsoByWindowsDoFn.java | 11 ++++------- 3 files changed, 15 insertions(+), 12 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java index 33cfe80a9231e..2317072b91b43 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferingWindowSet.java @@ -97,6 +97,13 @@ public void put(W window, V value) throws Exception { @Override public void remove(W window) throws Exception { + Set subWindows = mergeTree.get(window); + for (W w : subWindows) { + context.context.stepContext.deleteTagList( + bufferTag(w, windowFn.windowCoder(), inputCoder)); + } + context.context.stepContext.deleteTagList( + bufferTag(window, windowFn.windowCoder(), inputCoder)); mergeTree.remove(window); activeWindowManager.removeWindow(window); } @@ -113,7 +120,8 @@ public void merge(Collection otherWindows, W newWindow) throws Exception { } subWindows.addAll(mergeTree.get(other)); subWindows.add(other); - remove(other); + mergeTree.remove(other); + activeWindowManager.removeWindow(other); } mergeTree.put(newWindow, subWindows); activeWindowManager.addWindow(newWindow); @@ -156,8 +164,6 @@ protected Iterable finalValue(W window) throws Exception { for (V item : items) { toEmit.add(item); } - context.context.stepContext.deleteTagList(bufferTag( - curWindow, windowFn.windowCoder(), inputCoder)); } return toEmit; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java index f2b96c10434ec..3c20b5f1f3cbc 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java @@ -56,8 +56,8 @@ public void put(W window, V value) throws Exception { @Override public void remove(W window) throws Exception { - CodedTupleTag tag = bufferTag(window, windowFn.windowCoder(), inputCoder); - context.context.stepContext.deleteTagList(tag); + context.context.stepContext.deleteTagList( + bufferTag(window, windowFn.windowCoder(), inputCoder)); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java index 3496bef97953f..9868b33a2db1c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.PartitioningWindowFn; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.base.Preconditions; import java.io.IOException; @@ -94,13 +95,9 @@ public void processElement(ProcessContext processContext) throws Exception { .mergeWindows(new AbstractWindowSet.WindowMergeContext(windowSet, windowFn)); W window = WindowUtils.windowFromString(timer.tag(), windowFn.windowCoder()); - boolean windowExists; - try { - windowExists = windowSet.contains(window); - } catch (UnsupportedOperationException e) { - windowExists = true; - } - if (windowExists) { + + if ((windowFn instanceof PartitioningWindowFn) || windowSet.contains(window)) { + Preconditions.checkState(!timer.timestamp().isBefore(window.maxTimestamp())); windowSet.markCompleted(window); windowSet.flush(); } From 62b0dd48115255debb926b750435d8e68feda715 Mon Sep 17 00:00:00 2001 From: relax Date: Mon, 2 Feb 2015 12:09:23 -0800 Subject: [PATCH 0113/1541] Allows users of PubsubIO to specify which pubsub labels are used to propagate record timestamps and record ids. Strictly SDK changes, cannot yet be used. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85350708 --- .../cloud/dataflow/sdk/io/PubsubIO.java | 181 +++++++++++++++++- .../runners/dataflow/PubsubIOTranslator.java | 13 ++ .../dataflow/sdk/util/PropertyNames.java | 3 + 3 files changed, 188 insertions(+), 9 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index 4a379920249e7..060844699b550 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -153,6 +153,10 @@ public static Bound named(String name) { *

  1. Must end with a letter or a number.
  2. *
  3. Cannot begin with 'goog' prefix.
  4. * + * + * Dataflow will start reading data published on this topic from the time the pipeline is + * started. Any data published on the topic before the pipeline is started will not be read + * by Dataflow. */ public static Bound topic(String topic) { return new Bound().topic(topic); @@ -180,6 +184,46 @@ public static Bound subscription(String subscription) { return new Bound().subscription(subscription); } + /** + * Creates and returns a PubsubIO.Read PTransform where record timestamps are expected + * to be provided using the PubSub labeling API. The {@code } parameter + * specifies the label name. The label value sent to PubsSub is a numerical value representing + * the number of milliseconds since the Unix epoch. For example, if using the joda time classes, + * org.joda.time.Instant.getMillis() returns the correct value for this label. + * + * When this feature is used, + * + * If {@code } is not provided, the system will generate record timestamps the + * first time it sees each record. All windowing will be done relative to these timestamps. + * Windows are closed based on an estimate of when this source has finished producing data for + * a timestamp range, which means that late data can arrive after a window has been closed. The + * {#dropLateData} field allows you to control what to do with late data. + */ + public static Bound timestampLabel(String timestampLabel) { + return new Bound().timestampLabel(timestampLabel); + } + + /** + * If true, then late-arriving data from this source will be dropped. + */ + public static Bound dropLateData(boolean dropLateData) { + return new Bound().dropLateData(dropLateData); + } + + /** + * Creates and returns a PubSubIO.Read PTransform where unique record identifiers are + * expected to be provided using the PubSub labeling API. The {@code } parameter + * specifies the label name. The label value sent to PubSub can be any string value that + * uniquely identifies this record. + * + * if idLabel is not provided, Dataflow cannot guarantee that no duplicate data will be + * delivered on the PubSub stream. In this case, deduplication of the stream will be + * stricly best effort. + */ + public static Bound idLabel(String idLabel) { + return new Bound().idLabel(idLabel); + } + /** * A PTransform that reads from a PubSub source and returns * a unbounded PCollection containing the items from the stream. @@ -191,10 +235,18 @@ public static class Bound String topic; /** The Pubsub subscription to read from. */ String subscription; + /** The Pubsub label to read timestamps from. */ + String timestampLabel; + Boolean dropLateData; + /** The Pubsub label to read ids from. */ + String idLabel; + + Bound() { + this.dropLateData = true; + } - Bound() {} - - Bound(String name, String subscription, String topic) { + Bound(String name, String subscription, String topic, String timestampLabel, + boolean dropLateData, String idLabel) { super(name); if (subscription != null) { Validator.validateSubscriptionName(subscription); @@ -204,18 +256,57 @@ public static class Bound } this.subscription = subscription; this.topic = topic; + this.timestampLabel = timestampLabel; + this.dropLateData = dropLateData; + this.idLabel = idLabel; } + /** + * Returns a new TextIO.Read PTransform that's like this one but with the given + * step name. Does not modify the object. + */ public Bound named(String name) { - return new Bound(name, subscription, topic); + return new Bound(name, subscription, topic, timestampLabel, dropLateData, idLabel); } + /** + * Returns a new TextIO.Read PTransform that's like this one but reading from the + * given subscription. Does not modify the object. + */ public Bound subscription(String subscription) { - return new Bound(name, subscription, topic); + return new Bound(name, subscription, topic, timestampLabel, dropLateData, idLabel); } + /** + * Returns a new TextIO.Read PTransform that's like this one but reading from the + * give topic. Does not modify the object. + */ public Bound topic(String topic) { - return new Bound(name, subscription, topic); + return new Bound(name, subscription, topic, timestampLabel, dropLateData, idLabel); + } + + /** + * Returns a new TextIO.Read PTransform that's like this one but reading timestamps + * from the given PubSub label. Does not modify the object. + */ + public Bound timestampLabel(String timestampLabel) { + return new Bound(name, subscription, topic, timestampLabel, dropLateData, idLabel); + } + + /** + * Returns a new TextIO.Read PTransform that's like this one but with the specified + * setting for dropLateData. Does not modify the object. + */ + public Bound dropLateData(boolean dropLateData) { + return new Bound(name, subscription, topic, timestampLabel, dropLateData, idLabel); + } + + /** + * Returns a new TextIO.Read PTransform that's like this one but reading unique ids + * from the given PubSub label. Does not modify the object. + */ + public Bound idLabel(String idLabel) { + return new Bound(name, subscription, topic, timestampLabel, dropLateData, idLabel); } @Override @@ -250,6 +341,18 @@ public String getSubscription() { return subscription; } + public String getTimestampLabel() { + return timestampLabel; + } + + public boolean getDropLateData() { + return dropLateData; + } + + public String getIdLabel() { + return idLabel; + } + static { // TODO: Figure out how to make this work under // DirectPipelineRunner. @@ -278,6 +381,30 @@ public static Bound topic(String topic) { return new Bound().topic(topic); } + /** + * If specified, Dataflow will add a Pubsub label to each output record specifying the logical + * timestamp of the record. {@code } determines the label name. The label value + * is a numerical value representing the number of milliseconds since the Unix epoch. For + * example, if using the joda time classes, the org.joda.time.Instant(long) constructor can be + * used to parse this value. If the output from this sink is being read by another Dataflow + * source, then PubsubIO.Read.timestampLabel can be used to ensure that the other source reads + * these timestamps from the appropriate label. + */ + public static Bound timestampLabel(String timestampLabel) { + return new Bound().timestampLabel(timestampLabel); + } + + /** + * If specified, Dataflow will add a Pubsub label to each output record containing a unique + * identifier for that record. {@code } determines the label name. The label value + * is an opaque string value. This is useful if the the output from this sink is being read + * by another Dataflow source, in which case PubsubIO.Read.idLabel can be used to ensure that + * the other source reads these ids from the appropriate label. + */ + public static Bound idLabel(String idLabel) { + return new Bound().idLabel(idLabel); + } + /** * A PTransfrom that writes a unbounded {@code PCollection} * to a PubSub stream. @@ -287,23 +414,51 @@ public static class Bound extends PTransform, PDone> { /** The Pubsub topic to publish to. */ String topic; + String timestampLabel; + String idLabel; Bound() {} - Bound(String name, String topic) { + Bound(String name, String topic, String timestampLabel, String idLabel) { super(name); if (topic != null) { Validator.validateTopicName(topic); this.topic = topic; } + this.timestampLabel = timestampLabel; + this.idLabel = idLabel; } + /** + * Returns a new TextIO.Write PTransform that's like this one but with the given step + * name. Does not modify the object. + */ public Bound named(String name) { - return new Bound(name, topic); + return new Bound(name, topic, timestampLabel, idLabel); } + /** + * Returns a new TextIO.Write PTransform that's like this one but writing to the given + * topic. Does not modify the object. + */ public Bound topic(String topic) { - return new Bound(name, topic); + return new Bound(name, topic, timestampLabel, idLabel); + } + + /** + * Returns a new TextIO.Write PTransform that's like this one but publishing timestamps + * to the given PubSub label. Does not modify the object. + */ + public Bound timestampLabel(String timestampLabel) { + return new Bound(name, topic, timestampLabel, idLabel); + } + + /** + * Returns a new TextIO.Write PTransform that's like this one but publishing record ids + * to the given PubSub label. Does not modify the object. + */ + public Bound idLabel(String idLabel) { + return new Bound(name, topic, timestampLabel, idLabel); } @Override @@ -327,6 +482,14 @@ public String getTopic() { return topic; } + public String getTimestampLabel() { + return timestampLabel; + } + + public String getIdLabel() { + return idLabel; + } + static { // TODO: Figure out how to make this work under // DirectPipelineRunner. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java index 706397bddd37b..591fe8c79de89 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java @@ -57,6 +57,13 @@ private void translateReadHelper( if (transform.getSubscription() != null) { context.addInput(PropertyNames.PUBSUB_SUBSCRIPTION, transform.getSubscription()); } + if (transform.getTimestampLabel() != null) { + context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel()); + } + context.addInput(PropertyNames.PUBSUB_DROP_LATE_DATA, transform.getDropLateData()); + if (transform.getIdLabel() != null) { + context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel()); + } context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); // TODO: Orderedness? } @@ -83,6 +90,12 @@ private void translateWriteHelper( context.addStep(transform, "ParallelWrite"); context.addInput(PropertyNames.FORMAT, "pubsub"); context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic()); + if (transform.getTimestampLabel() != null) { + context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel()); + } + if (transform.getIdLabel() != null) { + context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel()); + } context.addEncodingInput( WindowedValue.getValueOnlyCoder(transform.getInput().getCoder())); context.addInput(PropertyNames.PARALLEL_INPUT, transform.getInput()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java index 0afe5ae411901..a22f7893a334c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java @@ -68,7 +68,10 @@ public class PropertyNames { public static final String OUTPUT_NAME = "output_name"; public static final String PARALLEL_INPUT = "parallel_input"; public static final String PHASE = "phase"; + public static final String PUBSUB_DROP_LATE_DATA = "pubsub_drop_late_data"; + public static final String PUBSUB_ID_LABEL = "pubsub_id_label"; public static final String PUBSUB_SUBSCRIPTION = "pubsub_subscription"; + public static final String PUBSUB_TIMESTAMP_LABEL = "pubsub_timestamp_label"; public static final String PUBSUB_TOPIC = "pubsub_topic"; public static final String SCALAR_FIELD_NAME = "value"; public static final String SERIALIZED_FN = "serialized_fn"; From 774c368edd60e6b3cdcd39ee6aa455b7ac7dcdf8 Mon Sep 17 00:00:00 2001 From: relax Date: Mon, 2 Feb 2015 15:34:06 -0800 Subject: [PATCH 0114/1541] Automated rollback of a previous change. *** Original change description *** Allows users of PubsubIO to specify which pubsub labels are used to propagate record timestamps and record ids. Strictly SDK changes, cannot yet be used. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85370721 --- .../cloud/dataflow/sdk/io/PubsubIO.java | 181 +----------------- .../runners/dataflow/PubsubIOTranslator.java | 13 -- .../dataflow/sdk/util/PropertyNames.java | 3 - 3 files changed, 9 insertions(+), 188 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index 060844699b550..4a379920249e7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -153,10 +153,6 @@ public static Bound named(String name) { *
  5. Must end with a letter or a number.
  6. *
  7. Cannot begin with 'goog' prefix.
  8. * - * - * Dataflow will start reading data published on this topic from the time the pipeline is - * started. Any data published on the topic before the pipeline is started will not be read - * by Dataflow. */ public static Bound topic(String topic) { return new Bound().topic(topic); @@ -184,46 +180,6 @@ public static Bound subscription(String subscription) { return new Bound().subscription(subscription); } - /** - * Creates and returns a PubsubIO.Read PTransform where record timestamps are expected - * to be provided using the PubSub labeling API. The {@code } parameter - * specifies the label name. The label value sent to PubsSub is a numerical value representing - * the number of milliseconds since the Unix epoch. For example, if using the joda time classes, - * org.joda.time.Instant.getMillis() returns the correct value for this label. - * - * When this feature is used, - * - * If {@code } is not provided, the system will generate record timestamps the - * first time it sees each record. All windowing will be done relative to these timestamps. - * Windows are closed based on an estimate of when this source has finished producing data for - * a timestamp range, which means that late data can arrive after a window has been closed. The - * {#dropLateData} field allows you to control what to do with late data. - */ - public static Bound timestampLabel(String timestampLabel) { - return new Bound().timestampLabel(timestampLabel); - } - - /** - * If true, then late-arriving data from this source will be dropped. - */ - public static Bound dropLateData(boolean dropLateData) { - return new Bound().dropLateData(dropLateData); - } - - /** - * Creates and returns a PubSubIO.Read PTransform where unique record identifiers are - * expected to be provided using the PubSub labeling API. The {@code } parameter - * specifies the label name. The label value sent to PubSub can be any string value that - * uniquely identifies this record. - * - * if idLabel is not provided, Dataflow cannot guarantee that no duplicate data will be - * delivered on the PubSub stream. In this case, deduplication of the stream will be - * stricly best effort. - */ - public static Bound idLabel(String idLabel) { - return new Bound().idLabel(idLabel); - } - /** * A PTransform that reads from a PubSub source and returns * a unbounded PCollection containing the items from the stream. @@ -235,18 +191,10 @@ public static class Bound String topic; /** The Pubsub subscription to read from. */ String subscription; - /** The Pubsub label to read timestamps from. */ - String timestampLabel; - Boolean dropLateData; - /** The Pubsub label to read ids from. */ - String idLabel; - - Bound() { - this.dropLateData = true; - } - Bound(String name, String subscription, String topic, String timestampLabel, - boolean dropLateData, String idLabel) { + Bound() {} + + Bound(String name, String subscription, String topic) { super(name); if (subscription != null) { Validator.validateSubscriptionName(subscription); @@ -256,57 +204,18 @@ public static class Bound } this.subscription = subscription; this.topic = topic; - this.timestampLabel = timestampLabel; - this.dropLateData = dropLateData; - this.idLabel = idLabel; } - /** - * Returns a new TextIO.Read PTransform that's like this one but with the given - * step name. Does not modify the object. - */ public Bound named(String name) { - return new Bound(name, subscription, topic, timestampLabel, dropLateData, idLabel); + return new Bound(name, subscription, topic); } - /** - * Returns a new TextIO.Read PTransform that's like this one but reading from the - * given subscription. Does not modify the object. - */ public Bound subscription(String subscription) { - return new Bound(name, subscription, topic, timestampLabel, dropLateData, idLabel); + return new Bound(name, subscription, topic); } - /** - * Returns a new TextIO.Read PTransform that's like this one but reading from the - * give topic. Does not modify the object. - */ public Bound topic(String topic) { - return new Bound(name, subscription, topic, timestampLabel, dropLateData, idLabel); - } - - /** - * Returns a new TextIO.Read PTransform that's like this one but reading timestamps - * from the given PubSub label. Does not modify the object. - */ - public Bound timestampLabel(String timestampLabel) { - return new Bound(name, subscription, topic, timestampLabel, dropLateData, idLabel); - } - - /** - * Returns a new TextIO.Read PTransform that's like this one but with the specified - * setting for dropLateData. Does not modify the object. - */ - public Bound dropLateData(boolean dropLateData) { - return new Bound(name, subscription, topic, timestampLabel, dropLateData, idLabel); - } - - /** - * Returns a new TextIO.Read PTransform that's like this one but reading unique ids - * from the given PubSub label. Does not modify the object. - */ - public Bound idLabel(String idLabel) { - return new Bound(name, subscription, topic, timestampLabel, dropLateData, idLabel); + return new Bound(name, subscription, topic); } @Override @@ -341,18 +250,6 @@ public String getSubscription() { return subscription; } - public String getTimestampLabel() { - return timestampLabel; - } - - public boolean getDropLateData() { - return dropLateData; - } - - public String getIdLabel() { - return idLabel; - } - static { // TODO: Figure out how to make this work under // DirectPipelineRunner. @@ -381,30 +278,6 @@ public static Bound topic(String topic) { return new Bound().topic(topic); } - /** - * If specified, Dataflow will add a Pubsub label to each output record specifying the logical - * timestamp of the record. {@code } determines the label name. The label value - * is a numerical value representing the number of milliseconds since the Unix epoch. For - * example, if using the joda time classes, the org.joda.time.Instant(long) constructor can be - * used to parse this value. If the output from this sink is being read by another Dataflow - * source, then PubsubIO.Read.timestampLabel can be used to ensure that the other source reads - * these timestamps from the appropriate label. - */ - public static Bound timestampLabel(String timestampLabel) { - return new Bound().timestampLabel(timestampLabel); - } - - /** - * If specified, Dataflow will add a Pubsub label to each output record containing a unique - * identifier for that record. {@code } determines the label name. The label value - * is an opaque string value. This is useful if the the output from this sink is being read - * by another Dataflow source, in which case PubsubIO.Read.idLabel can be used to ensure that - * the other source reads these ids from the appropriate label. - */ - public static Bound idLabel(String idLabel) { - return new Bound().idLabel(idLabel); - } - /** * A PTransfrom that writes a unbounded {@code PCollection} * to a PubSub stream. @@ -414,51 +287,23 @@ public static class Bound extends PTransform, PDone> { /** The Pubsub topic to publish to. */ String topic; - String timestampLabel; - String idLabel; Bound() {} - Bound(String name, String topic, String timestampLabel, String idLabel) { + Bound(String name, String topic) { super(name); if (topic != null) { Validator.validateTopicName(topic); this.topic = topic; } - this.timestampLabel = timestampLabel; - this.idLabel = idLabel; } - /** - * Returns a new TextIO.Write PTransform that's like this one but with the given step - * name. Does not modify the object. - */ public Bound named(String name) { - return new Bound(name, topic, timestampLabel, idLabel); + return new Bound(name, topic); } - /** - * Returns a new TextIO.Write PTransform that's like this one but writing to the given - * topic. Does not modify the object. - */ public Bound topic(String topic) { - return new Bound(name, topic, timestampLabel, idLabel); - } - - /** - * Returns a new TextIO.Write PTransform that's like this one but publishing timestamps - * to the given PubSub label. Does not modify the object. - */ - public Bound timestampLabel(String timestampLabel) { - return new Bound(name, topic, timestampLabel, idLabel); - } - - /** - * Returns a new TextIO.Write PTransform that's like this one but publishing record ids - * to the given PubSub label. Does not modify the object. - */ - public Bound idLabel(String idLabel) { - return new Bound(name, topic, timestampLabel, idLabel); + return new Bound(name, topic); } @Override @@ -482,14 +327,6 @@ public String getTopic() { return topic; } - public String getTimestampLabel() { - return timestampLabel; - } - - public String getIdLabel() { - return idLabel; - } - static { // TODO: Figure out how to make this work under // DirectPipelineRunner. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java index 591fe8c79de89..706397bddd37b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java @@ -57,13 +57,6 @@ private void translateReadHelper( if (transform.getSubscription() != null) { context.addInput(PropertyNames.PUBSUB_SUBSCRIPTION, transform.getSubscription()); } - if (transform.getTimestampLabel() != null) { - context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel()); - } - context.addInput(PropertyNames.PUBSUB_DROP_LATE_DATA, transform.getDropLateData()); - if (transform.getIdLabel() != null) { - context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel()); - } context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); // TODO: Orderedness? } @@ -90,12 +83,6 @@ private void translateWriteHelper( context.addStep(transform, "ParallelWrite"); context.addInput(PropertyNames.FORMAT, "pubsub"); context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic()); - if (transform.getTimestampLabel() != null) { - context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel()); - } - if (transform.getIdLabel() != null) { - context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel()); - } context.addEncodingInput( WindowedValue.getValueOnlyCoder(transform.getInput().getCoder())); context.addInput(PropertyNames.PARALLEL_INPUT, transform.getInput()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java index a22f7893a334c..0afe5ae411901 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java @@ -68,10 +68,7 @@ public class PropertyNames { public static final String OUTPUT_NAME = "output_name"; public static final String PARALLEL_INPUT = "parallel_input"; public static final String PHASE = "phase"; - public static final String PUBSUB_DROP_LATE_DATA = "pubsub_drop_late_data"; - public static final String PUBSUB_ID_LABEL = "pubsub_id_label"; public static final String PUBSUB_SUBSCRIPTION = "pubsub_subscription"; - public static final String PUBSUB_TIMESTAMP_LABEL = "pubsub_timestamp_label"; public static final String PUBSUB_TOPIC = "pubsub_topic"; public static final String SCALAR_FIELD_NAME = "value"; public static final String SERIALIZED_FN = "serialized_fn"; From d692b7c21dc8993b43ab4086ae41b9f9ed9604fe Mon Sep 17 00:00:00 2001 From: millsd Date: Mon, 2 Feb 2015 17:15:29 -0800 Subject: [PATCH 0115/1541] Adds protobuf for communicating with Windmill service. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85379329 --- sdk/src/main/proto/windmill.proto | 214 ++++++++++++++++++++++++++++++ 1 file changed, 214 insertions(+) create mode 100644 sdk/src/main/proto/windmill.proto diff --git a/sdk/src/main/proto/windmill.proto b/sdk/src/main/proto/windmill.proto new file mode 100644 index 0000000000000..88aa796417143 --- /dev/null +++ b/sdk/src/main/proto/windmill.proto @@ -0,0 +1,214 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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. + */ + +/* + * Protocol Buffers describing the interface between streaming Dataflow workers + * and the Windmill servers. + */ + +syntax = "proto2"; + +package windmill; + +option java_package = "com.google.cloud.dataflow.sdk.runners.worker.windmill"; +option java_outer_classname = "Windmill"; + +//////////////////////////////////////////////////////////////////////////////// +// API Data types + +message Message { + required int64 timestamp = 1 [default=-0x8000000000000000]; + required bytes data = 2; + optional bytes metadata = 3; +} + +message Timer { + required bytes tag = 1; + optional int64 timestamp = 2 [default=-0x8000000000000000]; +} + +message InputMessageBundle { + required string source_computation_id = 1; + repeated Message messages = 2; +} + +message KeyedMessageBundle { + required bytes key = 1; + repeated Message messages = 2; +} + +message OutputMessageBundle { + optional string destination_computation_id = 1; + optional string destination_stream_id = 3; + repeated KeyedMessageBundle bundles = 2; +} + +message PubSubMessageBundle { + required string topic = 1; + repeated Message messages = 2; +} + +message TimerBundle { + repeated Timer timers = 1; +} + +message Value { + required int64 timestamp = 1 [default=-0x8000000000000000]; + required bytes data = 2; +} + +message TagValue { + required bytes tag = 1; + optional Value value = 2; +} + +message TagList { + required bytes tag = 1; + optional int64 end_timestamp = 2 [default=-0x8000000000000000]; + repeated Value values = 3; +} + +message WorkItem { + required bytes key = 1; + required fixed64 work_token = 2; + + repeated InputMessageBundle message_bundles = 3; + optional TimerBundle timers = 4; +} + +message ComputationWorkItems { + required string computation_id = 1; + repeated WorkItem work = 2; +} + +//////////////////////////////////////////////////////////////////////////////// +// API calls + +// GetWork + +message GetWorkRequest { + required fixed64 client_id = 1; + optional int64 max_items = 2 [default = 0xffffffff]; +} + +message GetWorkResponse { + repeated ComputationWorkItems work = 1; +} + +// GetData + +message KeyedGetDataRequest { + required bytes key = 1; + required fixed64 work_token = 2; + repeated TagValue values_to_fetch = 3; + repeated TagList lists_to_fetch = 4; +} + +message ComputationGetDataRequest { + required string computation_id = 1; + repeated KeyedGetDataRequest requests = 2; +} + +message GetDataRequest { + repeated ComputationGetDataRequest requests = 1; +} + +message KeyedGetDataResponse { + required bytes key = 1; + // The response for this key is not populated due to the fetch failing. + optional bool failed = 2; + repeated TagValue values = 3; + repeated TagList lists = 4; +} + +message ComputationGetDataResponse { + required string computation_id = 1; + repeated KeyedGetDataResponse data = 2; +} + +message GetDataResponse { + repeated ComputationGetDataResponse data = 1; +} + +// CommitWork + +message Counter { + optional string name = 1; + enum Kind { + SUM = 0; + MAX = 1; + MIN = 2; + MEAN = 3; + }; + optional Kind kind = 2; + + // For SUM, MAX, MIN, AND, OR, MEAN at most one of the following should be + // set. For MEAN it is the sum + optional double double_scalar = 3; + optional int64 int_scalar = 4; + + // Only set for MEAN. Count of elements contributing to the sum. + optional int64 mean_count = 6; +} + +// next id: 9 +message WorkItemCommitRequest { + required bytes key = 1; + required fixed64 work_token = 2; + repeated OutputMessageBundle output_messages = 3; + repeated PubSubMessageBundle pubsub_messages = 7; + repeated Timer output_timers = 4; + repeated TagValue value_updates = 5; + repeated TagList list_updates = 6; + repeated Counter counter_updates = 8; +} + +message ComputationCommitWorkRequest { + required string computation_id = 1; + repeated WorkItemCommitRequest requests = 2; +} + +message CommitWorkRequest { + repeated ComputationCommitWorkRequest requests = 1; +} + +message CommitWorkResponse {} + +// Configuration + +message GetConfigRequest { + repeated string computations = 1; +} + +message GetConfigResponse { + repeated string cloud_works = 1; +} + +// Reporting + +message Exception { + repeated string stack_frames = 1; + optional Exception cause = 2; +} + +message ReportStatsRequest { + required string computation_id = 1; + required bytes key = 2; + required fixed64 work_token = 3; + repeated Exception exceptions = 4; +} + +message ReportStatsResponse {} From a46c40d27b25e11e64cf4df687747ae747cea8b4 Mon Sep 17 00:00:00 2001 From: chamikara Date: Mon, 2 Feb 2015 17:41:44 -0800 Subject: [PATCH 0116/1541] Updated DataStoreWordCount so that it can be adapted as an integration test for DataStoreIO. Also updated BasicSerializableSourceFormat.serializeToCloudSource() to not fail when estimated size calculation fails. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85381606 --- .../dataflow/examples/DatastoreWordCount.java | 58 +++++++++++++------ .../cloud/dataflow/sdk/io/DatastoreIO.java | 12 ++-- .../BasicSerializableSourceFormat.java | 16 ++++- 3 files changed, 61 insertions(+), 25 deletions(-) diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/DatastoreWordCount.java b/examples/src/main/java/com/google/cloud/dataflow/examples/DatastoreWordCount.java index ab31aae15ab76..ec9d864eefe15 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/DatastoreWordCount.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/DatastoreWordCount.java @@ -16,6 +16,7 @@ package com.google.cloud.dataflow.examples; +import com.google.api.services.datastore.DatastoreV1; import com.google.api.services.datastore.DatastoreV1.Entity; import com.google.api.services.datastore.DatastoreV1.Key; import com.google.api.services.datastore.DatastoreV1.Property; @@ -30,6 +31,8 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; @@ -75,7 +78,10 @@ static class GetContentFn extends DoFn { @Override public void processElement(ProcessContext c) { Map props = DatastoreHelper.getPropertyMap(c.element()); - c.output(DatastoreHelper.getString(props.get("content"))); + DatastoreV1.Value value = props.get("content"); + if (value != null) { + c.output(DatastoreHelper.getString(value)); + } } } @@ -94,9 +100,9 @@ public Entity makeEntity(String content) { // Create entities with same ancestor Key. Key ancestorKey = DatastoreHelper.makeKey(kind, "root").build(); Key key = DatastoreHelper.makeKey(ancestorKey, kind).build(); + entityBuilder.setKey(key); - entityBuilder.addProperty(Property.newBuilder() - .setName("content") + entityBuilder.addProperty(Property.newBuilder().setName("content") .setValue(Value.newBuilder().setStringValue(content))); return entityBuilder.build(); } @@ -112,7 +118,7 @@ public void processElement(ProcessContext c) { *

    * Inherits standard configuration options. */ - private static interface Options extends PipelineOptions { + public static interface Options extends PipelineOptions { @Description("Path of the file to read from and store to Datastore") @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") String getInput(); @@ -136,6 +142,11 @@ private static interface Options extends PipelineOptions { @Description("Read an existing dataset, do not write first") boolean isReadOnly(); void setReadOnly(boolean value); + + @Description("Number of output shards") + @Default.Integer(0) // If the system should choose automatically. + int getNumShards(); + void setNumShards(int value); } /** @@ -143,14 +154,23 @@ private static interface Options extends PipelineOptions { * text input. Forces use of DirectPipelineRunner for local execution mode. */ public static void writeDataToDatastore(Options options) { - // Runs locally via DirectPiplineRunner, as writing is not yet implemented - // for the other runners which is why we just create a PipelineOptions with defaults. - Pipeline p = Pipeline.create(PipelineOptionsFactory.create()); - p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) - .apply(ParDo.of(new CreateEntityFn(options.getKind()))) - .apply(DatastoreIO.write().to(options.getDataset())); - - p.run(); + // Storing the user-specified runner. + Class> tempRunner = options.getRunner(); + + try { + // Runs locally via DirectPiplineRunner, as writing is not yet implemented + // for the other runners. + options.setRunner(DirectPipelineRunner.class); + Pipeline p = Pipeline.create(options); + p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) + .apply(ParDo.of(new CreateEntityFn(options.getKind()))) + .apply(DatastoreIO.write().to(options.getDataset())); + + p.run(); + } finally { + // Resetting the runner to the user specified class. + options.setRunner(tempRunner); + } } /** @@ -163,12 +183,12 @@ public static void readDataFromDatastore(Options options) { Query query = q.build(); Pipeline p = Pipeline.create(options); - p.apply(DatastoreIO.readFrom(options.getDataset(), query) - .named("ReadShakespeareFromDatastore")) - .apply(ParDo.of(new GetContentFn())) - .apply(new WordCount.CountWords()) - .apply(TextIO.Write.named("WriteLines").to(options.getOutput())); - + p.apply(DatastoreIO.readFrom(options.getDataset(), query).named("ReadShakespeareFromDatastore")) + .apply(ParDo.of(new GetContentFn())) + .apply(new WordCount.CountWords()) + .apply(TextIO.Write.named("WriteLines") + .to(options.getOutput()) + .withNumShards(options.getNumShards())); p.run(); } @@ -185,7 +205,7 @@ public static void main(String args[]) { if (!options.isReadOnly()) { // First example: write data to Datastore for reading later. - // Note: this will insert new entries with the given kind. Existing entries + // Note: this will insert new entries with the given kind. Existing entries // should be cleared first, or the final counts will contain duplicates. // The Datastore Admin tool in the AppEngine console can be used to erase // all entries with a particular kind. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java index 6f3e3b8228a6e..4901ca30ead82 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java @@ -136,7 +136,13 @@ public class DatastoreIO { private static final Logger LOG = LoggerFactory.getLogger(DatastoreIO.class); - private static final String DEFAULT_HOST = "https://www.googleapis.com"; + public static final String DEFAULT_HOST = "https://www.googleapis.com"; + + /** + * Datastore has a limit of 500 mutations per batch operation, so we flush + * changes to Datastore every 500 entities. + */ + public static final int DATASTORE_BATCH_UPDATE_LIMIT = 500; /** * Returns an empty {@code DatastoreIO.Read} builder with the default host. @@ -484,9 +490,7 @@ private static void evaluateWriteHelper( List toInsert = new ArrayList<>(); for (Entity e : entitiesWithSameAncestor) { toInsert.add(e); - // Note that Datastore has limit as 500 for a batch operation, - // so just flush to Datastore with every 500 entties. - if (toInsert.size() >= 500) { + if (toInsert.size() >= DATASTORE_BATCH_UPDATE_LIMIT) { writeBatch(toInsert, datastore); toInsert.clear(); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormat.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormat.java index cb3f4c237e7d3..4f723e66e7315 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormat.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormat.java @@ -47,6 +47,9 @@ import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.common.worker.SourceFormat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -54,7 +57,7 @@ import java.util.NoSuchElementException; /** - * An helper class for supporting sources defined as {@code Source}. + * A helper class for supporting sources defined as {@code Source}. * * Provides a bridge between the high-level {@code Source} API and the raw * API-level {@code SourceFormat} API, by encoding the serialized @@ -65,6 +68,8 @@ public class BasicSerializableSourceFormat implements SourceFormat { private static final String SERIALIZED_SOURCE = "serialized_source"; private static final long DEFAULT_DESIRED_SHARD_SIZE_BYTES = 64 * (1 << 20); + private static final Logger LOG = LoggerFactory.getLogger(BasicSerializableSourceFormat.class); + private final PipelineOptions options; public BasicSerializableSourceFormat(PipelineOptions options) { @@ -163,7 +168,14 @@ private static com.google.api.services.dataflow.model.Source serializeToCloudSou SourceMetadata metadata = new SourceMetadata(); metadata.setProducesSortedKeys(source.producesSortedKeys(options)); - metadata.setEstimatedSizeBytes(source.getEstimatedSizeBytes(options)); + + // Size estimation is best effort so we continue even if it fails here. + try { + metadata.setEstimatedSizeBytes(source.getEstimatedSizeBytes(options)); + } catch (Exception e) { + LOG.warn("Size estimation of the source failed.", e); + } + cloudSource.setMetadata(metadata); return cloudSource; } From 1848854fc8afaa457b7a832cd9999f26b93d76d5 Mon Sep 17 00:00:00 2001 From: wan Date: Tue, 3 Feb 2015 09:47:53 -0800 Subject: [PATCH 0117/1541] Moves class GlobalWindows.GlobalWindow to top-level for consistency with other window types. This change is backward-incompatible but unlikely to affect many users. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85434463 --- .../transforms/windowing/GlobalWindow.java | 61 +++++++++++++++++++ .../transforms/windowing/GlobalWindows.java | 43 +------------ .../dataflow/sdk/util/WindowedValue.java | 4 +- .../sdk/runners/worker/ShuffleSinkTest.java | 3 +- 4 files changed, 66 insertions(+), 45 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java new file mode 100644 index 0000000000000..97c82eeac5b76 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; + +import org.joda.time.Instant; + +import java.io.InputStream; +import java.io.OutputStream; + +/** + * The default window into which all data is placed (via {@link GlobalWindows}). + */ +public class GlobalWindow extends BoundedWindow { + public static final GlobalWindow INSTANCE = new GlobalWindow(); + + @Override + public Instant maxTimestamp() { + return new Instant(Long.MAX_VALUE); + } + + private GlobalWindow() {} + + /** + * {@link Coder} for encoding and decoding {@code Window}s. + */ + public static class Coder extends AtomicCoder { + public static final Coder INSTANCE = new Coder(); + + @Override + public void encode(GlobalWindow window, OutputStream outStream, Context context) {} + + @Override + public GlobalWindow decode(InputStream inStream, Context context) { + return GlobalWindow.INSTANCE; + } + + @Override + public boolean isDeterministic() { + return true; + } + + private Coder() {} + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindows.java index d4858b85ba5c7..c52ec92a9d3b2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindows.java @@ -16,13 +16,8 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; -import com.google.cloud.dataflow.sdk.coders.AtomicCoder; import com.google.cloud.dataflow.sdk.coders.Coder; -import org.joda.time.Instant; - -import java.io.InputStream; -import java.io.OutputStream; import java.util.Arrays; import java.util.Collection; @@ -31,7 +26,7 @@ */ @SuppressWarnings("serial") public class GlobalWindows - extends NonMergingWindowFn { + extends NonMergingWindowFn { @Override public Collection assignWindows(AssignContext c) { return Arrays.asList(GlobalWindow.INSTANCE); @@ -46,40 +41,4 @@ public boolean isCompatible(WindowFn o) { public Coder windowCoder() { return GlobalWindow.Coder.INSTANCE; } - - /** - * The default window into which all data is placed. - */ - public static class GlobalWindow extends BoundedWindow { - public static final GlobalWindow INSTANCE = new GlobalWindow(); - - @Override - public Instant maxTimestamp() { - return new Instant(Long.MAX_VALUE); - } - - private GlobalWindow() {} - - /** - * {@link Coder} for encoding and decoding {@code Window}s. - */ - public static class Coder extends AtomicCoder { - public static final Coder INSTANCE = new Coder(); - - @Override - public void encode(GlobalWindow window, OutputStream outStream, Context context) {} - - @Override - public GlobalWindow decode(InputStream inStream, Context context) { - return GlobalWindow.INSTANCE; - } - - @Override - public boolean isDeterministic() { - return true; - } - - private Coder() {} - } - } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java index 0562b58b35c50..4897a3e10c229 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java @@ -26,7 +26,7 @@ import com.google.cloud.dataflow.sdk.coders.InstantCoder; import com.google.cloud.dataflow.sdk.coders.StandardCoder; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; import com.fasterxml.jackson.annotation.JsonCreator; @@ -72,7 +72,7 @@ public static WindowedValue of( public static WindowedValue valueInGlobalWindow(V value) { return new WindowedValue<>(value, new Instant(Long.MIN_VALUE), - Arrays.asList(GlobalWindows.GlobalWindow.INSTANCE)); + Arrays.asList(GlobalWindow.INSTANCE)); } /** diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java index b58fc38c8f46a..806588a8a3a81 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java @@ -23,6 +23,7 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.util.CoderUtils; @@ -107,7 +108,7 @@ private void runTestWriteUngroupingShuffleSink(List expected) // Ignore the key. byte[] valueBytes = record.getValue(); WindowedValue value = CoderUtils.decodeFromByteArray(windowedValueCoder, valueBytes); - Assert.assertEquals(Lists.newArrayList(GlobalWindows.GlobalWindow.INSTANCE), + Assert.assertEquals(Lists.newArrayList(GlobalWindow.INSTANCE), value.getWindows()); actual.add(value.getValue()); } From 56e23990d8fb4443464388576b86b963e2557601 Mon Sep 17 00:00:00 2001 From: davor Date: Tue, 3 Feb 2015 10:58:25 -0800 Subject: [PATCH 0118/1541] Updates to Dataflow README on GitHub. * Fixes the problem with DirectPipelineRunner and GCS locations. * Adds a note about Maven on Windows. * Explains how to run a bundled JAR. This fixes #6 on GitHub. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85441294 --- README.md | 72 ++++++++++++++++++++++++++++++++++++++----------------- 1 file changed, 50 insertions(+), 22 deletions(-) diff --git a/README.md b/README.md index 3e6bb3136a548..3b7869190e18d 100644 --- a/README.md +++ b/README.md @@ -31,22 +31,26 @@ for execution. * [`PipelineRunner`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunner.java): specifies where and how the pipeline should execute. -Currently there are three `PipelineRunners`: +We provide three PipelineRunners: 1. The [`DirectPipelineRunner`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java) runs the pipeline on your local machine. - 2. The -[`DataflowPipelineRunner`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java) + 2. The [`DataflowPipelineRunner`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java) submits the pipeline to the Dataflow Service, where it runs using managed -resources in the [Google Cloud Platform](http://cloud.google.com). +resources in the [Google Cloud Platform](https://cloud.google.com) (GCP). 3. The [`BlockingDataflowPipelineRunner`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/BlockingDataflowPipelineRunner.java) submits the pipeline to the Dataflow Service via the `DataflowPipelineRunner` -and then prints messages about the job status until execution is complete. +and then prints messages about the job status until the execution is complete. _The Dataflow Service is currently in the Alpha phase of development and access is limited to whitelisted users._ +Additionally, in partnership with [Cloudera](https://www.cloudera.com/), you can +run Dataflow pipelines on an [Apache Spark](https://spark.apache.org/) backend. +The relevant runner code is hosted in +[this](https://github.com/cloudera/spark-dataflow) repository. + ## Getting Started This repository consists of two modules: @@ -82,36 +86,60 @@ You can speed up the build and install process by using the following options: mvn -T 4 install After building and installing, you can execute the `WordCount` and other example -pipelines locally or in the cloud using Maven with command-line options. - -To execute the WordCount pipeline locally (using the default -`DirectPipelineRunner`) and write output to a local or -Google Cloud Storage (GCS) location, use the following command-line syntax: +pipelines using the `DirectPipelineRunner` on your local machine: mvn exec:java -pl examples \ -Dexec.mainClass=com.google.cloud.dataflow.examples.WordCount \ - -Dexec.args="--output=[ | gs://] + -Dexec.args="--input= --output=" If you have been whitelisted for Alpha access to the Dataflow Service and followed the [developer setup](https://cloud.google.com/dataflow/java-sdk/getting-started#DeveloperSetup) steps, you can use the `BlockingDataflowPipelineRunner` to execute the -`WordCount` example in the Google Cloud Platform (GCP). In this case, you -specify your project name, pipeline runner, the GCS staging location (staging -location should be entered in the form of `gs://bucket/staging-directory`), -and the GCS output (in the form of `gs://bucket/filename_prefix`). +`WordCount` example in the GCP. In this case, you specify your project name, +pipeline runner, and the staging location in +[Google Cloud Storage](https://cloud.google.com/storage/) (GCS), as follows: mvn exec:java -pl examples \ -Dexec.mainClass=com.google.cloud.dataflow.examples.WordCount \ - -Dexec.args="--project= --runner=BlockingDataflowPipelineRunner \ - --stagingLocation= --output=" + -Dexec.args="--project= --stagingLocation= --runner=BlockingDataflowPipelineRunner" + +GCS location should be entered in the form of +`gs://bucket/path/to/staging/directory`. GCP project refers to its name (not +number), which has been whitelisted for Cloud Dataflow. Refer to +[Google Cloud Platform](https://cloud.google.com/) for general instructions on +getting started with GCP. + +Alternatively, you may choose to bundle all dependencies into a single JAR and +execute it outside of the Maven environment. For example, after building and +installing as usual, you can execute the following commands to create the +bundled JAR of the `Examples` module and execute it both locally and in GCP: + + mvn bundle:bundle -pl examples -Refer to [Google Cloud Platform](https://cloud.google.com/) for general -instructions on getting started with GCP. + java -cp examples/target/google-cloud-dataflow-java-examples-all-bundled-manual_build.jar \ + com.google.cloud.dataflow.examples.WordCount \ + --input= --output= + + java -cp examples/target/google-cloud-dataflow-java-examples-all-bundled-manual_build.jar \ + com.google.cloud.dataflow.examples.WordCount \ + --project= --stagingLocation= --runner=BlockingDataflowPipelineRunner Other examples can be run similarly by replacing the `WordCount` class name with -`BigQueryTornadoes`, `DatastoreWordCount`, `TfIdf`, `TopWikipediaSessions`, etc. and -adjusting runtime options under the `Dexec.args` parameter, as specified in the -example itself. +`BigQueryTornadoes`, `DatastoreWordCount`, `TfIdf`, `TopWikipediaSessions`, etc. +and adjusting runtime options under the `Dexec.args` parameter, as specified in +the example itself. + +Note that when running Maven on Microsoft Windows platform, backslashes (`\`) +under the `Dexec.args` parameter should be escaped with another backslash. For +example, input file pattern of `c:\*.txt` should be entered as `c:\\*.txt`. + +## Contact Us + +We welcome all usage-related questions on [Stack Overflow](http://stackoverflow.com/questions/tagged/google-cloud-dataflow) +tagged with `google-cloud-dataflow`. + +Please use [issue tracker](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/issues) +on GitHub to report any bugs, comments or questions regarding SDK development. ## More Information From ffbf31fd08cef56f4b941e028ef89f3d7fb4a2b6 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Tue, 3 Feb 2015 12:59:28 -0800 Subject: [PATCH 0119/1541] Make test_wordcount.sh script executable by changing the file's mode. --- test_wordcount.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 test_wordcount.sh diff --git a/test_wordcount.sh b/test_wordcount.sh old mode 100644 new mode 100755 From 242b4c3025b74090e574087d54a2ae46d091e9b7 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Tue, 3 Feb 2015 16:25:37 -0800 Subject: [PATCH 0120/1541] Attempt to fix .travis.yml for OSX More information available here: https://github.com/travis-ci/travis-ci/issues/2839 https://jira.codehaus.org/browse/MNG-5658 https://jira.codehaus.org/browse/MNG-5686 https://github.com/Seagate/kinetic-cpp-client/blob/master/.travis.yml --- .travis.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.travis.yml b/.travis.yml index b9b15456657df..c860c7cfe55bb 100644 --- a/.travis.yml +++ b/.travis.yml @@ -11,6 +11,9 @@ notifications: on_success: change on_failure: always +before_install: + - if [ "$TRAVIS_OS_NAME" == "osx" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi + install: - mvn install clean -U -DskipTests=true From c3627f406e369f5b8070ff8c036481a7c527e2bf Mon Sep 17 00:00:00 2001 From: andersjohnson Date: Tue, 3 Feb 2015 16:43:34 -0800 Subject: [PATCH 0121/1541] Tweaks to support OSX. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85474161 --- test_wordcount.sh | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/test_wordcount.sh b/test_wordcount.sh index 81e2e2a6baa8c..699489c89b11c 100755 --- a/test_wordcount.sh +++ b/test_wordcount.sh @@ -17,7 +17,8 @@ set -e set -o pipefail -cd $(dirname $0) +MYDIR=$(dirname $0) || exit 2 +cd $MYDIR TOPDIR="." if [[ $# -gt 0 ]] @@ -33,7 +34,8 @@ function check_result_hash { local outfile_prefix=$2 local expected=$3 - local actual=$(md5sum $outfile_prefix-* | awk '{print $1}') + local actual=$(md5sum $outfile_prefix-* | awk '{print $1}' || \ + md5 -q $outfile_prefix-*) || exit 2 # OSX if [[ "$actual" != "$expected" ]] then echo "FAIL $name: Output hash mismatch. Got $actual, expected $expected." @@ -46,7 +48,8 @@ function check_result_hash { function get_outfile_prefix { local name=$1 - mktemp --tmpdir=/tmp -u "$name.out.XXXXXXXXXX" + # NOTE: mktemp on OSX doesn't support --tmpdir + mktemp -u "/tmp/$name.out.XXXXXXXXXX" } function run_via_mvn { @@ -54,7 +57,7 @@ function run_via_mvn { local input=$2 local expected_hash=$3 - local outfile_prefix="$(get_outfile_prefix "$name")" + local outfile_prefix="$(get_outfile_prefix "$name")" || exit 2 local cmd='mvn exec:java -f '"$TOPDIR"'/pom.xml -pl examples \ -Dexec.mainClass=com.google.cloud.dataflow.examples.WordCount \ -Dexec.args="--runner=DirectPipelineRunner --input='"$input"' --output='"$outfile_prefix"'"' @@ -68,7 +71,7 @@ function run_bundled { local input=$2 local expected_hash=$3 - local outfile_prefix="$(get_outfile_prefix "$name")" + local outfile_prefix="$(get_outfile_prefix "$name")" || exit 2 local cmd='java -cp '"$JAR_FILE"' \ com.google.cloud.dataflow.examples.WordCount \ --runner=DirectPipelineRunner \ From 95bb0ea26c43502b7e2e35e739f60ad94865ef59 Mon Sep 17 00:00:00 2001 From: millsd Date: Tue, 3 Feb 2015 17:38:06 -0800 Subject: [PATCH 0122/1541] Merge the streaming runner code into the main sdk. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85478753 --- sdk/pom.xml | 18 + .../sdk/runners/DataflowPipelineRunner.java | 2 +- .../sdk/runners/worker/PubsubReader.java | 98 ++ .../sdk/runners/worker/PubsubSink.java | 108 +++ .../sdk/runners/worker/ReaderFactory.java | 15 + .../sdk/runners/worker/SinkFactory.java | 4 + .../worker/StreamingDataflowWorker.java | 641 ++++++++++++ .../worker/UngroupedWindmillReader.java | 116 +++ .../sdk/runners/worker/WindmillSink.java | 130 +++ .../worker/WindowingWindmillReader.java | 157 +++ .../worker/windmill/WindmillServerStub.java | 45 + .../sdk/util/BoundedQueueExecutor.java | 69 ++ .../cloud/dataflow/sdk/util/StateFetcher.java | 147 +++ .../util/StreamingModeExecutionContext.java | 208 ++++ .../worker/StreamingDataflowWorkerTest.java | 913 ++++++++++++++++++ 15 files changed, 2670 insertions(+), 1 deletion(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PubsubReader.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PubsubSink.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedWindmillReader.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillSink.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindowingWindmillReader.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/windmill/WindmillServerStub.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BoundedQueueExecutor.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StateFetcher.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingModeExecutionContext.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java diff --git a/sdk/pom.xml b/sdk/pom.xml index 1b7992b00d382..f269e2bc70f6d 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -200,6 +200,12 @@ + + com.google.cloud.dataflow + google-cloud-dataflow-java-proto-library-all + 0.3.150203 + + com.google.apis google-api-services-bigquery @@ -359,6 +365,18 @@ 2.4 + + org.eclipse.jetty + jetty-server + 9.2.6.v20141205 + + + + org.eclipse.jetty + jetty-jmx + 9.2.6.v20141205 + + com.google.auto.service diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index 4da69ac460e5d..86781cd60eff3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -79,7 +79,7 @@ public class DataflowPipelineRunner extends PipelineRunner private DataflowPipelineRunnerHooks hooks; // Environment version information - private static final String ENVIRONMENT_MAJOR_VERSION = "0"; + private static final String ENVIRONMENT_MAJOR_VERSION = "1"; /** * Construct a runner from the provided options. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PubsubReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PubsubReader.java new file mode 100644 index 0000000000000..b39976f71e651 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PubsubReader.java @@ -0,0 +1,98 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.StreamingModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue.ValueOnlyWindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; +import java.util.concurrent.TimeUnit; + +/** + * A Reader that receives elements from Pubsub, via a Windmill server. + */ +class PubsubReader extends Reader> { + private final ValueOnlyWindowedValueCoder coder; + private StreamingModeExecutionContext context; + + PubsubReader(Coder> coder, StreamingModeExecutionContext context) { + this.coder = (ValueOnlyWindowedValueCoder) coder; + this.context = context; + } + + public static PubsubReader create( + PipelineOptions options, + CloudObject spec, + Coder> coder, + ExecutionContext context) { + return new PubsubReader<>(coder, (StreamingModeExecutionContext) context); + } + + @Override + public ReaderIterator> iterator() throws IOException { + return new PubsubReaderIterator(); + } + + class PubsubReaderIterator extends AbstractReaderIterator> { + private int bundleIndex = 0; + private int messageIndex = 0; + + @Override + public boolean hasNext() throws IOException { + Windmill.WorkItem work = context.getWork(); + return bundleIndex < work.getMessageBundlesCount() && + messageIndex < work.getMessageBundles(bundleIndex).getMessagesCount(); + } + + @Override + public WindowedValue next() throws IOException { + Windmill.Message message = + context.getWork().getMessageBundles(bundleIndex).getMessages(messageIndex); + if (messageIndex >= + context.getWork().getMessageBundles(bundleIndex).getMessagesCount() - 1) { + messageIndex = 0; + bundleIndex++; + } else { + messageIndex++; + } + long timestampMillis = TimeUnit.MICROSECONDS.toMillis(message.getTimestamp()); + InputStream data = message.getData().newInput(); + notifyElementRead(data.available()); + T value = (T) coder.getValueCoder().decode(data, Coder.Context.OUTER); + return WindowedValue.of(value, + new Instant(timestampMillis), + Arrays.asList(GlobalWindow.INSTANCE)); + } + } + + @Override + public boolean supportsRestart() { + return true; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PubsubSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PubsubSink.java new file mode 100644 index 0000000000000..e75d9155efee0 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PubsubSink.java @@ -0,0 +1,108 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.StreamingModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; +import com.google.protobuf.ByteString; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +/** + * A sink that writes to Pubsub, via a Windmill server. + * + * @param the type of the elements written to the sink + */ +class PubsubSink extends Sink> { + private String topic; + private Coder> coder; + private StreamingModeExecutionContext context; + + PubsubSink(String topic, + Coder> coder, + StreamingModeExecutionContext context) { + this.topic = topic; + this.coder = coder; + this.context = context; + } + + public static PubsubSink create(PipelineOptions options, + CloudObject spec, + Coder> coder, + ExecutionContext context) + throws Exception { + String topic = getString(spec, "pubsub_topic"); + return new PubsubSink<>(topic, coder, (StreamingModeExecutionContext) context); + } + + @Override + public SinkWriter> writer() { + return new PubsubWriter(topic); + } + + /** The SinkWriter for a PubsubSink. */ + class PubsubWriter implements SinkWriter> { + private Windmill.PubSubMessageBundle.Builder outputBuilder; + + private PubsubWriter(String topic) { + outputBuilder = Windmill.PubSubMessageBundle.newBuilder().setTopic(topic); + } + + private ByteString encode(Coder coder, S object) throws IOException { + ByteString.Output stream = ByteString.newOutput(); + coder.encode(object, stream, Coder.Context.OUTER); + return stream.toByteString(); + } + + @Override + public long add(WindowedValue data) throws IOException { + ByteString byteString = encode(coder, data); + + long timestampMicros = TimeUnit.MILLISECONDS.toMicros(data.getTimestamp().getMillis()); + outputBuilder.addMessages( + Windmill.Message.newBuilder() + .setData(byteString) + .setTimestamp(timestampMicros) + .build()); + + return byteString.size(); + } + + @Override + public void close() throws IOException { + Windmill.PubSubMessageBundle pubsubMessages = outputBuilder.build(); + if (pubsubMessages.getMessagesCount() > 0) { + context.getOutputBuilder().addPubsubMessages(pubsubMessages); + } + outputBuilder.clear(); + } + } + + @Override + public boolean supportsRestart() { + return true; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderFactory.java index 5d42970c16000..17c695482ffae 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderFactory.java @@ -63,6 +63,21 @@ private ReaderFactory() {} "GroupingShuffleSource", GroupingShuffleReaderFactory.class.getName()); predefinedReaderFactories.put("InMemorySource", InMemoryReaderFactory.class.getName()); predefinedReaderFactories.put("BigQuerySource", BigQueryReaderFactory.class.getName()); + predefinedReaderFactories.put( + "com.google.cloud.dataflow.sdk.runners.worker.BucketingWindmillSource", + WindowingWindmillReader.class.getName()); + predefinedReaderFactories.put( + "WindowingWindmillReader", WindowingWindmillReader.class.getName()); + predefinedReaderFactories.put( + "com.google.cloud.dataflow.sdk.runners.worker.UngroupedWindmillSource", + UngroupedWindmillReader.class.getName()); + predefinedReaderFactories.put( + "UngroupedWindmillReader", UngroupedWindmillReader.class.getName()); + predefinedReaderFactories.put( + "com.google.cloud.dataflow.sdk.runners.worker.PubsubSource", + PubsubReader.class.getName()); + predefinedReaderFactories.put( + "PubsubReader", PubsubReader.class.getName()); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactory.java index 590e7f8151347..363d830dffbd4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SinkFactory.java @@ -55,6 +55,10 @@ private SinkFactory() {} AvroSinkFactory.class.getName()); predefinedSinkFactories.put("ShuffleSink", ShuffleSinkFactory.class.getName()); + predefinedSinkFactories.put("PubsubSink", + PubsubSink.class.getName()); + predefinedSinkFactories.put("WindmillSink", + WindmillSink.class.getName()); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java new file mode 100644 index 0000000000000..9234d31c7e7fa --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -0,0 +1,641 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.api.services.dataflow.model.MapTask; +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; +import com.google.cloud.dataflow.sdk.runners.worker.windmill.WindmillServerStub; +import com.google.cloud.dataflow.sdk.util.BoundedQueueExecutor; +import com.google.cloud.dataflow.sdk.util.CloudCounterUtils; +import com.google.cloud.dataflow.sdk.util.StateFetcher; +import com.google.cloud.dataflow.sdk.util.StreamingModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.Transport; +import com.google.cloud.dataflow.sdk.util.UserCodeException; +import com.google.cloud.dataflow.sdk.util.Values; +import com.google.cloud.dataflow.sdk.util.common.Counter; +import com.google.cloud.dataflow.sdk.util.common.CounterSet; +import com.google.cloud.dataflow.sdk.util.common.worker.MapTaskExecutor; + +import org.eclipse.jetty.server.Request; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.AbstractHandler; + +import java.io.IOException; +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.logging.Level; +import java.util.logging.Logger; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +/** + * Implements a Streaming Dataflow worker. + */ +public class StreamingDataflowWorker { + private static final Logger LOG = Logger.getLogger(StreamingDataflowWorker.class.getName()); + static final int MAX_THREAD_POOL_SIZE = 100; + static final long THREAD_EXPIRATION_TIME_SEC = 60; + static final int MAX_THREAD_POOL_QUEUE_SIZE = 100; + static final long MAX_COMMIT_BYTES = 32 << 20; + static final int DEFAULT_STATUS_PORT = 8081; + // Memory threshold under which no new work will be processed. Set to 0 to disable pushback. + static final double PUSHBACK_THRESHOLD = 0.1; + static final String WINDMILL_SERVER_CLASS_NAME = + "com.google.cloud.dataflow.sdk.runners.worker.windmill.WindmillServer"; + + /** + * Indicates that the key token was invalid when data was attempted to be fetched. + */ + public static class KeyTokenInvalidException extends RuntimeException { + public KeyTokenInvalidException(String key) { + super("Unable to fetch data due to token mismatch for key " + key); + } + } + + static MapTask parseMapTask(String input) throws IOException { + return Transport.getJsonFactory() + .fromString(input, MapTask.class); + } + + public static void main(String[] args) throws Exception { + LOG.setLevel(Level.INFO); + String hostport = System.getProperty("windmill.hostport"); + if (hostport == null) { + throw new Exception("-Dwindmill.hostport must be set to the location of the windmill server"); + } + + int statusPort = DEFAULT_STATUS_PORT; + if (System.getProperties().containsKey("status_port")) { + statusPort = Integer.parseInt(System.getProperty("status_port")); + } + + ArrayList mapTasks = new ArrayList<>(); + for (int i = 0; i < args.length; i++) { + mapTasks.add(parseMapTask(args[i])); + } + + WindmillServerStub windmillServer = + (WindmillServerStub) Class.forName(WINDMILL_SERVER_CLASS_NAME) + .getDeclaredConstructor(String.class).newInstance(hostport); + + StreamingDataflowWorker worker = + new StreamingDataflowWorker(mapTasks, windmillServer); + worker.start(); + + worker.runStatusServer(statusPort); + } + + private ConcurrentMap instructionMap; + private ConcurrentMap> outputMap; + private ConcurrentMap> mapTaskExecutors; + private ThreadFactory threadFactory; + private BoundedQueueExecutor executor; + private WindmillServerStub windmillServer; + private Thread dispatchThread; + private Thread commitThread; + private AtomicBoolean running; + private StateFetcher stateFetcher; + private DataflowPipelineOptions options; + private long clientId; + private Server statusServer; + private AtomicReference lastException; + + /** Regular constructor. */ + public StreamingDataflowWorker( + List mapTasks, WindmillServerStub server) { + initialize(mapTasks, server); + options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setAppName("StreamingWorkerHarness"); + options.setStreaming(true); + + if (System.getProperties().containsKey("path_validator_class")) { + try { + options.setPathValidatorClass((Class) Class.forName( + System.getProperty("path_validator_class"))); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Unable to find validator class", e); + } + } + if (System.getProperties().containsKey("credential_factory_class")) { + try { + options.setCredentialFactoryClass((Class) Class.forName( + System.getProperty("credential_factory_class"))); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Unable to find credential factory class", e); + } + } + } + + /** The constructor that takes PipelineOptions. Should be used only by unit tests. */ + StreamingDataflowWorker( + List mapTasks, WindmillServerStub server, DataflowPipelineOptions options) { + initialize(mapTasks, server); + this.options = options; + } + + public void start() { + running.set(true); + dispatchThread = threadFactory.newThread(new Runnable() { + public void run() { + dispatchLoop(); + } + }); + dispatchThread.setPriority(Thread.MIN_PRIORITY); + dispatchThread.setName("DispatchThread"); + dispatchThread.start(); + + commitThread = threadFactory.newThread(new Runnable() { + public void run() { + commitLoop(); + } + }); + commitThread.setPriority(Thread.MAX_PRIORITY); + commitThread.setName("CommitThread"); + commitThread.start(); + } + + public void stop() { + try { + if (statusServer != null) { + statusServer.stop(); + } + running.set(false); + dispatchThread.join(); + executor.shutdown(); + if (!executor.awaitTermination(5, TimeUnit.MINUTES)) { + throw new RuntimeException("Process did not terminate within 5 minutes"); + } + for (ConcurrentLinkedQueue queue : mapTaskExecutors.values()) { + WorkerAndContext workerAndContext; + while ((workerAndContext = queue.poll()) != null) { + workerAndContext.getWorker().close(); + } + } + commitThread.join(); + } catch (Exception e) { + LOG.warning("Exception while shutting down: " + e); + e.printStackTrace(); + } + } + + /** Initializes the execution harness. */ + private void initialize(List mapTasks, WindmillServerStub server) { + this.instructionMap = new ConcurrentHashMap<>(); + this.outputMap = new ConcurrentHashMap<>(); + this.mapTaskExecutors = new ConcurrentHashMap<>(); + for (MapTask mapTask : mapTasks) { + addComputation(mapTask); + } + this.threadFactory = new ThreadFactory() { + private final Thread.UncaughtExceptionHandler handler = + new Thread.UncaughtExceptionHandler() { + public void uncaughtException(Thread thread, Throwable e) { + LOG.severe("Uncaught exception: " + e); + e.printStackTrace(); + System.exit(1); + } + }; + public Thread newThread(Runnable r) { + Thread t = new Thread(r); + t.setUncaughtExceptionHandler(handler); + t.setDaemon(true); + return t; + } + }; + this.executor = new BoundedQueueExecutor( + MAX_THREAD_POOL_SIZE, THREAD_EXPIRATION_TIME_SEC, TimeUnit.SECONDS, + MAX_THREAD_POOL_QUEUE_SIZE, threadFactory); + this.windmillServer = server; + this.running = new AtomicBoolean(); + this.stateFetcher = new StateFetcher(server); + this.clientId = new Random().nextLong(); + this.lastException = new AtomicReference<>(); + } + + public void runStatusServer(int statusPort) { + statusServer = new Server(statusPort); + statusServer.setHandler(new StatusHandler()); + try { + statusServer.start(); + LOG.info("Status server started on port " + statusPort); + statusServer.join(); + } catch (Exception e) { + LOG.warning("Status server failed to start: " + e); + } + } + + private void addComputation(MapTask mapTask) { + String computation = mapTask.getSystemName(); + if (!instructionMap.containsKey(computation)) { + LOG.info("Adding config for " + computation + ": " + mapTask); + outputMap.put(computation, new ConcurrentLinkedQueue()); + instructionMap.put(computation, mapTask); + mapTaskExecutors.put( + computation, + new ConcurrentLinkedQueue()); + } + } + + private static void sleep(int millis) { + try { + Thread.sleep(millis); + } catch (InterruptedException e) { + // NOLINT + } + } + + private void dispatchLoop() { + LOG.info("Dispatch starting"); + Runtime rt = Runtime.getRuntime(); + long lastPushbackLog = 0; + while (running.get()) { + + // If free memory is less than a percentage of total memory, block + // until current work drains and memory is released. + // Also force a GC to try to get under the memory threshold if possible. + while (rt.freeMemory() < rt.totalMemory() * PUSHBACK_THRESHOLD) { + if (lastPushbackLog < (lastPushbackLog = System.currentTimeMillis()) - 60 * 1000) { + LOG.warning("In pushback, not accepting new work. Free Memory: " + + rt.freeMemory() + "MB / " + rt.totalMemory() + "MB"); + System.gc(); + } + sleep(10); + } + + int backoff = 1; + Windmill.GetWorkResponse workResponse; + do { + workResponse = getWork(); + if (workResponse.getWorkCount() > 0) { + break; + } + sleep(backoff); + backoff = Math.min(1000, backoff * 2); + } while (running.get()); + for (final Windmill.ComputationWorkItems computationWork : workResponse.getWorkList()) { + for (final Windmill.WorkItem work : computationWork.getWorkList()) { + final String computation = computationWork.getComputationId(); + if (!instructionMap.containsKey(computation)) { + getConfig(computation); + } + executor.execute(new Runnable() { + public void run() { + process(computation, work); + } + }); + } + } + } + LOG.info("Dispatch done"); + } + + private void process( + final String computation, final Windmill.WorkItem work) { + LOG.log(Level.FINE, "Starting processing for " + computation + ":\n{0}", work); + + MapTask mapTask = instructionMap.get(computation); + if (mapTask == null) { + LOG.info("Received work for unknown computation: " + computation + + ". Known computations are " + instructionMap.keySet()); + return; + } + + Windmill.WorkItemCommitRequest.Builder outputBuilder = + Windmill.WorkItemCommitRequest.newBuilder() + .setKey(work.getKey()) + .setWorkToken(work.getWorkToken()); + + StreamingModeExecutionContext context = null; + MapTaskExecutor worker = null; + + try { + WorkerAndContext workerAndContext = mapTaskExecutors.get(computation).poll(); + if (workerAndContext == null) { + context = new StreamingModeExecutionContext(computation, stateFetcher); + worker = MapTaskExecutorFactory.create(options, mapTask, context); + } else { + worker = workerAndContext.getWorker(); + context = workerAndContext.getContext(); + } + + context.start(work, outputBuilder); + + // Blocks while executing work. + worker.execute(); + + buildCounters(worker.getOutputCounters(), outputBuilder); + + context.flushState(); + + mapTaskExecutors.get(computation).offer(new WorkerAndContext(worker, context)); + worker = null; + context = null; + } catch (Throwable t) { + if (worker != null) { + try { + worker.close(); + } catch (Exception e) { + LOG.warning("Failed to close worker: " + e.getMessage()); + e.printStackTrace(); + } + } + + t = t instanceof UserCodeException ? t.getCause() : t; + + if (t instanceof KeyTokenInvalidException) { + LOG.fine("Execution of work for " + computation + " for key " + work.getKey().toStringUtf8() + + " failed due to token expiration, will not retry locally."); + } else { + LOG.warning("Execution of work for " + computation + " for key " + + work.getKey().toStringUtf8() + " failed, retrying." + + "\nError: " + t.getMessage()); + t.printStackTrace(); + lastException.set(t); + LOG.fine("Failed work: " + work); + reportFailure(computation, work, t); + // Try again, but go to the end of the queue to avoid a tight loop. + sleep(60000); + executor.forceExecute(new Runnable() { + public void run() { + process(computation, work); + } + }); + } + return; + } + + Windmill.WorkItemCommitRequest output = outputBuilder.build(); + outputMap.get(computation).add(output); + LOG.fine("Processing done for work token: " + work.getWorkToken()); + } + + private void commitLoop() { + while (running.get()) { + Windmill.CommitWorkRequest.Builder commitRequestBuilder = + Windmill.CommitWorkRequest.newBuilder(); + long remainingCommitBytes = MAX_COMMIT_BYTES; + for (Map.Entry> entry : + outputMap.entrySet()) { + Windmill.ComputationCommitWorkRequest.Builder computationRequestBuilder = + Windmill.ComputationCommitWorkRequest.newBuilder(); + ConcurrentLinkedQueue queue = entry.getValue(); + while (remainingCommitBytes > 0) { + Windmill.WorkItemCommitRequest request = queue.poll(); + if (request == null) { + break; + } + remainingCommitBytes -= request.getSerializedSize(); + computationRequestBuilder.addRequests(request); + } + if (computationRequestBuilder.getRequestsCount() > 0) { + computationRequestBuilder.setComputationId(entry.getKey()); + commitRequestBuilder.addRequests(computationRequestBuilder); + } + } + if (commitRequestBuilder.getRequestsCount() > 0) { + Windmill.CommitWorkRequest commitRequest = commitRequestBuilder.build(); + LOG.log(Level.FINE, "Commit: {0}", commitRequest); + commitWork(commitRequest); + } + if (remainingCommitBytes > 0) { + sleep(100); + } + } + } + + private Windmill.GetWorkResponse getWork() { + return windmillServer.getWork( + Windmill.GetWorkRequest.newBuilder() + .setClientId(clientId) + .setMaxItems(100) + .build()); + } + + private void commitWork(Windmill.CommitWorkRequest request) { + windmillServer.commitWork(request); + } + + private void getConfig(String computation) { + Windmill.GetConfigRequest request = + Windmill.GetConfigRequest.newBuilder().addComputations(computation).build(); + for (String serializedMapTask : windmillServer.getConfig(request).getCloudWorksList()) { + try { + addComputation(parseMapTask(serializedMapTask)); + } catch (IOException e) { + LOG.warning("Parsing MapTask failed: " + serializedMapTask); + e.printStackTrace(); + } + } + } + + private void buildCounters(CounterSet counterSet, + Windmill.WorkItemCommitRequest.Builder builder) { + for (MetricUpdate metricUpdate : + CloudCounterUtils.extractCounters(counterSet, true /* delta */)) { + Windmill.Counter.Kind kind; + String cloudKind = metricUpdate.getKind(); + if (cloudKind.equals(Counter.AggregationKind.SUM.name())) { + kind = Windmill.Counter.Kind.SUM; + } else if (cloudKind.equals(Counter.AggregationKind.MEAN.name())) { + kind = Windmill.Counter.Kind.MEAN; + } else if (cloudKind.equals(Counter.AggregationKind.MAX.name())) { + kind = Windmill.Counter.Kind.MAX; + } else if (cloudKind.equals(Counter.AggregationKind.MIN.name())) { + kind = Windmill.Counter.Kind.MIN; + } else { + LOG.log(Level.FINE, "Unhandled counter type: " + metricUpdate.getKind()); + return; + } + Windmill.Counter.Builder counterBuilder = builder.addCounterUpdatesBuilder(); + counterBuilder.setName(metricUpdate.getName().getName()).setKind(kind); + Object element = null; + if (kind == Windmill.Counter.Kind.MEAN) { + Object meanCount = metricUpdate.getMeanCount(); + if (meanCount != null) { + try { + Long longValue = Values.asLong(meanCount); + if (longValue != 0) { + counterBuilder.setMeanCount(longValue); + } + } catch (ClassCastException e) { + // Nothing to do. + } + } + element = metricUpdate.getMeanSum(); + } else { + element = metricUpdate.getScalar(); + } + if (element != null) { + try { + Double doubleValue = Values.asDouble(element); + if (doubleValue != 0) { + counterBuilder.setDoubleScalar(doubleValue); + } + } catch (ClassCastException e) { + // Nothing to do. + } + try { + Long longValue = Values.asLong(element); + if (longValue != 0) { + counterBuilder.setIntScalar(longValue); + } + } catch (ClassCastException e) { + // Nothing to do. + } + } + } + } + + private Windmill.Exception buildExceptionReport(Throwable t) { + Windmill.Exception.Builder builder = Windmill.Exception.newBuilder(); + + builder.addStackFrames(t.toString()); + for (StackTraceElement frame : t.getStackTrace()) { + builder.addStackFrames(frame.toString()); + } + if (t.getCause() != null) { + builder.setCause(buildExceptionReport(t.getCause())); + } + + return builder.build(); + } + + private void reportFailure(String computation, Windmill.WorkItem work, Throwable t) { + windmillServer.reportStats(Windmill.ReportStatsRequest.newBuilder() + .setComputationId(computation) + .setKey(work.getKey()) + .setWorkToken(work.getWorkToken()) + .addExceptions(buildExceptionReport(t)) + .build()); + } + + private static class WorkerAndContext { + public MapTaskExecutor worker; + public StreamingModeExecutionContext context; + + public WorkerAndContext(MapTaskExecutor worker, StreamingModeExecutionContext context) { + this.worker = worker; + this.context = context; + } + + public MapTaskExecutor getWorker() { + return worker; + } + + public StreamingModeExecutionContext getContext() { + return context; + } + } + + private class StatusHandler extends AbstractHandler { + @Override + public void handle( + String target, Request baseRequest, + HttpServletRequest request, HttpServletResponse response) + throws IOException, ServletException { + + response.setContentType("text/html;charset=utf-8"); + response.setStatus(HttpServletResponse.SC_OK); + baseRequest.setHandled(true); + + PrintWriter responseWriter = response.getWriter(); + + responseWriter.println(""); + + printHeader(responseWriter); + + printMetrics(responseWriter); + + printResources(responseWriter); + + printLastException(responseWriter); + + printSpecs(responseWriter); + + responseWriter.println(""); + } + } + + private void printHeader(PrintWriter response) { + response.println("

    Streaming Worker Harness

    "); + response.println("Running: " + running.get() + "
    "); + response.println("ID: " + clientId + "
    "); + } + + private void printMetrics(PrintWriter response) { + response.println("

    Metrics

    "); + response.println("Worker Threads: " + executor.getPoolSize() + + "/" + MAX_THREAD_POOL_QUEUE_SIZE + "
    "); + response.println("Active Threads: " + executor.getActiveCount() + "
    "); + response.println("Work Queue Size: " + executor.getQueue().size() + "
    "); + response.println("Commit Queues:
      "); + for (Map.Entry> entry + : outputMap.entrySet()) { + response.print("
    • "); + response.print(entry.getKey()); + response.print(": "); + response.print(entry.getValue().size()); + response.println("
    • "); + } + response.println("
    "); + } + + private void printResources(PrintWriter response) { + Runtime rt = Runtime.getRuntime(); + int mb = 1024 * 1024; + + response.append("

    Resources

    \n"); + response.append("Total Memory: " + rt.totalMemory() / mb + "MB
    \n"); + response.append("Used Memory: " + (rt.totalMemory() - rt.freeMemory()) / mb + "MB
    \n"); + response.append("Max Memory: " + rt.maxMemory() / mb + "MB
    \n"); + } + + private void printSpecs(PrintWriter response) { + response.append("

    Specs

    \n"); + for (Map.Entry entry : instructionMap.entrySet()) { + response.println("

    " + entry.getKey() + "

    "); + response.print(""); + } + } + + private void printLastException(PrintWriter response) { + Throwable t = lastException.get(); + if (t != null) { + response.println("

    Last Exception

    "); + StringWriter writer = new StringWriter(); + t.printStackTrace(new PrintWriter(writer)); + response.println(writer.toString().replace("\t", "  ").replace("\n", "
    ")); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedWindmillReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedWindmillReader.java new file mode 100644 index 0000000000000..bbe63746965c1 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/UngroupedWindmillReader.java @@ -0,0 +1,116 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.StreamingModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; +import com.google.cloud.dataflow.sdk.values.KV; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Collection; +import java.util.concurrent.TimeUnit; + +/** + * A Reader that receives input data from a Windmill server, and returns it as + * individual elements. + */ +class UngroupedWindmillReader extends Reader> { + private final Coder valueCoder; + private final Coder> windowsCoder; + private StreamingModeExecutionContext context; + + UngroupedWindmillReader(Coder> coder, StreamingModeExecutionContext context) { + FullWindowedValueCoder inputCoder = (FullWindowedValueCoder) coder; + this.valueCoder = inputCoder.getValueCoder(); + this.windowsCoder = inputCoder.getWindowsCoder(); + this.context = context; + } + + public static UngroupedWindmillReader create(PipelineOptions options, + CloudObject spec, + Coder coder, + ExecutionContext context) { + return new UngroupedWindmillReader<>(coder, (StreamingModeExecutionContext) context); + } + + @Override + public ReaderIterator> iterator() throws IOException { + return new UngroupedWindmillReaderIterator(); + } + + class UngroupedWindmillReaderIterator extends AbstractReaderIterator> { + private int bundleIndex = 0; + private int messageIndex = 0; + + @Override + public boolean hasNext() throws IOException { + Windmill.WorkItem work = context.getWork(); + return bundleIndex < work.getMessageBundlesCount() && + messageIndex < work.getMessageBundles(bundleIndex).getMessagesCount(); + } + + @Override + public WindowedValue next() throws IOException { + Windmill.Message message = + context.getWork().getMessageBundles(bundleIndex).getMessages(messageIndex); + if (messageIndex >= + context.getWork().getMessageBundles(bundleIndex).getMessagesCount() - 1) { + messageIndex = 0; + bundleIndex++; + } else { + messageIndex++; + } + Instant timestampMillis = new Instant(TimeUnit.MICROSECONDS.toMillis(message.getTimestamp())); + InputStream data = message.getData().newInput(); + InputStream metadata = message.getMetadata().newInput(); + Collection windows = decode(windowsCoder, metadata); + if (valueCoder instanceof KvCoder) { + KvCoder kvCoder = (KvCoder) valueCoder; + InputStream key = context.getSerializedKey().newInput(); + notifyElementRead(key.available() + data.available() + metadata.available()); + return WindowedValue.of((T) KV.of(decode(kvCoder.getKeyCoder(), key), + decode(kvCoder.getValueCoder(), data)), + timestampMillis, + windows); + } else { + notifyElementRead(data.available() + metadata.available()); + return WindowedValue.of(decode(valueCoder, data), timestampMillis, windows); + } + } + + private S decode(Coder coder, InputStream input) throws IOException { + return coder.decode(input, Coder.Context.OUTER); + } + } + + @Override + public boolean supportsRestart() { + return true; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillSink.java new file mode 100644 index 0000000000000..3bfcac116bf47 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillSink.java @@ -0,0 +1,130 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.getString; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.StreamingModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.worker.Sink; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.protobuf.ByteString; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +class WindmillSink extends Sink> { + private WindmillStreamWriter writer; + private final Coder valueCoder; + private final Coder windowsCoder; + private StreamingModeExecutionContext context; + + WindmillSink(String destinationName, + Coder> coder, + StreamingModeExecutionContext context) { + this.writer = new WindmillStreamWriter(destinationName); + FullWindowedValueCoder inputCoder = (FullWindowedValueCoder) coder; + this.valueCoder = inputCoder.getValueCoder(); + this.windowsCoder = inputCoder.getWindowsCoder(); + this.context = context; + } + + public static WindmillSink create(PipelineOptions options, + CloudObject spec, + Coder> coder, + ExecutionContext context) throws Exception { + return new WindmillSink<>(getString(spec, "stream_id"), coder, + (StreamingModeExecutionContext) context); + } + + @Override + public SinkWriter> writer() { + return writer; + } + + class WindmillStreamWriter implements SinkWriter> { + private Map productionMap; + private final String destinationName; + + private WindmillStreamWriter(String destinationName) { + this.destinationName = destinationName; + productionMap = new HashMap(); + } + + private ByteString encode(Coder coder, S object) throws IOException { + ByteString.Output stream = ByteString.newOutput(); + coder.encode(object, stream, Coder.Context.OUTER); + return stream.toByteString(); + } + + @Override + public long add(WindowedValue data) throws IOException { + ByteString key, value; + ByteString windows = encode(windowsCoder, data.getWindows()); + if (valueCoder instanceof KvCoder) { + KvCoder kvCoder = (KvCoder) valueCoder; + KV kv = (KV) data.getValue(); + key = encode(kvCoder.getKeyCoder(), kv.getKey()); + value = encode(kvCoder.getValueCoder(), kv.getValue()); + } else { + key = context.getSerializedKey(); + value = encode(valueCoder, data.getValue()); + } + Windmill.KeyedMessageBundle.Builder keyedOutput = productionMap.get(key); + if (keyedOutput == null) { + keyedOutput = Windmill.KeyedMessageBundle.newBuilder().setKey(key); + productionMap.put(key, keyedOutput); + } + + long timestampMicros = TimeUnit.MILLISECONDS.toMicros(data.getTimestamp().getMillis()); + Windmill.Message.Builder builder = Windmill.Message.newBuilder() + .setTimestamp(timestampMicros) + .setData(value) + .setMetadata(windows); + keyedOutput.addMessages(builder.build()); + return key.size() + value.size() + windows.size(); + } + + @Override + public void close() throws IOException { + Windmill.OutputMessageBundle.Builder outputBuilder = + Windmill.OutputMessageBundle.newBuilder().setDestinationStreamId(destinationName); + + for (Windmill.KeyedMessageBundle.Builder keyedOutput : productionMap.values()) { + outputBuilder.addBundles(keyedOutput.build()); + } + if (outputBuilder.getBundlesCount() > 0) { + context.getOutputBuilder().addOutputMessages(outputBuilder.build()); + } + productionMap.clear(); + } + } + + @Override + public boolean supportsRestart() { + return true; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindowingWindmillReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindowingWindmillReader.java new file mode 100644 index 0000000000000..8bd6d89ec6e2f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindowingWindmillReader.java @@ -0,0 +1,157 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.TimerOrElement.TimerOrElementCoder; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.StreamingModeExecutionContext; +import com.google.cloud.dataflow.sdk.util.TimerOrElement; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; +import com.google.cloud.dataflow.sdk.values.KV; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.TimeUnit; + +/** + * A Reader that receives input data from a Windmill server, and returns it as + * groups of elements and timers. + */ +class WindowingWindmillReader extends Reader>> { + private final Coder valueCoder; + private final Coder> windowsCoder; + private StreamingModeExecutionContext context; + + WindowingWindmillReader(Coder>> coder, + StreamingModeExecutionContext context) { + FullWindowedValueCoder> inputCoder = + (FullWindowedValueCoder>) coder; + this.windowsCoder = inputCoder.getWindowsCoder(); + this.valueCoder = ((TimerOrElementCoder) inputCoder.getValueCoder()).getElementCoder(); + this.context = context; + } + + public static WindowingWindmillReader create(PipelineOptions options, + CloudObject spec, + Coder coder, + ExecutionContext context) { + return new WindowingWindmillReader<>(coder, (StreamingModeExecutionContext) context); + } + + @Override + public ReaderIterator>> iterator() throws IOException { + return new WindowingWindmillReaderIterator(); + } + + class WindowingWindmillReaderIterator + extends AbstractReaderIterator>> { + private int bundleIndex = 0; + private int messageIndex = 0; + private int timerIndex = 0; + + private boolean hasMoreMessages() { + Windmill.WorkItem work = context.getWork(); + return bundleIndex < work.getMessageBundlesCount() && + messageIndex < work.getMessageBundles(bundleIndex).getMessagesCount(); + } + + private boolean hasMoreTimers() { + Windmill.WorkItem work = context.getWork(); + return work.hasTimers() && timerIndex < work.getTimers().getTimersCount(); + } + + @Override + public boolean hasNext() throws IOException { + return hasMoreMessages() || hasMoreTimers(); + } + + @Override + public WindowedValue> next() throws IOException { + if (hasMoreTimers()) { + if (valueCoder instanceof KvCoder) { + Windmill.Timer timer = context.getWork().getTimers().getTimers(timerIndex++); + long timestampMillis = TimeUnit.MICROSECONDS.toMillis(timer.getTimestamp()); + + KvCoder kvCoder = (KvCoder) valueCoder; + Object key = kvCoder.getKeyCoder().decode( + context.getSerializedKey().newInput(), Coder.Context.OUTER); + + return WindowedValue.of(TimerOrElement.timer(timer.getTag().toStringUtf8(), + new Instant(timestampMillis), + key), + new Instant(timestampMillis), + new ArrayList()); + } else { + throw new RuntimeException("Timer set on non-keyed DoFn"); + } + } else { + Windmill.Message message = + context.getWork().getMessageBundles(bundleIndex).getMessages(messageIndex); + + if (messageIndex >= + context.getWork().getMessageBundles(bundleIndex).getMessagesCount() - 1) { + messageIndex = 0; + bundleIndex++; + } else { + messageIndex++; + } + Instant timestampMillis = + new Instant(TimeUnit.MICROSECONDS.toMillis(message.getTimestamp())); + InputStream data = message.getData().newInput(); + InputStream metadata = message.getMetadata().newInput(); + Collection windows = decode(windowsCoder, metadata); + if (valueCoder instanceof KvCoder) { + KvCoder kvCoder = (KvCoder) valueCoder; + InputStream key = context.getSerializedKey().newInput(); + notifyElementRead(key.available() + data.available() + metadata.available()); + return WindowedValue.of( + TimerOrElement.element((T) KV.of(decode(kvCoder.getKeyCoder(), key), + decode(kvCoder.getValueCoder(), data))), + timestampMillis, + windows); + } else { + notifyElementRead(data.available() + metadata.available()); + return WindowedValue.of(TimerOrElement.element(decode(valueCoder, data)), + timestampMillis, + windows); + } + } + } + + private S decode(Coder coder, InputStream input) throws IOException { + return coder.decode(input, Coder.Context.OUTER); + } + } + + @Override + public boolean supportsRestart() { + return true; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/windmill/WindmillServerStub.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/windmill/WindmillServerStub.java new file mode 100644 index 0000000000000..105c8486db446 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/windmill/WindmillServerStub.java @@ -0,0 +1,45 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker.windmill; + +/** + * Stub for communicating with a Windmill server. + */ +public abstract class WindmillServerStub { + /** + * Get a batch of work to process. + */ + public abstract Windmill.GetWorkResponse getWork(Windmill.GetWorkRequest request); + /** + * Get addition data such as state needed to process work. + */ + public abstract Windmill.GetDataResponse getData(Windmill.GetDataRequest request); + /** + * Commit the work, issuing any output productions, state modifications etc. + */ + public abstract Windmill.CommitWorkResponse commitWork( + Windmill.CommitWorkRequest request); + /** + * Get configuration data from the server. + */ + public abstract Windmill.GetConfigResponse getConfig(Windmill.GetConfigRequest request); + + /** + * Report execution information to the server. + */ + public abstract Windmill.ReportStatsResponse reportStats(Windmill.ReportStatsRequest request); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BoundedQueueExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BoundedQueueExecutor.java new file mode 100644 index 0000000000000..983946915c205 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BoundedQueueExecutor.java @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +/** + * Executor that blocks on execute() if its queue is full. + */ +public class BoundedQueueExecutor extends ThreadPoolExecutor { + private static class ReducableSemaphore extends Semaphore { + ReducableSemaphore(int permits) { + super(permits); + } + + public void reducePermits(int permits) { + super.reducePermits(permits); + } + } + private ReducableSemaphore semaphore; + + public BoundedQueueExecutor(int maximumPoolSize, + long keepAliveTime, + TimeUnit unit, + int maximumQueueSize, + ThreadFactory threadFactory) { + super(maximumQueueSize, maximumPoolSize, keepAliveTime, unit, + new LinkedBlockingQueue(), threadFactory); + this.semaphore = new ReducableSemaphore(maximumQueueSize); + allowCoreThreadTimeOut(true); + } + + // Before adding a Runnable to the queue, acquire the semaphore. + @Override + public void execute(Runnable r) { + semaphore.acquireUninterruptibly(); + super.execute(r); + } + + // Forcibly add something to the queue, ignoring the length limit. + public void forceExecute(Runnable r) { + semaphore.reducePermits(1); + super.execute(r); + } + + // Release the semaphore after taking a Runnable off the queue. + @Override + public void beforeExecute(Thread t, Runnable r) { + semaphore.release(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StateFetcher.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StateFetcher.java new file mode 100644 index 0000000000000..c3fae8851f563 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StateFetcher.java @@ -0,0 +1,147 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.runners.worker.StreamingDataflowWorker; +import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; +import com.google.cloud.dataflow.sdk.runners.worker.windmill.WindmillServerStub; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; +import com.google.protobuf.ByteString; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Class responsible for fetching state from the windmill server. + */ +public class StateFetcher { + private WindmillServerStub server; + + public StateFetcher(WindmillServerStub server) { + this.server = server; + } + + public Map, Object> fetch( + String computation, ByteString key, long workToken, String prefix, + List> tags) throws CoderException, IOException { + Map, Object> resultMap = new HashMap<>(); + if (tags.isEmpty()) { + return resultMap; + } + + Windmill.KeyedGetDataRequest.Builder requestBuilder = Windmill.KeyedGetDataRequest.newBuilder() + .setKey(key) + .setWorkToken(workToken); + + Map> tagMap = new HashMap<>(); + for (CodedTupleTag tag : tags) { + ByteString tagString = ByteString.copyFromUtf8(prefix + tag.getId()); + requestBuilder.addValuesToFetch( + Windmill.TagValue.newBuilder() + .setTag(tagString) + .build()); + tagMap.put(tagString, tag); + } + + Windmill.GetDataResponse response = server.getData( + Windmill.GetDataRequest.newBuilder() + .addRequests( + Windmill.ComputationGetDataRequest.newBuilder() + .setComputationId(computation) + .addRequests(requestBuilder.build()) + .build()) + .build()); + + if (response.getDataCount() != 1 + || !response.getData(0).getComputationId().equals(computation) + || response.getData(0).getDataCount() != 1 + || !response.getData(0).getData(0).getKey().equals(key)) { + throw new IOException("Invalid data response, expected single computation and key"); + } + Windmill.KeyedGetDataResponse keyResponse = response.getData(0).getData(0); + if (keyResponse.getFailed()) { + throw new StreamingDataflowWorker.KeyTokenInvalidException(key.toStringUtf8()); + } + + for (Windmill.TagValue tv : keyResponse.getValuesList()) { + CodedTupleTag tag = tagMap.get(tv.getTag()); + if (tag != null) { + if (tv.getValue().hasData() && !tv.getValue().getData().isEmpty()) { + resultMap.put(tag, tag.getCoder().decode(tv.getValue().getData().newInput(), + Coder.Context.OUTER)); + } else { + resultMap.put(tag, null); + } + } + } + + return resultMap; + } + + public List fetchList( + String computation, ByteString key, long workToken, String prefix, CodedTupleTag tag) + throws IOException { + + ByteString tagString = ByteString.copyFromUtf8(prefix + tag.getId()); + Windmill.GetDataRequest request = Windmill.GetDataRequest.newBuilder() + .addRequests( + Windmill.ComputationGetDataRequest.newBuilder() + .setComputationId(computation) + .addRequests( + Windmill.KeyedGetDataRequest.newBuilder() + .setKey(key) + .setWorkToken(workToken) + .addListsToFetch( + Windmill.TagList.newBuilder() + .setTag(tagString) + .setEndTimestamp(Long.MAX_VALUE) + .build()) + .build()) + .build()) + .build(); + + Windmill.GetDataResponse response = server.getData(request); + + if (response.getDataCount() != 1 + || !response.getData(0).getComputationId().equals(computation) + || response.getData(0).getDataCount() != 1 + || !response.getData(0).getData(0).getKey().equals(key)) { + throw new IOException("Invalid data response, expected single computation and key\n"); + } + + Windmill.KeyedGetDataResponse keyResponse = response.getData(0).getData(0); + if (keyResponse.getFailed()) { + throw new StreamingDataflowWorker.KeyTokenInvalidException(key.toStringUtf8()); + } + if (keyResponse.getListsCount() != 1 + || !keyResponse.getLists(0).getTag().equals(tagString)) { + throw new IOException("Expected single list for tag " + tagString); + } + Windmill.TagList tagList = keyResponse.getLists(0); + List result = new ArrayList<>(); + for (Windmill.Value value : tagList.getValuesList()) { + result.add(tag.getCoder().decode(value.getData().newInput(), Coder.Context.OUTER)); + } + + return result; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingModeExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingModeExecutionContext.java new file mode 100644 index 0000000000000..13801339126b7 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingModeExecutionContext.java @@ -0,0 +1,208 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; +import com.google.cloud.dataflow.sdk.values.CodedTupleTagMap; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.protobuf.ByteString; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * {@link ExecutionContext} for use in streaming mode. + */ +public class StreamingModeExecutionContext extends ExecutionContext { + private String computation; + private Windmill.WorkItem work; + private StateFetcher stateFetcher; + private Windmill.WorkItemCommitRequest.Builder outputBuilder; + + public StreamingModeExecutionContext(String computation, StateFetcher stateFetcher) { + this.computation = computation; + this.stateFetcher = stateFetcher; + } + + public void start(Windmill.WorkItem work, Windmill.WorkItemCommitRequest.Builder outputBuilder) { + this.work = work; + this.outputBuilder = outputBuilder; + } + + @Override + public ExecutionContext.StepContext createStepContext(String stepName) { + return new StepContext(stepName); + } + + @Override + public void setTimer(String timer, Instant timestamp) { + long timestampMicros = TimeUnit.MILLISECONDS.toMicros(timestamp.getMillis()); + outputBuilder.addOutputTimers( + Windmill.Timer.newBuilder() + .setTimestamp(timestampMicros) + .setTag(ByteString.copyFromUtf8(timer)) + .build()); + } + + @Override + public void deleteTimer(String timer) { + outputBuilder.addOutputTimers( + Windmill.Timer.newBuilder().setTag(ByteString.copyFromUtf8(timer)).build()); + } + + public ByteString getSerializedKey() { + return work.getKey(); + } + + public long getWorkToken() { + return work.getWorkToken(); + } + + public Windmill.WorkItem getWork() { + return work; + } + + public Windmill.WorkItemCommitRequest.Builder getOutputBuilder() { + return outputBuilder; + } + + public void flushState() { + for (ExecutionContext.StepContext stepContext : getAllStepContexts()) { + ((StepContext) stepContext).flushState(); + } + } + + public Map, Object> lookupState( + String prefix, List> tags) throws CoderException, IOException { + return stateFetcher.fetch(computation, getSerializedKey(), getWorkToken(), prefix, tags); + } + + class StepContext extends ExecutionContext.StepContext { + private final String mangledPrefix; + private Map, KV> stateCache = new HashMap<>(); + private Map, List>> tagListUpdates = new HashMap<>(); + + public StepContext(String stepName) { + super(stepName); + // Mangle such that there are no partially overlapping prefixes. + this.mangledPrefix = stepName.length() + ":" + stepName; + } + + @Override + public void store(CodedTupleTag tag, T value) throws CoderException, IOException { + ByteString.Output stream = ByteString.newOutput(); + tag.getCoder().encode(value, stream, Coder.Context.OUTER); + stateCache.put(tag, KV.of(value, stream.toByteString())); + } + + @Override + public CodedTupleTagMap lookup(List> tags) + throws CoderException, IOException { + List> tagsToLookup = new ArrayList<>(); + List> residentTags = new ArrayList<>(); + for (CodedTupleTag tag : tags) { + if (stateCache.containsKey(tag)) { + residentTags.add(tag); + } else { + tagsToLookup.add(tag); + } + } + Map, Object> result = + StreamingModeExecutionContext.this.lookupState(mangledPrefix, tagsToLookup); + for (CodedTupleTag tag : residentTags) { + result.put(tag, stateCache.get(tag).getKey()); + } + return CodedTupleTagMap.of(result); + } + + @Override + public void writeToTagList(CodedTupleTag tag, T value, Instant timestamp) + throws IOException { + List> list = tagListUpdates.get(tag); + if (list == null) { + list = new ArrayList<>(); + tagListUpdates.put(tag, list); + } + ByteString.Output stream = ByteString.newOutput(); + tag.getCoder().encode(value, stream, Coder.Context.OUTER); + list.add(KV.of(stream.toByteString(), timestamp)); + } + + @Override + public Iterable readTagList(CodedTupleTag tag) throws IOException { + return stateFetcher.fetchList( + computation, getSerializedKey(), getWorkToken(), mangledPrefix, tag); + } + + @Override + public void deleteTagList(CodedTupleTag tag) { + outputBuilder.addListUpdates( + Windmill.TagList.newBuilder() + .setTag(serializeTag(tag)) + .setEndTimestamp(Long.MAX_VALUE) + .build()); + } + + public void flushState() { + for (Map.Entry, KV> entry : stateCache.entrySet()) { + CodedTupleTag tag = entry.getKey(); + ByteString encodedValue = entry.getValue().getValue(); + outputBuilder.addValueUpdates( + Windmill.TagValue.newBuilder() + .setTag(serializeTag(tag)) + .setValue( + Windmill.Value.newBuilder() + .setData(encodedValue) + .setTimestamp(Long.MAX_VALUE) + .build()) + .build()); + } + + for (Map.Entry, List>> entry : + tagListUpdates.entrySet()) { + CodedTupleTag tag = entry.getKey(); + Windmill.TagList.Builder listBuilder = + Windmill.TagList.newBuilder() + .setTag(serializeTag(tag)); + for (KV item : entry.getValue()) { + long timestampMicros = TimeUnit.MILLISECONDS.toMicros(item.getValue().getMillis()); + listBuilder.addValues( + Windmill.Value.newBuilder() + .setData(item.getKey()) + .setTimestamp(timestampMicros)); + } + outputBuilder.addListUpdates(listBuilder.build()); + } + + stateCache.clear(); + tagListUpdates.clear(); + } + + private ByteString serializeTag(CodedTupleTag tag) { + return ByteString.copyFromUtf8(mangledPrefix + tag.getId()); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java new file mode 100644 index 0000000000000..9c5c5cbd24f0c --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java @@ -0,0 +1,913 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.util.Structs.addObject; +import static com.google.cloud.dataflow.sdk.util.Structs.addString; + +import com.google.api.services.dataflow.model.InstructionInput; +import com.google.api.services.dataflow.model.InstructionOutput; +import com.google.api.services.dataflow.model.MapTask; +import com.google.api.services.dataflow.model.ParDoInstruction; +import com.google.api.services.dataflow.model.ParallelInstruction; +import com.google.api.services.dataflow.model.ReadInstruction; +import com.google.api.services.dataflow.model.Sink; +import com.google.api.services.dataflow.model.Source; +import com.google.api.services.dataflow.model.WriteInstruction; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CollectionCoder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.ListCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; +import com.google.cloud.dataflow.sdk.runners.worker.windmill.WindmillServerStub; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.DoFnInfo; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.util.StringUtils; +import com.google.cloud.dataflow.sdk.util.TimerOrElement.TimerOrElementCoder; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue.FullWindowedValueCoder; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; +import com.google.cloud.dataflow.sdk.values.CodedTupleTagMap; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.protobuf.ByteString; +import com.google.protobuf.TextFormat; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; +import org.junit.matchers.JUnitMatchers; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +/** Unit tests for {@link StreamingDataflowWorker}. */ +@RunWith(JUnit4.class) +public class StreamingDataflowWorkerTest { + private static final IntervalWindow DEFAULT_WINDOW = + new IntervalWindow(new Instant(1234), new Duration(1000)); + + private static class FakeWindmillServer extends WindmillServerStub { + private Queue workToOffer; + private Queue dataToOffer; + private Map commitsReceived; + private LinkedBlockingQueue exceptions; + private int commitsRequested = 0; + + public FakeWindmillServer() { + workToOffer = new ConcurrentLinkedQueue(); + dataToOffer = new ConcurrentLinkedQueue(); + commitsReceived = new ConcurrentHashMap(); + exceptions = new LinkedBlockingQueue<>(); + } + + public void addWorkToOffer(Windmill.GetWorkResponse work) { + workToOffer.add(work); + } + + public void addDataToOffer(Windmill.GetDataResponse data) { + dataToOffer.add(data); + } + + @Override + public Windmill.GetWorkResponse getWork(Windmill.GetWorkRequest request) { + Windmill.GetWorkResponse response = workToOffer.poll(); + if (response == null) { + return Windmill.GetWorkResponse.newBuilder().build(); + } + return response; + } + + @Override + public Windmill.GetDataResponse getData(Windmill.GetDataRequest request) { + Windmill.GetDataResponse response = dataToOffer.poll(); + if (response == null) { + return Windmill.GetDataResponse.newBuilder().build(); + } + return response; + } + + @Override + public Windmill.CommitWorkResponse commitWork(Windmill.CommitWorkRequest request) { + for (Windmill.ComputationCommitWorkRequest computationRequest : request.getRequestsList()) { + for (Windmill.WorkItemCommitRequest commit : computationRequest.getRequestsList()) { + commitsReceived.put(commit.getWorkToken(), commit); + } + } + return Windmill.CommitWorkResponse.newBuilder().build(); + } + + @Override + public Windmill.GetConfigResponse getConfig(Windmill.GetConfigRequest request) { + return Windmill.GetConfigResponse.newBuilder().build(); + } + + @Override + public Windmill.ReportStatsResponse reportStats(Windmill.ReportStatsRequest request) { + for (Windmill.Exception exception : request.getExceptionsList()) { + try { + exceptions.put(exception); + } catch (InterruptedException e) {} + } + return Windmill.ReportStatsResponse.newBuilder().build(); + } + + public Map waitForAndGetCommits(int numCommits) { + while (commitsReceived.size() < commitsRequested + numCommits) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) {} + } + + commitsRequested += numCommits; + + return commitsReceived; + } + + public Windmill.Exception getException() throws InterruptedException { + return exceptions.take(); + } + } + + private ParallelInstruction makeWindowingSourceInstruction(Coder coder) { + CloudObject encodedCoder = FullWindowedValueCoder.of( + TimerOrElementCoder.of(coder), IntervalWindow.getCoder()).asCloudObject(); + return new ParallelInstruction() + .setSystemName("source") + .setRead(new ReadInstruction().setSource( + new Source() + .setSpec(CloudObject.forClass(WindowingWindmillReader.class)) + .setCodec(encodedCoder))) + .setOutputs(Arrays.asList( + new InstructionOutput() + .setName("read_output") + .setCodec(encodedCoder))); + } + + private ParallelInstruction makeSourceInstruction(Coder coder) { + return new ParallelInstruction() + .setSystemName("source") + .setRead(new ReadInstruction().setSource( + new Source() + .setSpec(CloudObject.forClass(UngroupedWindmillReader.class)) + .setCodec(WindowedValue.getFullCoder(coder, IntervalWindow.getCoder()) + .asCloudObject()))) + .setOutputs(Arrays.asList( + new InstructionOutput() + .setName("read_output") + .setCodec(WindowedValue.getFullCoder(coder, IntervalWindow.getCoder()) + .asCloudObject()))); + } + + private ParallelInstruction makeDoFnInstruction( + DoFn doFn, int producerIndex, Coder outputCoder) { + CloudObject spec = CloudObject.forClassName("DoFn"); + addString(spec, PropertyNames.SERIALIZED_FN, + StringUtils.byteArrayToJsonString( + SerializableUtils.serializeToByteArray(new DoFnInfo(doFn, null)))); + return new ParallelInstruction() + .setSystemName("parDo") + .setParDo(new ParDoInstruction() + .setInput( + new InstructionInput().setProducerInstructionIndex(producerIndex).setOutputNum(0)) + .setNumOutputs(1) + .setUserFn(spec)) + .setOutputs(Arrays.asList( + new InstructionOutput() + .setName("par_do_output") + .setCodec(WindowedValue.getFullCoder(outputCoder, IntervalWindow.getCoder()) + .asCloudObject()))); + } + + private ParallelInstruction makeSinkInstruction(Coder coder, int producerIndex) { + CloudObject spec = CloudObject.forClass(WindmillSink.class); + addString(spec, "stream_id", "out"); + return new ParallelInstruction() + .setSystemName("sink") + .setWrite(new WriteInstruction() + .setInput( + new InstructionInput().setProducerInstructionIndex(producerIndex).setOutputNum(0)) + .setSink(new Sink() + .setSpec(spec) + .setCodec(WindowedValue.getFullCoder(coder, IntervalWindow.getCoder()) + .asCloudObject()))); + } + + private MapTask makeMapTask(List instructions) { + return new MapTask() + .setStageName("computation") + .setSystemName("computation") + .setInstructions(instructions); + } + + private Windmill.GetWorkResponse buildTimerInput(String input) throws Exception { + Windmill.GetWorkResponse.Builder builder = Windmill.GetWorkResponse.newBuilder(); + TextFormat.merge(input, builder); + return builder.build(); + } + + private Windmill.GetWorkResponse buildInput(String input, byte[] metadata) throws Exception { + Windmill.GetWorkResponse.Builder builder = Windmill.GetWorkResponse.newBuilder(); + TextFormat.merge(input, builder); + Windmill.InputMessageBundle.Builder messageBundleBuilder = + builder.getWorkBuilder(0) + .getWorkBuilder(0) + .getMessageBundlesBuilder(0); + for (Windmill.Message.Builder messageBuilder : messageBundleBuilder.getMessagesBuilderList()) { + messageBuilder.setMetadata(ByteString.copyFrom(metadata)); + } + return builder.build(); + } + + private Windmill.GetDataResponse buildData(String input) throws Exception { + Windmill.GetDataResponse.Builder builder = Windmill.GetDataResponse.newBuilder(); + TextFormat.merge(input, builder); + return builder.build(); + } + + private Windmill.GetWorkResponse makeInput(int index, long timestamp) throws Exception { + return buildInput( + "work {" + + " computation_id: \"computation\"" + + " work {" + + " key: \"key\"" + + " work_token: " + index + + " message_bundles {" + + " source_computation_id: \"upstream\"" + + " messages {" + + " timestamp: " + timestamp + + " data: \"data" + index + "\"" + + " }" + + " }" + + " }" + + "}", + CoderUtils.encodeToByteArray( + CollectionCoder.of(IntervalWindow.getCoder()), Arrays.asList(DEFAULT_WINDOW))); + } + + private Windmill.WorkItemCommitRequest buildExpectedOutput(String output) throws Exception { + Windmill.WorkItemCommitRequest.Builder builder = Windmill.WorkItemCommitRequest.newBuilder(); + TextFormat.merge(output, builder); + return builder.build(); + } + + private Windmill.WorkItemCommitRequest buildExpectedOutput(String output, byte[] metadata) + throws Exception { + Windmill.WorkItemCommitRequest.Builder builder = Windmill.WorkItemCommitRequest.newBuilder(); + TextFormat.merge(output, builder); + builder.getOutputMessagesBuilder(0) + .getBundlesBuilder(0) + .getMessagesBuilder(0) + .setMetadata(ByteString.copyFrom(metadata)); + return builder.build(); + } + + private Windmill.WorkItemCommitRequest makeExpectedOutput(int index, long timestamp, String key) + throws Exception { + return buildExpectedOutput( + "key: \"key\" " + + "work_token: " + index + " " + + "output_messages {" + + " destination_stream_id: \"out\"" + + " bundles {" + + " key: \"" + key + "\"" + + " messages {" + + " timestamp: " + timestamp + + " data: \"data" + index + "\"" + + " metadata: \"\"" + + " }" + + " }" + + "}", + CoderUtils.encodeToByteArray(CollectionCoder.of(IntervalWindow.getCoder()), + Arrays.asList(DEFAULT_WINDOW))); + } + + private DataflowPipelineOptions createTestingPipelineOptions() { + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options.setAppName("StreamingWorkerHarnessTest"); + options.setStreaming(true); + return options; + } + + private Windmill.WorkItemCommitRequest stripCounters(Windmill.WorkItemCommitRequest request) { + return Windmill.WorkItemCommitRequest.newBuilder(request).clearCounterUpdates().build(); + } + + private Windmill.WorkItemCommitRequest stripProcessingTimeCounters( + Windmill.WorkItemCommitRequest request) { + Windmill.WorkItemCommitRequest.Builder builder = + Windmill.WorkItemCommitRequest.newBuilder(request).clearCounterUpdates(); + TreeMap sortedCounters = new TreeMap<>(); + for (Windmill.Counter counter : request.getCounterUpdatesList()) { + String name = counter.getName(); + if (!(name.startsWith("computation-") && name.endsWith("-msecs"))) { + sortedCounters.put(name, counter); + } + } + for (Windmill.Counter counter : sortedCounters.values()) { + builder.addCounterUpdates(counter); + } + return builder.build(); + } + + + @Test public void testBasicHarness() throws Exception { + List instructions = Arrays.asList( + makeSourceInstruction(StringUtf8Coder.of()), + makeSinkInstruction(StringUtf8Coder.of(), 0)); + + FakeWindmillServer server = new FakeWindmillServer(); + StreamingDataflowWorker worker = new StreamingDataflowWorker( + Arrays.asList(makeMapTask(instructions)), server, createTestingPipelineOptions()); + worker.start(); + + final int numIters = 2000; + for (int i = 0; i < numIters; ++i) { + server.addWorkToOffer(makeInput(i, TimeUnit.MILLISECONDS.toMicros(i))); + } + + Map result = server.waitForAndGetCommits(numIters); + worker.stop(); + + for (int i = 0; i < numIters; ++i) { + Assert.assertTrue(result.containsKey((long) i)); + Assert.assertEquals(makeExpectedOutput(i, TimeUnit.MILLISECONDS.toMicros(i), "key"), + stripCounters(result.get((long) i))); + } + } + + static class ChangeKeysFn extends DoFn, KV> { + @Override + public void processElement(ProcessContext c) { + KV elem = c.element(); + c.output(KV.of(elem.getKey() + "_" + elem.getValue(), elem.getValue())); + } + } + + @Test public void testKeyChange() throws Exception { + KvCoder kvCoder = KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); + + List instructions = Arrays.asList( + makeSourceInstruction(kvCoder), + makeDoFnInstruction(new ChangeKeysFn(), 0, kvCoder), + makeSinkInstruction(kvCoder, 1)); + + FakeWindmillServer server = new FakeWindmillServer(); + server.addWorkToOffer(makeInput(0, 0)); + server.addWorkToOffer(makeInput(1, TimeUnit.MILLISECONDS.toMicros(1))); + StreamingDataflowWorker worker = new StreamingDataflowWorker( + Arrays.asList(makeMapTask(instructions)), server, createTestingPipelineOptions()); + worker.start(); + + Map result = server.waitForAndGetCommits(2); + + Assert.assertEquals(makeExpectedOutput(0, 0, "key_data0"), stripCounters(result.get(0L))); + Assert.assertEquals(makeExpectedOutput(1, TimeUnit.MILLISECONDS.toMicros(1), "key_data1"), + stripCounters(result.get(1L))); + } + + static class TestStateFn extends DoFn, KV> + implements DoFn.RequiresKeyedState { + @Override + public void processElement(ProcessContext c) { + try { + CodedTupleTag stateTag = CodedTupleTag.of("state", StringUtf8Coder.of()); + CodedTupleTag emptyStateTag = + CodedTupleTag.of("other_state", StringUtf8Coder.of()); + CodedTupleTagMap result = + c.keyedState().lookup(Arrays.asList(stateTag, emptyStateTag)); + Assert.assertNull(result.get(emptyStateTag)); + String state = result.get(stateTag); + state += "-" + c.element().getValue(); + c.keyedState().store(CodedTupleTag.of("state", StringUtf8Coder.of()), state); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + @Test public void testState() throws Exception { + KvCoder kvCoder = KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); + + List instructions = Arrays.asList( + makeSourceInstruction(kvCoder), + makeDoFnInstruction(new TestStateFn(), 0, kvCoder), + makeSinkInstruction(kvCoder, 1)); + + FakeWindmillServer server = new FakeWindmillServer(); + StreamingDataflowWorker worker = new StreamingDataflowWorker( + Arrays.asList(makeMapTask(instructions)), server, createTestingPipelineOptions()); + worker.start(); + + server.addDataToOffer(buildData( + "data {" + + " computation_id: \"computation\"" + + " data {" + + " key: \"key0\"" + + " values {" + + " tag: \"5:parDostate\"" + + " value {" + + " timestamp: 0" + + " data: \"key0\"" + + " }" + + " }" + + " }" + + "}")); + server.addDataToOffer(buildData( + "data {" + + " computation_id: \"computation\"" + + " data {" + + " key: \"key0\"" + + " values {" + + " tag: \"5:Stagestate\"" + + " value {" + + " timestamp: 1" + + " data: \"key0\"" + + " }" + + " }" + + " }" + + "}")); + + server.addWorkToOffer(buildInput( + "work {" + + " computation_id: \"computation\"" + + " work {" + + " key: \"key0\"" + + " work_token: 0" + + " message_bundles {" + + " source_computation_id: \"upstream\"" + + " messages {" + + " timestamp: 0" + + " data: \"0\"" + + " }" + + " messages {" + + " timestamp: 1" + + " data: \"1\"" + + " }" + + " }" + + " }" + + "}", + CoderUtils.encodeToByteArray(CollectionCoder.of(IntervalWindow.getCoder()), + Arrays.asList(DEFAULT_WINDOW)))); + + server.waitForAndGetCommits(1); + + server.addDataToOffer(buildData( + "data {" + + " computation_id: \"computation\"" + + " data {" + + " key: \"key1\"" + + " values {" + + " tag: \"5:parDostate\"" + + " value {" + + " timestamp: 0" + + " data: \"key1\"" + + " }" + + " }" + + " }" + + "}")); + + server.addWorkToOffer(buildInput( + "work {" + + " computation_id: \"computation\"" + + " work {" + + " key: \"key1\"" + + " work_token: 1" + + " message_bundles {" + + " source_computation_id: \"upstream\"" + + " messages {" + + " timestamp: 2" + + " data: \"2\"" + + " }" + + " }" + + " }" + + "}", + CoderUtils.encodeToByteArray(CollectionCoder.of(IntervalWindow.getCoder()), + Arrays.asList(DEFAULT_WINDOW)))); + + Map result = server.waitForAndGetCommits(1); + + Assert.assertEquals(buildExpectedOutput( + "key: \"key0\" " + + "work_token: 0 " + + "value_updates {" + + " tag: \"5:parDostate\"" + + " value {" + + " timestamp: 9223372036854775807" + + " data: \"key0-0-1\"" + + " }" + + "} " + + "counter_updates {" + + " name: \"par_do_output-ElementCount\"" + + " kind: SUM" + + "} " + + "counter_updates {" + + " name: \"read_output-ElementCount\"" + + " kind: SUM" + + " int_scalar: 2" + + "} " + + "counter_updates {" + + " name: \"read_output-MeanByteCount\"" + + " kind: MEAN" + + " int_scalar: 70" + + " mean_count: 2" + + "} " + + "counter_updates {" + + " name: \"sink-ByteCount\"" + + " kind: SUM" + + "}" + + "counter_updates {" + + " name: \"source-ByteCount\"" + + " kind: SUM" + + " int_scalar: 10" + + "} "), + stripProcessingTimeCounters(result.get(0L))); + + Assert.assertEquals(buildExpectedOutput( + "key: \"key1\" " + + "work_token: 1 " + + "value_updates {" + + " tag: \"5:parDostate\"" + + " value {" + + " timestamp: 9223372036854775807" + + " data: \"key1-2\"" + + " }" + + "}" + + "counter_updates {" + + " name: \"par_do_output-ElementCount\"" + + " kind: SUM" + + "} " + + "counter_updates {" + + " name: \"read_output-ElementCount\"" + + " kind: SUM" + + " int_scalar: 1" + + "} " + + "counter_updates {" + + " name: \"read_output-MeanByteCount\"" + + " kind: MEAN" + + " int_scalar: 35" + + " mean_count: 1" + + "} " + + "counter_updates {" + + " name: \"sink-ByteCount\"" + + " kind: SUM" + + "} " + + "counter_updates {" + + " name: \"source-ByteCount\"" + + " kind: SUM" + + " int_scalar: 5" + + "} "), + stripProcessingTimeCounters(result.get(1L))); + } + + static class TestExceptionFn extends DoFn { + @Override + public void processElement(ProcessContext c) throws Exception { + try { + throw new Exception("Exception!"); + } catch (Exception e) { + throw new Exception("Another exception!", e); + } + } + } + + @Test public void testExceptions() throws Exception { + List instructions = Arrays.asList( + makeSourceInstruction(StringUtf8Coder.of()), + makeDoFnInstruction(new TestExceptionFn(), 0, StringUtf8Coder.of()), + makeSinkInstruction(StringUtf8Coder.of(), 1)); + + FakeWindmillServer server = new FakeWindmillServer(); + server.addWorkToOffer(buildInput( + "work {" + + " computation_id: \"computation\"" + + " work {" + + " key: \"key0\"" + + " work_token: 0" + + " message_bundles {" + + " source_computation_id: \"upstream\"" + + " messages {" + + " timestamp: 0" + + " data: \"0\"" + + " }" + + " }" + + " }" + + "}", + CoderUtils.encodeToByteArray(CollectionCoder.of(IntervalWindow.getCoder()), + Arrays.asList(DEFAULT_WINDOW)))); + + StreamingDataflowWorker worker = new StreamingDataflowWorker( + Arrays.asList(makeMapTask(instructions)), server, createTestingPipelineOptions()); + worker.start(); + + Windmill.Exception exception = server.getException(); + + Assert.assertThat(exception.getStackFrames(0), + JUnitMatchers.containsString("Another exception!")); + Assert.assertThat(exception.getStackFrames(1), + JUnitMatchers.containsString("processElement")); + Assert.assertTrue(exception.hasCause()); + + Assert.assertThat(exception.getCause().getStackFrames(0), + JUnitMatchers.containsString("Exception!")); + Assert.assertThat(exception.getCause().getStackFrames(1), + JUnitMatchers.containsString("processElement")); + Assert.assertFalse(exception.getCause().hasCause()); + } + + private static class TestTimerFn + extends AssignWindowsDoFn, BoundedWindow> { + public TestTimerFn() { + super(null); + } + @Override + public void processElement(ProcessContext c) { + c.output(KV.of("key0", Long.toString(c.timestamp().getMillis()))); + } + } + + @Test public void testTimers() throws Exception { + KvCoder kvCoder = KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); + + List instructions = Arrays.asList( + makeWindowingSourceInstruction(kvCoder), + makeDoFnInstruction(new TestTimerFn(), 0, kvCoder), + makeSinkInstruction(kvCoder, 1)); + + FakeWindmillServer server = new FakeWindmillServer(); + + server.addWorkToOffer(buildTimerInput( + "work {" + + " computation_id: \"computation\"" + + " work {" + + " key: \"key0\"" + + " work_token: 0" + + " timers {" + + " timers {" + + " tag: \"tag\"" + + " timestamp: 3000" + + " }" + + " }" + + " }" + + "}")); + + StreamingDataflowWorker worker = new StreamingDataflowWorker( + Arrays.asList(makeMapTask(instructions)), server, createTestingPipelineOptions()); + worker.start(); + + Map result = server.waitForAndGetCommits(1); + + Assert.assertEquals(buildExpectedOutput( + "key: \"key0\" " + + "work_token: 0 " + + "output_messages {" + + " destination_stream_id: \"out\"" + + " bundles {" + + " key: \"key0\"" + + " messages {" + + " timestamp: 3000" + + " data: \"3\"" + + " }" + + " }" + + "} ", + CoderUtils.encodeToByteArray(CollectionCoder.of(IntervalWindow.getCoder()), + new ArrayList())), + stripCounters(result.get(0L))); + } + + @Test public void testAssignWindows() throws Exception { + Duration gapDuration = Duration.standardSeconds(1); + CloudObject spec = CloudObject.forClassName("AssignWindowsDoFn"); + addString(spec, PropertyNames.SERIALIZED_FN, + StringUtils.byteArrayToJsonString( + SerializableUtils.serializeToByteArray(FixedWindows.of(gapDuration)))); + + ParallelInstruction addWindowsInstruction = + new ParallelInstruction() + .setSystemName("AssignWindows") + .setParDo(new ParDoInstruction() + .setInput(new InstructionInput().setProducerInstructionIndex(0).setOutputNum(0)) + .setNumOutputs(1) + .setUserFn(spec)) + .setOutputs(Arrays.asList(new InstructionOutput() + .setName("output") + .setCodec(WindowedValue.getFullCoder(StringUtf8Coder.of(), + IntervalWindow.getCoder()).asCloudObject()))); + + List instructions = Arrays.asList( + makeSourceInstruction(StringUtf8Coder.of()), + addWindowsInstruction, + makeSinkInstruction(StringUtf8Coder.of(), 1)); + + FakeWindmillServer server = new FakeWindmillServer(); + + server.addWorkToOffer(makeInput(0, 0)); + server.addWorkToOffer(makeInput(1000000, 1000000)); + + StreamingDataflowWorker worker = new StreamingDataflowWorker( + Arrays.asList(makeMapTask(instructions)), server, createTestingPipelineOptions()); + worker.start(); + + Map result = server.waitForAndGetCommits(2); + + Assert.assertEquals(buildExpectedOutput( + "key: \"key\" " + + "work_token: 0 " + + "output_messages {" + + " destination_stream_id: \"out\"" + + " bundles {" + + " key: \"key\"" + + " messages {" + + " timestamp: 0" + + " data: \"data0\"" + + " }" + + " }" + + "} ", + CoderUtils.encodeToByteArray( + CollectionCoder.of(IntervalWindow.getCoder()), + Arrays.asList(new IntervalWindow(new Instant(0), new Instant(1000))))), + stripCounters(result.get(0L))); + + Windmill.WorkItemCommitRequest.Builder expected = buildExpectedOutput( + "key: \"key\" " + + "work_token: 1000000 " + + "output_messages {" + + " destination_stream_id: \"out\"" + + " bundles {" + + " key: \"key\"" + + " messages {" + + " timestamp: 1000000" + + " data: \"data1000000\"" + + " }" + + " }" + + "} ", + CoderUtils.encodeToByteArray( + CollectionCoder.of(IntervalWindow.getCoder()), + Arrays.asList(new IntervalWindow(new Instant(1000), new Instant(2000))))).toBuilder(); + + Assert.assertEquals(expected.build(), stripCounters(result.get(1000000L))); + } + + @Test public void testMergeWindows() throws Exception { + Coder> kvCoder = KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); + Coder>> windowedKvCoder = + FullWindowedValueCoder.of(kvCoder, IntervalWindow.getCoder()); + KvCoder> groupedCoder = + KvCoder.of(StringUtf8Coder.of(), ListCoder.of(StringUtf8Coder.of())); + Coder>>> windowedGroupedCoder = + FullWindowedValueCoder.of(groupedCoder, IntervalWindow.getCoder()); + + CloudObject spec = CloudObject.forClassName("MergeWindowsDoFn"); + addString(spec, PropertyNames.SERIALIZED_FN, + StringUtils.byteArrayToJsonString( + SerializableUtils.serializeToByteArray(FixedWindows.of(Duration.standardSeconds(1))))); + addObject(spec, PropertyNames.INPUT_CODER, windowedKvCoder.asCloudObject()); + + ParallelInstruction mergeWindowsInstruction = + new ParallelInstruction() + .setSystemName("MergeWindows") + .setParDo(new ParDoInstruction() + .setInput(new InstructionInput().setProducerInstructionIndex(0).setOutputNum(0)) + .setNumOutputs(1) + .setUserFn(spec)) + .setOutputs(Arrays.asList(new InstructionOutput() + .setName("output") + .setCodec(windowedGroupedCoder.asCloudObject()))); + + List instructions = Arrays.asList( + makeWindowingSourceInstruction(kvCoder), + mergeWindowsInstruction, + makeSinkInstruction(groupedCoder, 1)); + + FakeWindmillServer server = new FakeWindmillServer(); + + StreamingDataflowWorker worker = new StreamingDataflowWorker( + Arrays.asList(makeMapTask(instructions)), server, createTestingPipelineOptions()); + worker.start(); + + server.addWorkToOffer(buildInput( + "work {" + + " computation_id: \"computation\"" + + " work {" + + " key: \"key\"" + + " work_token: 0" + + " message_bundles {" + + " source_computation_id: \"upstream\"" + + " messages {" + + " timestamp: 0" + + " data: \"data0\"" + + " }" + + " }" + + " }" + + "}", + CoderUtils.encodeToByteArray( + CollectionCoder.of(IntervalWindow.getCoder()), + Arrays.asList(new IntervalWindow(new Instant(0), new Instant(1000)))))); + + Map result = server.waitForAndGetCommits(1); + + Assert.assertEquals(buildExpectedOutput( + "key: \"key\" " + + "work_token: 0 " + + "output_timers {" + + " tag: \"gAAAAAAAAAA=\"" + + " timestamp: 999000" + + "} " + + "list_updates {" + + " tag: \"12:MergeWindowsbuffer:gAAAAAAAAAA=\"" + + " values {" + + " timestamp: 0" + + " data: \"data0\"" + + " }" + + "}"), + stripCounters(result.get(0L))); + + server.addWorkToOffer(buildTimerInput( + "work {" + + " computation_id: \"computation\"" + + " work {" + + " key: \"key\"" + + " work_token: 1" + + " timers {" + + " timers {" + + " tag: \"gAAAAAAAAAA=\"" + + " timestamp: 999000" + + " }" + + " }" + + " }" + + "}")); + server.addDataToOffer(buildData( + "data {" + + " computation_id: \"computation\"" + + " data {" + + " key: \"key\"" + + " lists {" + + " tag: \"12:MergeWindowsbuffer:gAAAAAAAAAA=\"" + + " values {" + + " timestamp: 0" + + " data: \"data0\"" + + " }" + + " }" + + " }" + + "}")); + + result = server.waitForAndGetCommits(1); + + Assert.assertEquals(buildExpectedOutput( + "key: \"key\" " + + "work_token: 1 " + + "output_messages {" + + " destination_stream_id: \"out\"" + + " bundles {" + + " key: \"key\"" + + " messages {" + + " timestamp: 999000" + + " data: \"\000\000\000\001\005data0\"" + + " }" + + " }" + + "} " + + "list_updates {" + + " tag: \"12:MergeWindowsbuffer:gAAAAAAAAAA=\"" + + " end_timestamp: 9223372036854775807" + + "}", + CoderUtils.encodeToByteArray( + CollectionCoder.of(IntervalWindow.getCoder()), + Arrays.asList(new IntervalWindow(new Instant(0), new Instant(1000))))), + stripCounters(result.get(1L))); + } +} From 22ce583ee784e045c987312baa861ed1ab9b5cad Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Thu, 5 Feb 2015 14:58:25 -0800 Subject: [PATCH 0123/1541] Update .travis.yml Try to enable testing with multiple JDKs on Linux platform using Travis, while also supporting multiple operating systems. This is a work-around of the following Travis issue: https://github.com/travis-ci/travis-ci/issues/2317. --- .travis.yml | 25 +++++++++++++++++++++---- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/.travis.yml b/.travis.yml index c860c7cfe55bb..ed72aff6253c4 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,9 +1,5 @@ language: java -os: - - linux - - osx - notifications: email: recipients: @@ -11,8 +7,29 @@ notifications: on_success: change on_failure: always +os: + - linux + - osx + +env: + matrix: + - CUSTOM_JDK="oraclejdk8" + - CUSTOM_JDK="oraclejdk7" + - CUSTOM_JDK="openjdk7" + +matrix: + exclude: + # On OSX, run with default JDK only. + - os: osx + env: CUSTOM_JDK="oraclejdk8" + - os: osx + env: CUSTOM_JDK="oraclejdk7" + - os: osx + env: CUSTOM_JDK="openjdk7" + before_install: - if [ "$TRAVIS_OS_NAME" == "osx" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi + - if [ "$TRAVIS_OS_NAME" == "linux" ]; then jdk_switcher use "CUSTOM_JDK"; fi install: - mvn install clean -U -DskipTests=true From 6618ea7f3a9eb2a4fd39623d158cc2bba6b3efda Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Thu, 5 Feb 2015 15:05:41 -0800 Subject: [PATCH 0124/1541] Update .travis.yml * Fix an issue in .travis.yml where CUSTOM_JDK environment variable is not prefixed with a dollar sign. * Fix an issue in .travis.yml where OSX doesn't have any JDK specified. --- .travis.yml | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index ed72aff6253c4..4815f28e12944 100644 --- a/.travis.yml +++ b/.travis.yml @@ -13,6 +13,7 @@ os: env: matrix: + - CUSTOM_JDK="default" - CUSTOM_JDK="oraclejdk8" - CUSTOM_JDK="oraclejdk7" - CUSTOM_JDK="openjdk7" @@ -26,10 +27,13 @@ matrix: env: CUSTOM_JDK="oraclejdk7" - os: osx env: CUSTOM_JDK="openjdk7" + # On Linux, run with specific JDKs only. + - os: linux + env: CUSTOM_JDK="default" before_install: - if [ "$TRAVIS_OS_NAME" == "osx" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi - - if [ "$TRAVIS_OS_NAME" == "linux" ]; then jdk_switcher use "CUSTOM_JDK"; fi + - if [ "$TRAVIS_OS_NAME" == "linux" ]; then jdk_switcher use "$CUSTOM_JDK"; fi install: - mvn install clean -U -DskipTests=true From 4e37c016b43eea27d100e161eaa4cc5cd621c6be Mon Sep 17 00:00:00 2001 From: jlewi Date: Wed, 4 Feb 2015 23:17:33 -0800 Subject: [PATCH 0125/1541] Fix a bug that was causing the BlockingDataflowRunner to exit before printing out some messages. We need to make sure that we don't drop messages that arrive in between when list job messages is called and when we get the status of the job. To fix this we change the order of operations. 1. Get the status of the job. 2. List all messages. 3. Check if the status of the job is done; importantly we use the result of GetJob from step 1. This ensures that we always call list messages after determining that the job is done. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85595332 --- .../sdk/runners/DataflowPipelineJob.java | 36 ++++++++++--------- 1 file changed, 20 insertions(+), 16 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java index c1facb0288b84..c1e18cd63eaee 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java @@ -118,9 +118,18 @@ public JobState waitToFinish( int errorGettingMessages = 0; int errorGettingJobStatus = 0; while (true) { - if (System.currentTimeMillis() >= endTime) { - // Timed out. - return null; + // Get the state of the job before listing messages. This ensures we always fetch job + // messages after the job finishes to ensure we have all them. + Job job = null; + try { + job = dataflowClient.v1b3().projects().jobs().get(project, jobId).execute(); + } catch (GoogleJsonResponseException | SocketTimeoutException e) { + if (++errorGettingJobStatus > 5) { + // We want to continue to wait for the job to finish so + // we ignore this error, but warn occasionally if it keeps happening. + LOG.warn("There were problems getting job status: ", e); + errorGettingJobStatus = 0; + } } if (messageHandler != null) { @@ -145,19 +154,14 @@ public JobState waitToFinish( } // Check if the job is done. - try { - Job job = dataflowClient.v1b3().projects().jobs().get(project, jobId).execute(); - JobState state = JobState.toState(job.getCurrentState()); - if (state.isTerminal()) { - return state; - } - } catch (GoogleJsonResponseException | SocketTimeoutException e) { - if (++errorGettingJobStatus > 5) { - // We want to continue to wait for the job to finish so - // we ignore this error, but warn occasionally if it keeps happening. - LOG.warn("There were problems getting job status: ", e); - errorGettingJobStatus = 0; - } + JobState state = JobState.toState(job.getCurrentState()); + if (state.isTerminal()) { + return state; + } + + if (System.currentTimeMillis() >= endTime) { + // Timed out. + return null; } // Job not yet done. Wait a little, then check again. From adca9145fbb69cdfcf0dc62de34a8b7fc6eae9a3 Mon Sep 17 00:00:00 2001 From: malo Date: Thu, 5 Feb 2015 10:20:31 -0800 Subject: [PATCH 0126/1541] Fix NullPointerException that happens when an exception is raised while looking for the job. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85633961 --- .../cloud/dataflow/sdk/runners/DataflowPipelineJob.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java index c1e18cd63eaee..a2fe55255fe3d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineJob.java @@ -154,9 +154,11 @@ public JobState waitToFinish( } // Check if the job is done. - JobState state = JobState.toState(job.getCurrentState()); - if (state.isTerminal()) { - return state; + if (job != null) { + JobState state = JobState.toState(job.getCurrentState()); + if (state.isTerminal()) { + return state; + } } if (System.currentTimeMillis() >= endTime) { From 0193099ac550ec8963a61357df5a7d2ee214ee84 Mon Sep 17 00:00:00 2001 From: samuelw Date: Thu, 5 Feb 2015 13:18:57 -0800 Subject: [PATCH 0127/1541] Add failed response to ReportStats. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85651192 --- sdk/src/main/proto/windmill.proto | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/proto/windmill.proto b/sdk/src/main/proto/windmill.proto index 88aa796417143..2135ce53be263 100644 --- a/sdk/src/main/proto/windmill.proto +++ b/sdk/src/main/proto/windmill.proto @@ -211,4 +211,6 @@ message ReportStatsRequest { repeated Exception exceptions = 4; } -message ReportStatsResponse {} +message ReportStatsResponse { + optional bool failed = 1; +} From 3872228bde72de38b3b891e0d595cee59a15c2d2 Mon Sep 17 00:00:00 2001 From: kirpichov Date: Thu, 5 Feb 2015 14:41:37 -0800 Subject: [PATCH 0128/1541] Logging which stop position was unreported, when tearing down a task which has an unreported stop position. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85659485 --- .../dataflow/sdk/util/common/worker/WorkProgressUpdater.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java index b50afb4b5f886..4794fe3efb2ec 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java @@ -126,7 +126,8 @@ public void stopReportingProgress() throws Exception { // We send a final progress report in case there was an unreported stop position update. if (stopPositionToService != null) { - LOG.info("Sending final progress update with unreported stop position."); + LOG.info("Sending final progress update with unreported stop position: {} " + + "for work item: {}", stopPositionToService, workString()); reportProgressHelper(); // This call can fail with an exception } From e5c3a0d33b4e291d0ce8ae1b7530c7996af59b36 Mon Sep 17 00:00:00 2001 From: davor Date: Thu, 5 Feb 2015 16:06:11 -0800 Subject: [PATCH 0129/1541] Dataflow JavaDoc: fix typo (vary --> very). [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85667847 --- .../java/com/google/cloud/dataflow/sdk/transforms/ParDo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java index f61b197bc58a3..dee5096f83613 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java @@ -425,7 +425,7 @@ *

    The Google Cloud Dataflow service applies fusion as much as * possible, greatly reducing the cost of executing pipelines. As a * result, it is essentially "free" to write ParDo operations in a - * vary modular, composable style, each ParDo operation doing one + * very modular, composable style, each ParDo operation doing one * clear task, and stringing together sequences of ParDo operations to * get the desired overall effect. Such programs can be easier to * understand, easier to unit-test, easier to extend and evolve, and From 6e8cfbc96405030ad65ee9a8aaac8e8a886c9b9f Mon Sep 17 00:00:00 2001 From: andersjohnson Date: Fri, 6 Feb 2015 08:46:17 -0800 Subject: [PATCH 0130/1541] Allow actual output line ordering to vary. Also, add more diagnostics in case of an output mismatch. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85720217 --- test_wordcount.sh | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/test_wordcount.sh b/test_wordcount.sh index 699489c89b11c..45f5a6bd93625 100755 --- a/test_wordcount.sh +++ b/test_wordcount.sh @@ -34,12 +34,14 @@ function check_result_hash { local outfile_prefix=$2 local expected=$3 - local actual=$(md5sum $outfile_prefix-* | awk '{print $1}' || \ - md5 -q $outfile_prefix-*) || exit 2 # OSX + local actual=$(sort $outfile_prefix-* | md5sum | awk '{print $1}' || \ + sort $outfile_prefix-* | md5 -q) || exit 2 # OSX if [[ "$actual" != "$expected" ]] then echo "FAIL $name: Output hash mismatch. Got $actual, expected $expected." PASS="" + echo "head hexdump of actual:" + head $outfile_prefix-* | hexdump -c else echo "pass $name" # Output files are left behind in /tmp @@ -107,12 +109,12 @@ echo "Generating bundled JAR file" >&2 mvn bundle:bundle -f $TOPDIR/pom.xml -pl examples check_for_jar_file -run_all_ways wordcount1 "LICENSE" f4af56cd6f6f127536d586a6adcefba1 -run_all_ways wordcount2 "./LICENSE" f4af56cd6f6f127536d586a6adcefba1 -run_all_ways wordcount3 "$PWD/LICENSE" f4af56cd6f6f127536d586a6adcefba1 -run_all_ways wordcount4 "L*N?E*" f4af56cd6f6f127536d586a6adcefba1 -run_all_ways wordcount5 "./LICE*N?E" f4af56cd6f6f127536d586a6adcefba1 -run_all_ways wordcount6 "$PWD/*LIC?NSE" f4af56cd6f6f127536d586a6adcefba1 +run_all_ways wordcount1 "LICENSE" 9e164209cfe94043e3d64e6cc1c11a0c +run_all_ways wordcount2 "./LICENSE" 9e164209cfe94043e3d64e6cc1c11a0c +run_all_ways wordcount3 "$PWD/LICENSE" 9e164209cfe94043e3d64e6cc1c11a0c +run_all_ways wordcount4 "L*N?E*" 9e164209cfe94043e3d64e6cc1c11a0c +run_all_ways wordcount5 "./LICE*N?E" 9e164209cfe94043e3d64e6cc1c11a0c +run_all_ways wordcount6 "$PWD/*LIC?NSE" 9e164209cfe94043e3d64e6cc1c11a0c if [[ ! "$PASS" ]] then From 1d32572577e4dc290191e3c98a6c968f02ea4271 Mon Sep 17 00:00:00 2001 From: millsd Date: Fri, 6 Feb 2015 10:49:37 -0800 Subject: [PATCH 0131/1541] Adds streaming Dataflow examples. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85730302 --- .../dataflow/examples/PubsubFileInjector.java | 105 ++++++++++++ .../examples/StreamingWordExtract.java | 141 +++++++++++++++ .../dataflow/examples/WindowingWordCount.java | 162 ++++++++++++++++++ 3 files changed, 408 insertions(+) create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/PubsubFileInjector.java create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/StreamingWordExtract.java create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/WindowingWordCount.java diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/PubsubFileInjector.java b/examples/src/main/java/com/google/cloud/dataflow/examples/PubsubFileInjector.java new file mode 100644 index 0000000000000..ac9c15e809949 --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/PubsubFileInjector.java @@ -0,0 +1,105 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.pubsub.model.PublishRequest; +import com.google.api.services.pubsub.model.PubsubMessage; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.StreamingOptions; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.RateLimiting; +import com.google.cloud.dataflow.sdk.util.Transport; + +import java.io.IOException; + +/** + * A batch Dataflow pipeline for injecting a set of GCS files into + * a PubSub topic line by line. + * + *

    This is useful for testing streaming + * pipelines. Note that since batch pipelines might retry chunks, this + * does _not_ guarantee exactly-once injection of file data. Some lines may + * be published multiple times. + *

    + */ +public class PubsubFileInjector { + + /** A DoFn that publishes lines to Google Cloud PubSub. */ + static class Publish extends DoFn { + private String outputTopic; + public transient Pubsub pubsub; + + Publish(String outputTopic) { + this.outputTopic = outputTopic; + } + + @Override + public void startBundle(Context context) { + StreamingOptions options = + context.getPipelineOptions().as(StreamingOptions.class); + this.pubsub = Transport.newPubsubClient(options).build(); + } + + @Override + public void processElement(ProcessContext c) throws IOException { + PubsubMessage pubsubMessage = new PubsubMessage(); + pubsubMessage.encodeData(c.element().getBytes()); + PublishRequest publishRequest = new PublishRequest(); + publishRequest.setTopic(outputTopic).setMessage(pubsubMessage); + this.pubsub.topics().publish(publishRequest).execute(); + } + } + + /** + * Command line parameter options. + */ + private interface PubsubFileInjectorOptions extends PipelineOptions { + @Description("GCS location of files.") + @Validation.Required + String getInput(); + void setInput(String value); + + @Description("Topic to publish on.") + @Validation.Required + String getOutputTopic(); + void setOutputTopic(String value); + } + + /** + * Sets up and starts streaming pipeline. + */ + public static void main(String[] args) { + PubsubFileInjectorOptions options = PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(PubsubFileInjectorOptions.class); + + Pipeline pipeline = Pipeline.create(options); + + pipeline + .apply(TextIO.Read.from(options.getInput())) + .apply(RateLimiting.perWorker(new Publish(options.getOutputTopic())) + .withMaxParallelism(20)); + + pipeline.run(); + } +} diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/StreamingWordExtract.java b/examples/src/main/java/com/google/cloud/dataflow/examples/StreamingWordExtract.java new file mode 100644 index 0000000000000..2f8dc079991df --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/StreamingWordExtract.java @@ -0,0 +1,141 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.PubsubIO; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; + +import java.util.ArrayList; + +/** + * A streaming Dataflow Example using BigQuery output. + * + *

    This pipeline example lines of text from a PubSub topic, splits each line + * into individual words, capitalizes those words, and writes the output to + * a BigQuery table.

    + * + *

    To run this example using the Dataflow service, you must provide an input + * pubsub topic and an output BigQuery table, using the {@literal --inputTopic} + * {@literal --dataset} and {@literal --table} options. Since this is a streaming + * pipeline that never completes, select the non-blocking pipeline runner + * {@literal --runner=DataflowPipelineRunner}. + */ +public class StreamingWordExtract { + + /** A DoFn that tokenizes lines of text into individual words. */ + static class ExtractWords extends DoFn { + @Override + public void processElement(ProcessContext c) { + String[] words = c.element().split("[^a-zA-Z']+"); + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + /** A DoFn that uppercases a word. */ + static class Uppercase extends DoFn { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().toUpperCase()); + } + } + + /** + * Converts strings into BigQuery rows. + */ + static class StringToRowConverter extends DoFn { + + /** + * In this example, put the whole string into single BigQuery field. + */ + @Override + public void processElement(ProcessContext c) { + c.output(new TableRow().set("string_field", c.element())); + } + + static TableSchema getSchema() { + return new TableSchema().setFields(new ArrayList() { + // Compose the list of TableFieldSchema from tableSchema. + { + add(new TableFieldSchema().setName("string_field").setType("STRING")); + } + }); + } + } + + /** + * Command line parameter options. + */ + private interface StreamingWordExtractOptions extends PipelineOptions { + @Description("Input Pubsub topic") + @Validation.Required + String getInputTopic(); + void setInputTopic(String value); + + @Description("BigQuery dataset name") + @Validation.Required + String getDataset(); + void setDataset(String value); + + @Description("BigQuery table name") + @Validation.Required + String getTable(); + void setTable(String value); + } + + /** + * Sets up and starts streaming pipeline. + */ + public static void main(String[] args) { + StreamingWordExtractOptions options = PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(StreamingWordExtractOptions.class); + DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); + dataflowOptions.setStreaming(true); + + Pipeline pipeline = Pipeline.create(options); + + String tableSpec = new StringBuilder() + .append(dataflowOptions.getProject()).append(":") + .append(options.getDataset()).append(".") + .append(options.getTable()) + .toString(); + pipeline + .apply(PubsubIO.Read.topic(options.getInputTopic())) + .apply(ParDo.of(new ExtractWords())) + .apply(ParDo.of(new Uppercase())) + .apply(ParDo.of(new StringToRowConverter())) + .apply(BigQueryIO.Write.to(tableSpec) + .withSchema(StringToRowConverter.getSchema())); + + pipeline.run(); + } +} diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/WindowingWordCount.java b/examples/src/main/java/com/google/cloud/dataflow/examples/WindowingWordCount.java new file mode 100644 index 0000000000000..a8e9dde7e7092 --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/WindowingWordCount.java @@ -0,0 +1,162 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** + * An example that counts words in Shakespeare. For a detailed walkthrough of this + * example see: + * https://cloud.google.com/dataflow/java-sdk/wordcount-example + * + * To execute this pipeline locally, specify general pipeline configuration: + * --project= + * and example configuration: + * --output=[ | gs://] + * + * To execute this pipeline using the Dataflow service, specify pipeline configuration: + * --project= --stagingLocation=gs:// + * --runner=BlockingDataflowPipelineRunner + * and example configuration: + * --output=gs:// + * + * The input file defaults to gs://dataflow-samples/shakespeare/kinglear.txt and can be + * overridden with --input. + */ +public class WindowingWordCount { + + /** A DoFn that tokenizes lines of text into individual words with timestamp. */ + static class ExtractWordsWithTimestampFn extends DoFn { + @Override + public void processElement(ProcessContext c) { + String[] words = c.element().split("[^a-zA-Z']+"); + for (String word : words) { + if (!word.isEmpty()) { + c.outputWithTimestamp(word, new Instant(System.currentTimeMillis())); + } + } + } + } + + /** A DoFn that converts a Word and Count into a printable string. */ + static class FormatCountsFn extends DoFn, String> { + @Override + public void processElement(ProcessContext c) { + String output = "Element: " + c.element().getKey() + + " Value: " + c.element().getValue() + + " Timestamp: " + c.timestamp() + + " Windows: (" + c.windows() + ")"; + c.output(output); + } + } + + /** + * A PTransform that converts a PCollection containing lines of text into a PCollection of + * formatted word counts. + *

    + * Although this pipeline fragment could be inlined, bundling it as a PTransform allows for easy + * reuse, modular testing, and an improved monitoring experience. + */ + public static class CountWords extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection lines) { + + // Convert lines of text into individual words. + PCollection words = lines.apply( + ParDo.of(new ExtractWordsWithTimestampFn())); + + PCollection windowedWords = words.apply( + Window.into(FixedWindows.of(Duration.millis(1)))); + + // Count the number of times each word occurs. + PCollection> wordCounts = + windowedWords.apply(Count.perElement()); + + // Format each word and count into a printable string. + PCollection results = wordCounts.apply( + ParDo.of(new FormatCountsFn())); + + return results; + } + } + + private interface Options extends PipelineOptions { + @Description("Path of the file to read from") + @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") + String getInput(); + void setInput(String value); + + @Description("Path of the file to write to") + String getOutput(); + void setOutput(String value); + + /** + * By default (numShards == 0), the system will choose the shard count. + * Most programs will not need this option. + */ + @Description("Number of output shards (0 if the system should choose automatically)") + @Default.Integer(0) + int getNumShards(); + void setNumShard(int value); + } + + private static String getOutputLocation(Options options) { + DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); + if (options.getOutput() != null) { + return options.getOutput(); + } else if (dataflowOptions.getStagingLocation() != null) { + return GcsPath.fromUri(dataflowOptions.getStagingLocation()) + .resolve("counts.txt").toString(); + } else { + throw new IllegalArgumentException("Must specify --output or --stagingLocation"); + } + } + + + + public static void main(String[] args) { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + + Pipeline p = Pipeline.create(options); + + p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) + .apply(new CountWords()) + .apply(TextIO.Write.named("WriteCounts") + .to(getOutputLocation(options)) + .withNumShards(options.getNumShards())); + + p.run(); + } +} From 424c56df97be1b4b76cde18386a95b8a30a793f4 Mon Sep 17 00:00:00 2001 From: peihe Date: Fri, 6 Feb 2015 12:42:59 -0800 Subject: [PATCH 0132/1541] Refactor: 1. skipping deleteTimer() is an optimization for Dataflow Streaming Runner. Move it from PartitionBufferingWindowSet to StreamingActiveWindowManager. 2. remove the extra poll() from GroupAlsoByWindowsDoFn (windows are removed by WindowSets). [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85740534 --- .../sdk/util/GroupAlsoByWindowsDoFn.java | 4 ++-- .../sdk/util/PartitionBufferingWindowSet.java | 3 ++- .../util/StreamingGroupAlsoByWindowsDoFn.java | 22 +++++++++++-------- 3 files changed, 17 insertions(+), 12 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java index 26297d0eba974..84fa834824ab8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java @@ -351,10 +351,10 @@ public Instant nextTimestamp() { } /** - * Returns and removes the next window. + * Returns the next window. */ public W getWindow() { - return windows.poll(); + return windows.peek(); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java index 3c20b5f1f3cbc..885392cc27605 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java @@ -32,7 +32,7 @@ * WindowSet itself is never exposed to user code, allowing * a much simpler (and cheaper) implementation. * - * This WindowSet only works with {@link StreamingGroupAlsoByWindowsDoFn}. + *

    This WindowSet only works with {@link StreamingGroupAlsoByWindowsDoFn}. */ class PartitionBufferingWindowSet extends AbstractWindowSet, W> { @@ -58,6 +58,7 @@ public void put(W window, V value) throws Exception { public void remove(W window) throws Exception { context.context.stepContext.deleteTagList( bufferTag(window, windowFn.windowCoder(), inputCoder)); + activeWindowManager.removeWindow(window); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java index 9868b33a2db1c..a0aebe7538704 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java @@ -76,7 +76,7 @@ public void processElement(ProcessContext processContext) throws Exception { K key = element.getKey(); VI value = element.getValue(); AbstractWindowSet windowSet = createWindowSet( - key, context, new StreamingActiveWindowManager<>(context, windowFn.windowCoder())); + key, context, new StreamingActiveWindowManager<>(windowFn, context)); for (BoundedWindow window : context.windows()) { windowSet.put((W) window, value); @@ -86,8 +86,7 @@ public void processElement(ProcessContext processContext) throws Exception { } else { TimerOrElement timer = context.element(); AbstractWindowSet windowSet = createWindowSet( - (K) timer.key(), context, new StreamingActiveWindowManager<>( - context, windowFn.windowCoder())); + (K) timer.key(), context, new StreamingActiveWindowManager<>(windowFn, context)); // Attempt to merge windows before emitting; that may remove the current window under // consideration. @@ -106,26 +105,31 @@ public void processElement(ProcessContext processContext) throws Exception { private static class StreamingActiveWindowManager implements AbstractWindowSet.ActiveWindowManager { + WindowFn windowFn; DoFnProcessContext context; - Coder coder; StreamingActiveWindowManager( - DoFnProcessContext context, - Coder coder) { + WindowFn windowFn, + DoFnProcessContext context) { + this.windowFn = windowFn; this.context = context; - this.coder = coder; } @Override public void addWindow(W window) throws IOException { context.context.stepContext.getExecutionContext().setTimer( - WindowUtils.windowToString(window, coder), window.maxTimestamp()); + WindowUtils.windowToString(window, windowFn.windowCoder()), window.maxTimestamp()); } @Override public void removeWindow(W window) throws IOException { + if (windowFn instanceof PartitioningWindowFn) { + // For PartitioningWindowFn, each window triggers exactly one timer. + // And, timers are automatically deleted once they are fired. + return; + } context.context.stepContext.getExecutionContext().deleteTimer( - WindowUtils.windowToString(window, coder)); + WindowUtils.windowToString(window, windowFn.windowCoder())); } } } From 1c34edf27fa5a23146838bbcc83d4fb10c8b083f Mon Sep 17 00:00:00 2001 From: davor Date: Fri, 6 Feb 2015 14:12:42 -0800 Subject: [PATCH 0133/1541] Dataflow SDK: prepare a test protobuf for testing future Proto2Coder implementation. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85748245 --- sdk/src/main/proto/README.md | 27 ++++++++++++ .../proto/proto2_coder_test_messages.proto | 43 +++++++++++++++++++ 2 files changed, 70 insertions(+) create mode 100644 sdk/src/main/proto/README.md create mode 100644 sdk/src/main/proto/proto2_coder_test_messages.proto diff --git a/sdk/src/main/proto/README.md b/sdk/src/main/proto/README.md new file mode 100644 index 0000000000000..ea299389a63b5 --- /dev/null +++ b/sdk/src/main/proto/README.md @@ -0,0 +1,27 @@ +## Protocol Buffers in Google Cloud Dataflow + +This directory contains the Protocol Buffer messages used in Google Cloud +Dataflow. + +They aren't, however, used during the Maven build process, and are included here +for completeness only. Instead, the following artifact on Maven Central contains +the binary version of the generated code from these Protocol Buffers: + + + com.google.cloud.dataflow + google-cloud-dataflow-java-proto-library-all + LATEST + + +Please follow this process for testing changes: + +* Make changes to the Protocol Buffer messages in this directory. +* Use `protoc` to create a new Java library of the compiled generated code. +* Install that Java binary into your local Maven repository. +* Update SDK's `pom.xml` to pick up the newly installed library, instead of +downloading it from Maven Central. + +Once the changes are ready for submission, please separate them into two +commits. The first commit should update the Protocol Buffer messages only. After +that, we need to update the generated artifact on Maven Central. Finally, +changes that make use of the Protocol Buffer changes may be committed. diff --git a/sdk/src/main/proto/proto2_coder_test_messages.proto b/sdk/src/main/proto/proto2_coder_test_messages.proto new file mode 100644 index 0000000000000..56efd89c36ad5 --- /dev/null +++ b/sdk/src/main/proto/proto2_coder_test_messages.proto @@ -0,0 +1,43 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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. + */ + +/* + * Protocol Buffer messages used for testing Proto2Coder implementation. + */ + +syntax = "proto2"; + +package proto2_coder_test_messages; + +option java_package = "com.google.cloud.dataflow.sdk.coders"; + +message MessageA { + optional string field1 = 1; + repeated MessageB field2 = 2; +} + +message MessageB { + optional bool field1 = 1; +} + +message MessageC { + extensions 100 to 105; +} + +extend MessageC { + optional MessageA field1 = 101; + optional MessageB field2 = 102; +} From 6edf5729e7fdf0ab693e79d2e89e9012056ed535 Mon Sep 17 00:00:00 2001 From: davor Date: Fri, 6 Feb 2015 15:12:25 -0800 Subject: [PATCH 0134/1541] Dataflow SDK: update SDK's pom.xml to pick up a newer version of our proto-library artifact. The newer version includes a windmill protobuf fix and test protobufs for Proto2Coder. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85753678 --- sdk/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/pom.xml b/sdk/pom.xml index f269e2bc70f6d..b3537af67fd96 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -203,7 +203,7 @@ com.google.cloud.dataflow google-cloud-dataflow-java-proto-library-all - 0.3.150203 + 0.3.150206 From fa8b3e06d142535e0384393427d6276134888f9b Mon Sep 17 00:00:00 2001 From: samuelw Date: Fri, 6 Feb 2015 15:23:41 -0800 Subject: [PATCH 0135/1541] Fix infinite retries of work on lease expiration in streaming mode. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85754733 --- .../worker/StreamingDataflowWorker.java | 37 +++++++++++-------- 1 file changed, 22 insertions(+), 15 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java index 9234d31c7e7fa..07d58b89975be 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -381,14 +381,19 @@ private void process( t.printStackTrace(); lastException.set(t); LOG.fine("Failed work: " + work); - reportFailure(computation, work, t); - // Try again, but go to the end of the queue to avoid a tight loop. - sleep(60000); - executor.forceExecute(new Runnable() { - public void run() { - process(computation, work); - } - }); + if (reportFailure(computation, work, t)) { + // Try again, after some delay and at the end of the queue to avoid a tight loop. + sleep(60000); + executor.forceExecute(new Runnable() { + public void run() { + process(computation, work); + } + }); + } else { + // If we failed to report the error, the item is invalid and should + // not be retried internally. It will be retried at the higher level. + LOG.fine("Aborting processing for work token: " + work.getWorkToken()); + } } return; } @@ -529,13 +534,15 @@ private Windmill.Exception buildExceptionReport(Throwable t) { return builder.build(); } - private void reportFailure(String computation, Windmill.WorkItem work, Throwable t) { - windmillServer.reportStats(Windmill.ReportStatsRequest.newBuilder() - .setComputationId(computation) - .setKey(work.getKey()) - .setWorkToken(work.getWorkToken()) - .addExceptions(buildExceptionReport(t)) - .build()); + private boolean reportFailure(String computation, Windmill.WorkItem work, Throwable t) { + Windmill.ReportStatsResponse response = + windmillServer.reportStats(Windmill.ReportStatsRequest.newBuilder() + .setComputationId(computation) + .setKey(work.getKey()) + .setWorkToken(work.getWorkToken()) + .addExceptions(buildExceptionReport(t)) + .build()); + return response.getFailed(); } private static class WorkerAndContext { From 946bdd094770409ca0a5cd4d50571a4e37a05e10 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Fri, 6 Feb 2015 15:45:47 -0800 Subject: [PATCH 0136/1541] Update README.md --- sdk/src/main/proto/README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/proto/README.md b/sdk/src/main/proto/README.md index ea299389a63b5..fa4e925c982c6 100644 --- a/sdk/src/main/proto/README.md +++ b/sdk/src/main/proto/README.md @@ -16,8 +16,8 @@ the binary version of the generated code from these Protocol Buffers: Please follow this process for testing changes: * Make changes to the Protocol Buffer messages in this directory. -* Use `protoc` to create a new Java library of the compiled generated code. -* Install that Java binary into your local Maven repository. +* Use `protoc` to generate the new code, and compile it into a new Java library. +* Install that Java library into your local Maven repository. * Update SDK's `pom.xml` to pick up the newly installed library, instead of downloading it from Maven Central. From cc96bbafed2cba8c4eb7e30097a5c8ee8d25d29f Mon Sep 17 00:00:00 2001 From: andersjohnson Date: Fri, 6 Feb 2015 17:09:39 -0800 Subject: [PATCH 0137/1541] Set LC_ALL=C for sorting to make it consistent across platforms. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85763192 --- test_wordcount.sh | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/test_wordcount.sh b/test_wordcount.sh index 45f5a6bd93625..f852d2f6f77b5 100755 --- a/test_wordcount.sh +++ b/test_wordcount.sh @@ -34,8 +34,8 @@ function check_result_hash { local outfile_prefix=$2 local expected=$3 - local actual=$(sort $outfile_prefix-* | md5sum | awk '{print $1}' || \ - sort $outfile_prefix-* | md5 -q) || exit 2 # OSX + local actual=$(LC_ALL=C sort $outfile_prefix-* | md5sum | awk '{print $1}' \ + || LC_ALL=C sort $outfile_prefix-* | md5 -q) || exit 2 # OSX if [[ "$actual" != "$expected" ]] then echo "FAIL $name: Output hash mismatch. Got $actual, expected $expected." @@ -109,12 +109,12 @@ echo "Generating bundled JAR file" >&2 mvn bundle:bundle -f $TOPDIR/pom.xml -pl examples check_for_jar_file -run_all_ways wordcount1 "LICENSE" 9e164209cfe94043e3d64e6cc1c11a0c -run_all_ways wordcount2 "./LICENSE" 9e164209cfe94043e3d64e6cc1c11a0c -run_all_ways wordcount3 "$PWD/LICENSE" 9e164209cfe94043e3d64e6cc1c11a0c -run_all_ways wordcount4 "L*N?E*" 9e164209cfe94043e3d64e6cc1c11a0c -run_all_ways wordcount5 "./LICE*N?E" 9e164209cfe94043e3d64e6cc1c11a0c -run_all_ways wordcount6 "$PWD/*LIC?NSE" 9e164209cfe94043e3d64e6cc1c11a0c +run_all_ways wordcount1 "LICENSE" c5350a5ad4bb51e3e018612b4b044097 +run_all_ways wordcount2 "./LICENSE" c5350a5ad4bb51e3e018612b4b044097 +run_all_ways wordcount3 "$PWD/LICENSE" c5350a5ad4bb51e3e018612b4b044097 +run_all_ways wordcount4 "L*N?E*" c5350a5ad4bb51e3e018612b4b044097 +run_all_ways wordcount5 "./LICE*N?E" c5350a5ad4bb51e3e018612b4b044097 +run_all_ways wordcount6 "$PWD/*LIC?NSE" c5350a5ad4bb51e3e018612b4b044097 if [[ ! "$PASS" ]] then From b6c742e5b537eeca3646bde02a73db78d35ef495 Mon Sep 17 00:00:00 2001 From: relax Date: Sat, 7 Feb 2015 18:40:04 -0800 Subject: [PATCH 0138/1541] Rollback: Fix infinite retries of work on lease expiration in streaming mode. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85810964 --- .../worker/StreamingDataflowWorker.java | 37 ++++++++----------- 1 file changed, 15 insertions(+), 22 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java index 07d58b89975be..9234d31c7e7fa 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -381,19 +381,14 @@ private void process( t.printStackTrace(); lastException.set(t); LOG.fine("Failed work: " + work); - if (reportFailure(computation, work, t)) { - // Try again, after some delay and at the end of the queue to avoid a tight loop. - sleep(60000); - executor.forceExecute(new Runnable() { - public void run() { - process(computation, work); - } - }); - } else { - // If we failed to report the error, the item is invalid and should - // not be retried internally. It will be retried at the higher level. - LOG.fine("Aborting processing for work token: " + work.getWorkToken()); - } + reportFailure(computation, work, t); + // Try again, but go to the end of the queue to avoid a tight loop. + sleep(60000); + executor.forceExecute(new Runnable() { + public void run() { + process(computation, work); + } + }); } return; } @@ -534,15 +529,13 @@ private Windmill.Exception buildExceptionReport(Throwable t) { return builder.build(); } - private boolean reportFailure(String computation, Windmill.WorkItem work, Throwable t) { - Windmill.ReportStatsResponse response = - windmillServer.reportStats(Windmill.ReportStatsRequest.newBuilder() - .setComputationId(computation) - .setKey(work.getKey()) - .setWorkToken(work.getWorkToken()) - .addExceptions(buildExceptionReport(t)) - .build()); - return response.getFailed(); + private void reportFailure(String computation, Windmill.WorkItem work, Throwable t) { + windmillServer.reportStats(Windmill.ReportStatsRequest.newBuilder() + .setComputationId(computation) + .setKey(work.getKey()) + .setWorkToken(work.getWorkToken()) + .addExceptions(buildExceptionReport(t)) + .build()); } private static class WorkerAndContext { From 911a0859be6cb359ebe181f565468b1c8b209834 Mon Sep 17 00:00:00 2001 From: malo Date: Sat, 7 Feb 2015 20:53:24 -0800 Subject: [PATCH 0139/1541] Add messages about reasons why stop positions are not accepted for FileBasedReader and InMemoryReader. Stop proposeStopPosition from looking into the iterator if it is in Finish state. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85815080 --- .../sdk/runners/worker/FileBasedReader.java | 8 +++- .../sdk/runners/worker/InMemoryReader.java | 6 ++- .../sdk/util/common/worker/Operation.java | 38 +++++++++++++------ .../sdk/util/common/worker/ReadOperation.java | 16 +++++--- .../util/common/worker/ReadOperationTest.java | 6 ++- 5 files changed, 54 insertions(+), 20 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java index c0c16d88f26bf..53ebbd52ad82f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java @@ -207,22 +207,26 @@ public Position updateStopPosition(Progress proposedStopPosition) { Long byteOffset = stopPosition.getPosition().getByteOffset(); if (byteOffset == null) { - LOG.warn("A stop position other than byte offset is not supported in a " - + "file-based Source."); + LOG.warn("A proposed stop position must be a byte offset for a file-based Source."); return null; } if (byteOffset <= offset) { // Proposed stop position is not after the current position: // No stop position update. + LOG.warn("The proposed stop position " + byteOffset + + " is past the current position " + offset); return null; } if (endOffset != null && byteOffset >= endOffset) { // Proposed stop position is after the current stop (end) position: No // stop position update. + LOG.warn("The proposed stop position " + byteOffset + + " is after the current stop position " + endOffset); return null; } + LOG.info("Updated the stop position to offset " + byteOffset); this.endOffset = byteOffset; return cloudPositionToReaderPosition(stopPosition.getPosition()); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java index 487daa1affe45..72107b979d841 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java @@ -140,16 +140,20 @@ public Position updateStopPosition(Progress proposedStopPosition) { Long recordIndex = stopPosition.getRecordIndex(); if (recordIndex == null) { - LOG.warn("A stop position other than record index is not supported in InMemoryReader."); + LOG.warn("A proposed stop position must be a record index for InMemoryReader."); return null; } if (recordIndex <= index || recordIndex >= endPosition) { // Proposed stop position is not after the current position or proposed // stop position is after the current stop (end) position: No stop // position update. + LOG.warn("The proposed stop position " + recordIndex + + " is not between the current stop position " + index + + " and the current stop position " + endPosition); return null; } + LOG.info("Updated the stop position to record " + recordIndex); this.endPosition = recordIndex.intValue(); return cloudPositionToReaderPosition(stopPosition); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Operation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Operation.java index 207d034d4f3d6..31312b7238fc6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Operation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Operation.java @@ -21,14 +21,14 @@ /** * The abstract base class for Operations, which correspond to * Instructions in the original MapTask InstructionGraph. - * + *

    * Call start() to start the operation. - * + *

    * A read operation's start() method actually reads the data, and in * effect runs the pipeline. - * + *

    * Call finish() to finish the operation. - * + *

    * Since both start() and finish() may call process() on * this operation's consumers, start an operation after * starting its consumers, and finish an operation before @@ -61,6 +61,11 @@ public enum InitializationState { public InitializationState initializationState = InitializationState.UNSTARTED; + /** The lock protecting the initialization state. InitializationState is only + * written from one thread, but can be read by concurrent threads. + */ + protected final Object initializationStateLock = new Object(); + protected final StateSampler stateSampler; protected final int startState; @@ -80,7 +85,7 @@ public Operation(String operationName, } /** - * Checks that this oepration is not yet started, throwing an + * Checks that this operation is not yet started, throwing an * exception otherwise. */ void checkUnstarted() { @@ -93,7 +98,7 @@ && supportsRestart()))) { } /** - * Checks that this oepration has been started but not yet finished, + * Checks that this operation has been started but not yet finished, * throwing an exception otherwise. */ void checkStarted() { @@ -104,7 +109,7 @@ void checkStarted() { } /** - * Checks that this oepration has been finished, throwing an + * Checks that this operation has been finished, throwing an * exception otherwise. */ void checkFinished() { @@ -114,13 +119,22 @@ void checkFinished() { } } + /** + * Returns true if this Operation has been finished. + */ + boolean isFinished() { + return (initializationState == InitializationState.FINISHED); + } + /** * Starts this Operation's execution. Called after all successsor * consuming operations have been started. */ public void start() throws Exception { - checkUnstarted(); - initializationState = InitializationState.STARTED; + synchronized (initializationStateLock) { + checkUnstarted(); + initializationState = InitializationState.STARTED; + } } /** @@ -128,8 +142,10 @@ public void start() throws Exception { * predecessor producing operations have been finished. */ public void finish() throws Exception { - checkStarted(); - initializationState = InitializationState.FINISHED; + synchronized (initializationStateLock) { + checkStarted(); + initializationState = InitializationState.FINISHED; + } } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java index b727961692ba6..73b289ce9a227 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java @@ -33,7 +33,7 @@ /** * A read operation. - * + *

    * Its start() method iterates through all elements of the source * and emits them on its output. */ @@ -227,12 +227,18 @@ public Reader.Progress getProgress() { * {@code null} if the source iterator has not been initialized */ public Reader.Position proposeStopPosition(Reader.Progress proposedStopPosition) { - synchronized (sourceIteratorLock) { - if (readerIterator == null) { - LOG.warn("Iterator has not been initialized, returning null stop position."); + synchronized (initializationStateLock) { + if (isFinished()) { + LOG.warn("Iterator is in the Finished state, returning null stop position."); return null; } - return readerIterator.updateStopPosition(proposedStopPosition); + synchronized (sourceIteratorLock) { + if (readerIterator == null) { + LOG.warn("Iterator has not been initialized, returning null stop position."); + return null; + } + return readerIterator.updateStopPosition(proposedStopPosition); + } } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java index 922e32f1887af..8d754bd0d4368 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java @@ -197,7 +197,6 @@ public void testGetProgressAndProposeStopPosition() throws Exception { cloudProgressToReaderProgress(makeApproximateProgress(proposedStopPosition)))); readOperation.start(); - readOperation.finish(); TestTextReader.TestTextReaderIterator testIterator = (TestTextReader.TestTextReaderIterator) readOperation.readerIterator; @@ -218,6 +217,11 @@ public void testGetProgressAndProposeStopPosition() throws Exception { receiver.progresses, contains( makeApproximateProgress(1L), makeApproximateProgress(2L), makeApproximateProgress(3L))); + + readOperation.finish(); + + Assert.assertNull(readOperation.proposeStopPosition( + cloudProgressToReaderProgress(makeApproximateProgress(proposedStopPosition)))); } @Test From f42c13c8b917f1228f78ec16b63dcc14b7e83bac Mon Sep 17 00:00:00 2001 From: kirpichov Date: Sun, 8 Feb 2015 12:25:21 -0800 Subject: [PATCH 0140/1541] Generalizes updateStopPosition to requestFork, which can potentially return something that isn't a position. A few minor cleanups along the way. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85842155 --- .../BasicSerializableSourceFormat.java | 16 +- .../sdk/runners/worker/BigQueryReader.java | 7 +- .../worker/DataflowWorkProgressUpdater.java | 17 +- .../sdk/runners/worker/DataflowWorker.java | 22 +- .../sdk/runners/worker/FileBasedReader.java | 56 ++- .../runners/worker/GroupingShuffleReader.java | 40 +- .../sdk/runners/worker/InMemoryReader.java | 49 ++- .../worker/SourceTranslationUtils.java | 53 ++- .../util/common/worker/MapTaskExecutor.java | 5 +- .../sdk/util/common/worker/ReadOperation.java | 12 +- .../sdk/util/common/worker/Reader.java | 75 +++- .../sdk/util/common/worker/WorkExecutor.java | 8 +- .../common/worker/WorkProgressUpdater.java | 25 +- .../BasicSerializableSourceFormatTest.java | 15 +- .../DataflowWorkProgressUpdaterTest.java | 143 ++++--- .../worker/GroupingShuffleReaderTest.java | 257 ++++++------- .../runners/worker/InMemoryReaderTest.java | 85 +++-- .../sdk/runners/worker/ReaderTestUtils.java | 72 ++++ .../sdk/runners/worker/TextReaderTest.java | 308 +++++++-------- .../common/worker/MapTaskExecutorTest.java | 35 +- .../util/common/worker/ReadOperationTest.java | 351 ++++++++---------- 21 files changed, 832 insertions(+), 819 deletions(-) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderTestUtils.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormat.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormat.java index 4f723e66e7315..e2ab2d86aa13b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormat.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormat.java @@ -45,6 +45,7 @@ import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.cloud.dataflow.sdk.util.common.worker.SourceFormat; import org.slf4j.Logger; @@ -100,9 +101,8 @@ public OperationResponse performSourceOperation(OperationRequest request) throws */ @SuppressWarnings("unchecked") public static com.google.cloud.dataflow.sdk.util.common.worker.Reader create( - final PipelineOptions options, CloudObject spec, - final Coder> coder, final ExecutionContext executionContext) - throws Exception { + final PipelineOptions options, CloudObject spec, final Coder> coder, + final ExecutionContext executionContext) throws Exception { final Source source = (Source) deserializeFromCloudSource(spec); return new com.google.cloud.dataflow.sdk.util.common.worker.Reader() { @Override @@ -151,13 +151,12 @@ private SourceGetMetadataResponse performGetMetadata(SourceGetMetadataRequest re return response; } - private static Source deserializeFromCloudSource(Map spec) - throws Exception { + private static Source deserializeFromCloudSource(Map spec) throws Exception { return (Source) deserializeFromByteArray( Base64.decodeBase64(getString(spec, SERIALIZED_SOURCE)), "Source"); } - private static com.google.api.services.dataflow.model.Source serializeToCloudSource( + static com.google.api.services.dataflow.model.Source serializeToCloudSource( Source source, PipelineOptions options) throws Exception { com.google.api.services.dataflow.model.Source cloudSource = new com.google.api.services.dataflow.model.Source(); @@ -276,13 +275,12 @@ public void close() throws IOException { } @Override - public com.google.cloud.dataflow.sdk.util.common.worker.Reader.Progress getProgress() { + public Reader.Progress getProgress() { return null; } @Override - public com.google.cloud.dataflow.sdk.util.common.worker.Reader.Position updateStopPosition( - com.google.cloud.dataflow.sdk.util.common.worker.Reader.Progress proposedStopPosition) { + public Reader.ForkResult requestFork(Reader.ForkRequest request) { return null; } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReader.java index 3d442f3ffc768..1a6a2937d8f9d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReader.java @@ -16,8 +16,6 @@ package com.google.cloud.dataflow.sdk.runners.worker; -import static com.google.api.client.util.Preconditions.checkNotNull; - import com.google.api.services.bigquery.Bigquery; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; @@ -97,11 +95,10 @@ public Progress getProgress() { } @Override - public Position updateStopPosition(Progress proposedStopPosition) { - // For now updating the stop position is not supported because this source + public ForkResult requestFork(ForkRequest forkRequest) { + // For now fork is not supported because this source // is used only when an entire table needs to be read by each worker (used // as a side input for instance). - checkNotNull(proposedStopPosition); throw new UnsupportedOperationException(); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java index 250eb431ad4a3..66c1fd1a2c862 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java @@ -18,7 +18,7 @@ import static com.google.cloud.dataflow.sdk.runners.worker.DataflowWorker.buildStatus; import static com.google.cloud.dataflow.sdk.runners.worker.DataflowWorker.uniqueId; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.toForkRequest; import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudDuration; import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime; import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; @@ -75,30 +75,23 @@ protected long getWorkUnitLeaseExpirationTimestamp() { @Override protected void reportProgressHelper() throws Exception { WorkItemStatus status = buildStatus(workItem, false/*completed*/, worker.getOutputCounters(), - worker.getOutputMetrics(), options, worker.getWorkerProgress(), stopPositionToService, + worker.getOutputMetrics(), options, worker.getWorkerProgress(), forkResultToReport, null/*sourceOperationResponse*/, null/*errors*/); status.setRequestedLeaseDuration(toCloudDuration(Duration.millis(requestedLeaseDurationMs))); WorkItemServiceState result = workUnitClient.reportWorkItemStatus(status); if (result != null) { // Resets state after a successful progress report. - stopPositionToService = null; + forkResultToReport = null; progressReportIntervalMs = nextProgressReportInterval( fromCloudDuration(workItem.getReportStatusInterval()).getMillis(), leaseRemainingTime(getLeaseExpirationTimestamp(result))); ApproximateProgress suggestedStopPoint = result.getSuggestedStopPoint(); - if (suggestedStopPoint == null && result.getSuggestedStopPosition() != null) { - suggestedStopPoint = - new ApproximateProgress().setPosition(result.getSuggestedStopPosition()); - } - if (suggestedStopPoint != null) { - LOG.info("Proposing stop progress on work unit {} at proposed stopping point {}", - workString(), suggestedStopPoint); - stopPositionToService = - worker.proposeStopPosition(cloudProgressToReaderProgress(suggestedStopPoint)); + LOG.info("Proposing fork of work unit {} at {}", workString(), suggestedStopPoint); + forkResultToReport = worker.requestFork(toForkRequest(suggestedStopPoint)); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java index 555e15a0bc403..3a46be9c173cd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java @@ -17,9 +17,9 @@ package com.google.cloud.dataflow.sdk.runners.worker; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudSourceOperationResponseToSourceOperationResponse; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.readerProgressToCloudProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceOperationResponseToCloudSourceOperationResponse; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.toCloudPosition; import com.google.api.services.dataflow.model.MetricUpdate; import com.google.api.services.dataflow.model.Status; @@ -219,8 +219,8 @@ private static String buildCloudStackTrace(Throwable t) { private void reportStatus(DataflowWorkerHarnessOptions options, String status, WorkItem workItem, @Nullable CounterSet counters, @Nullable Collection> metrics, - @Nullable SourceFormat.OperationResponse operationResponse, - @Nullable List errors) throws IOException { + @Nullable SourceFormat.OperationResponse operationResponse, @Nullable List errors) + throws IOException { LOG.info("{} processing work item {}", status, uniqueId(workItem)); WorkItemStatus workItemStatus = buildStatus(workItem, true/*completed*/, counters, metrics, options, null, null, operationResponse, errors); @@ -230,9 +230,8 @@ private void reportStatus(DataflowWorkerHarnessOptions options, String status, W static WorkItemStatus buildStatus(WorkItem workItem, boolean completed, @Nullable CounterSet counters, @Nullable Collection> metrics, DataflowWorkerHarnessOptions options, @Nullable Reader.Progress progress, - @Nullable Reader.Position stopPosition, - @Nullable SourceFormat.OperationResponse operationResponse, - @Nullable List errors) { + @Nullable Reader.ForkResult forkResult, + @Nullable SourceFormat.OperationResponse operationResponse, @Nullable List errors) { WorkItemStatus status = new WorkItemStatus(); status.setWorkItemId(Long.toString(workItem.getId())); status.setCompleted(completed); @@ -272,10 +271,13 @@ static WorkItemStatus buildStatus(WorkItem workItem, boolean completed, } if (progress != null) { - status.setProgress(sourceProgressToCloudProgress(progress)); + status.setProgress(readerProgressToCloudProgress(progress)); } - if (stopPosition != null) { - status.setStopPosition(sourcePositionToCloudPosition(stopPosition)); + if (forkResult instanceof Reader.ForkResultWithPosition) { + Reader.ForkResultWithPosition asPosition = (Reader.ForkResultWithPosition) forkResult; + status.setStopPosition(toCloudPosition(asPosition.getAcceptedPosition())); + } else if (forkResult != null) { + throw new IllegalArgumentException("Unexpected type of fork result: " + forkResult); } if (workItem.getSourceOperationTask() != null) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java index 53ebbd52ad82f..218f1f5b5f661 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java @@ -19,7 +19,7 @@ import static com.google.api.client.util.Preconditions.checkNotNull; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.forkRequestToApproximateProgress; import com.google.api.services.dataflow.model.ApproximateProgress; import com.google.cloud.dataflow.sdk.coders.Coder; @@ -52,8 +52,10 @@ public abstract class FileBasedReader extends Reader { protected static final int BUF_SIZE = 200; protected final String filename; + @Nullable protected final Long startPosition; + @Nullable protected final Long endPosition; protected final Coder coder; @@ -191,44 +193,37 @@ public Progress getProgress() { } @Override - public Position updateStopPosition(Progress proposedStopPosition) { - checkNotNull(proposedStopPosition); - - // Currently we only support stop position in byte offset of - // CloudPosition in a file-based Reader. If stop position in - // other types is proposed, the end position in iterator will - // not be updated, and return null. - com.google.api.services.dataflow.model.ApproximateProgress stopPosition = - sourceProgressToCloudProgress(proposedStopPosition); - if (stopPosition == null) { - LOG.warn("A stop position other than CloudPosition is not supported now."); + public ForkResult requestFork(ForkRequest forkRequest) { + checkNotNull(forkRequest); + + // Currently, file-based Reader only supports fork at a byte offset. + ApproximateProgress forkProgress = forkRequestToApproximateProgress(forkRequest); + com.google.api.services.dataflow.model.Position forkPosition = forkProgress.getPosition(); + if (forkPosition == null) { + LOG.warn("FileBasedReader only supports fork at a Position. Requested: {}", forkRequest); return null; } - - Long byteOffset = stopPosition.getPosition().getByteOffset(); - if (byteOffset == null) { - LOG.warn("A proposed stop position must be a byte offset for a file-based Source."); + Long forkOffset = forkPosition.getByteOffset(); + if (forkOffset == null) { + LOG.warn("FileBasedReader only supports fork at byte offset. Requested: {}", forkPosition); return null; } - if (byteOffset <= offset) { - // Proposed stop position is not after the current position: - // No stop position update. - LOG.warn("The proposed stop position " + byteOffset - + " is past the current position " + offset); + if (forkOffset <= offset) { + LOG.info("Already progressed to offset {} which is after the requested fork offset {}", + offset, forkOffset); return null; } - if (endOffset != null && byteOffset >= endOffset) { - // Proposed stop position is after the current stop (end) position: No - // stop position update. - LOG.warn("The proposed stop position " + byteOffset - + " is after the current stop position " + endOffset); - return null; + if (endOffset != null && forkOffset >= endOffset) { + throw new IllegalArgumentException( + "Fork requested at an offset beyond the end of the current range: " + forkOffset + + " >= " + endOffset); } - LOG.info("Updated the stop position to offset " + byteOffset); - this.endOffset = byteOffset; - return cloudPositionToReaderPosition(stopPosition.getPosition()); + this.endOffset = forkOffset; + LOG.info("Forked FileBasedReader at offset {}", forkOffset); + + return new ForkResultWithPosition(cloudPositionToReaderPosition(forkPosition)); } /** @@ -262,7 +257,6 @@ private void computeNextElement() throws IOException { * Factory interface for creating a decompressing {@link InputStream}. */ public interface DecompressingStreamFactory { - /** * Create a decompressing {@link InputStream} from an existing {@link InputStream}. * diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java index 7d0f92313e9fd..0cb1d98408a79 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java @@ -19,7 +19,7 @@ import static com.google.api.client.util.Preconditions.checkNotNull; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.forkRequestToApproximateProgress; import com.google.api.client.util.Preconditions; import com.google.api.services.dataflow.model.ApproximateProgress; @@ -225,39 +225,41 @@ public Progress getProgress() { * {@code KV>} to be returned by the {@link GroupingShuffleReaderIterator}. */ @Override - public Position updateStopPosition(Progress proposedStopPosition) { - checkNotNull(proposedStopPosition); - com.google.api.services.dataflow.model.Position stopCloudPosition = - sourceProgressToCloudProgress(proposedStopPosition).getPosition(); - if (stopCloudPosition == null) { - LOG.warn("A stop position other than a Position is not supported now."); + public ForkResult requestFork(ForkRequest forkRequest) { + checkNotNull(forkRequest); + ApproximateProgress forkProgress = forkRequestToApproximateProgress(forkRequest); + com.google.api.services.dataflow.model.Position forkPosition = forkProgress.getPosition(); + if (forkPosition == null) { + LOG.warn("GroupingShuffleReader only supports fork at a Position. Requested: {}", + forkRequest); return null; } - - if (stopCloudPosition.getShufflePosition() == null) { - LOG.warn("A stop position other than shuffle position is not supported in " - + "a grouping shuffle source: " + stopCloudPosition.toString()); + String forkShufflePosition = forkPosition.getShufflePosition(); + if (forkShufflePosition == null) { + LOG.warn("GroupingShuffleReader only supports fork at a shuffle position. Requested: {}", + forkPosition); return null; } ByteArrayShufflePosition newStopPosition = - ByteArrayShufflePosition.fromBase64(stopCloudPosition.getShufflePosition()); - + ByteArrayShufflePosition.fromBase64(forkShufflePosition); if (newStopPosition.compareTo(promisedPosition) <= 0) { - LOG.warn("Proposed stop position: " + stopCloudPosition.getShufflePosition() - + " <= promised position: " + promisedPosition.encodeBase64()); + LOG.info("Already progressed to promised shuffle position {} " + + "which is after the requested fork shuffle position {}", + promisedPosition.encodeBase64(), forkShufflePosition); return null; } if (this.stopPosition != null && newStopPosition.compareTo(this.stopPosition) >= 0) { - LOG.warn("Proposed stop position: " + stopCloudPosition.getShufflePosition() + throw new IllegalArgumentException( + "Fork requested at a shuffle position beyond the end of the current range: " + + forkShufflePosition + " >= current stop position: " + this.stopPosition.encodeBase64()); - return null; } this.stopPosition = newStopPosition; - LOG.info("Updated the stop position to " + stopCloudPosition.getShufflePosition()); + LOG.info("Forked GroupingShuffleReader at {}", forkShufflePosition); - return cloudPositionToReaderPosition(stopCloudPosition); + return new ForkResultWithPosition(cloudPositionToReaderPosition(forkPosition)); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java index 72107b979d841..3fa6cb61a7387 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java @@ -19,7 +19,7 @@ import static com.google.api.client.util.Preconditions.checkNotNull; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.forkRequestToApproximateProgress; import static java.lang.Math.min; import com.google.api.services.dataflow.model.ApproximateProgress; @@ -124,38 +124,37 @@ public Progress getProgress() { } @Override - public Position updateStopPosition(Progress proposedStopPosition) { - checkNotNull(proposedStopPosition); - - // Currently we only support stop position in record index of - // an API Position in InMemoryReader. If stop position in other types is - // proposed, the end position in iterator will not be updated, - // and return null. - com.google.api.services.dataflow.model.Position stopPosition = - sourceProgressToCloudProgress(proposedStopPosition).getPosition(); - if (stopPosition == null) { - LOG.warn("A stop position other than a Dataflow API Position is not currently supported."); + public ForkResult requestFork(ForkRequest forkRequest) { + checkNotNull(forkRequest); + + com.google.api.services.dataflow.model.Position forkPosition = + forkRequestToApproximateProgress(forkRequest).getPosition(); + if (forkPosition == null) { + LOG.warn("InMemoryReader only supports fork at a Position. Requested: {}", forkRequest); return null; } - Long recordIndex = stopPosition.getRecordIndex(); - if (recordIndex == null) { - LOG.warn("A proposed stop position must be a record index for InMemoryReader."); + Long forkIndex = forkPosition.getRecordIndex(); + if (forkIndex == null) { + LOG.warn("InMemoryReader only supports fork at a record index. Requested: {}", + forkPosition); return null; } - if (recordIndex <= index || recordIndex >= endPosition) { - // Proposed stop position is not after the current position or proposed - // stop position is after the current stop (end) position: No stop - // position update. - LOG.warn("The proposed stop position " + recordIndex - + " is not between the current stop position " + index - + " and the current stop position " + endPosition); + if (forkIndex <= index) { + LOG.info("Already progressed to index {} which is after the requested fork index {}", + index, forkIndex); return null; } + if (forkIndex >= endPosition) { + throw new IllegalArgumentException( + "Fork requested at an index beyond the end of the current range: " + forkIndex + + " >= " + endPosition); + } + + this.endPosition = forkIndex.intValue(); + LOG.info("Forked InMemoryReader at index {}", forkIndex); - LOG.info("Updated the stop position to record " + recordIndex); - this.endPosition = recordIndex.intValue(); - return cloudPositionToReaderPosition(stopPosition); + return new ForkResultWithPosition(cloudPositionToReaderPosition(forkPosition)); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java index 07229941a82af..6386d038fbf79 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java @@ -50,9 +50,8 @@ public static Reader.Position cloudPositionToReaderPosition(@Nullable Position c return cloudPosition == null ? null : new DataflowReaderPosition(cloudPosition); } - public static SourceFormat.OperationRequest - cloudSourceOperationRequestToSourceOperationRequest( - @Nullable SourceOperationRequest request) { + public static SourceFormat.OperationRequest cloudSourceOperationRequestToSourceOperationRequest( + @Nullable SourceOperationRequest request) { return request == null ? null : new DataflowSourceOperationRequest(request); } @@ -62,18 +61,13 @@ public static Reader.Position cloudPositionToReaderPosition(@Nullable Position c return response == null ? null : new DataflowSourceOperationResponse(response); } - public static SourceFormat.SourceSpec cloudSourceToSourceSpec( - @Nullable Source cloudSource) { - return cloudSource == null ? null : new DataflowSourceSpec(cloudSource); + public static ApproximateProgress readerProgressToCloudProgress( + @Nullable Reader.Progress readerProgress) { + return readerProgress == null ? null : ((DataflowReaderProgress) readerProgress).cloudProgress; } - public static ApproximateProgress sourceProgressToCloudProgress( - @Nullable Reader.Progress sourceProgress) { - return sourceProgress == null ? null : ((DataflowReaderProgress) sourceProgress).cloudProgress; - } - - public static Position sourcePositionToCloudPosition(@Nullable Reader.Position sourcePosition) { - return sourcePosition == null ? null : ((DataflowReaderPosition) sourcePosition).cloudPosition; + public static Position toCloudPosition(@Nullable Reader.Position readerPosition) { + return readerPosition == null ? null : ((DataflowReaderPosition) readerPosition).cloudPosition; } @@ -91,8 +85,19 @@ public static Source sourceSpecToCloudSource(@Nullable SourceFormat.SourceSpec s return (spec == null) ? null : ((DataflowSourceSpec) spec).cloudSource; } + public static ApproximateProgress forkRequestToApproximateProgress( + @Nullable Reader.ForkRequest stopRequest) { + return (stopRequest == null) ? null : ((DataflowForkRequest) stopRequest).approximateProgress; + } + + public static Reader.ForkRequest toForkRequest( + @Nullable ApproximateProgress approximateProgress) { + return (approximateProgress == null) ? null : new DataflowForkRequest(approximateProgress); + } + static class DataflowReaderProgress implements Reader.Progress { public final ApproximateProgress cloudProgress; + public DataflowReaderProgress(ApproximateProgress cloudProgress) { this.cloudProgress = cloudProgress; } @@ -100,6 +105,7 @@ public DataflowReaderProgress(ApproximateProgress cloudProgress) { static class DataflowReaderPosition implements Reader.Position { public final Position cloudPosition; + public DataflowReaderPosition(Position cloudPosition) { this.cloudPosition = cloudPosition; } @@ -107,6 +113,7 @@ public DataflowReaderPosition(Position cloudPosition) { static class DataflowSourceOperationRequest implements SourceFormat.OperationRequest { public final SourceOperationRequest cloudRequest; + public DataflowSourceOperationRequest(SourceOperationRequest cloudRequest) { this.cloudRequest = cloudRequest; } @@ -114,6 +121,7 @@ public DataflowSourceOperationRequest(SourceOperationRequest cloudRequest) { static class DataflowSourceOperationResponse implements SourceFormat.OperationResponse { public final SourceOperationResponse cloudResponse; + public DataflowSourceOperationResponse(SourceOperationResponse cloudResponse) { this.cloudResponse = cloudResponse; } @@ -121,6 +129,7 @@ public DataflowSourceOperationResponse(SourceOperationResponse cloudResponse) { static class DataflowSourceSpec implements SourceFormat.SourceSpec { public final Source cloudSource; + public DataflowSourceSpec(Source cloudSource) { this.cloudSource = cloudSource; } @@ -138,8 +147,8 @@ public static Map cloudSourceToDictionary(Source source) { cloudSourceMetadataToDictionary(source.getMetadata())); } if (source.getDoesNotNeedSplitting() != null) { - addBoolean(res, PropertyNames.SOURCE_DOES_NOT_NEED_SPLITTING, - source.getDoesNotNeedSplitting()); + addBoolean( + res, PropertyNames.SOURCE_DOES_NOT_NEED_SPLITTING, source.getDoesNotNeedSplitting()); } return res; } @@ -147,12 +156,10 @@ public static Map cloudSourceToDictionary(Source source) { private static Map cloudSourceMetadataToDictionary(SourceMetadata metadata) { Map res = new HashMap<>(); if (metadata.getProducesSortedKeys() != null) { - addBoolean( - res, PropertyNames.SOURCE_PRODUCES_SORTED_KEYS, metadata.getProducesSortedKeys()); + addBoolean(res, PropertyNames.SOURCE_PRODUCES_SORTED_KEYS, metadata.getProducesSortedKeys()); } if (metadata.getEstimatedSizeBytes() != null) { - addLong( - res, PropertyNames.SOURCE_ESTIMATED_SIZE_BYTES, metadata.getEstimatedSizeBytes()); + addLong(res, PropertyNames.SOURCE_ESTIMATED_SIZE_BYTES, metadata.getEstimatedSizeBytes()); } if (metadata.getInfinite() != null) { addBoolean(res, PropertyNames.SOURCE_IS_INFINITE, metadata.getInfinite()); @@ -167,4 +174,12 @@ public static Source dictionaryToCloudSource(Map params) throws // translated, because they only make sense in cloud Source objects produced by the user. return res; } + + private static class DataflowForkRequest implements Reader.ForkRequest { + public final ApproximateProgress approximateProgress; + + private DataflowForkRequest(ApproximateProgress approximateProgress) { + this.approximateProgress = approximateProgress; + } + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java index 793343d6d1bb4..717c44aeec422 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java @@ -85,9 +85,8 @@ public Reader.Progress getWorkerProgress() throws Exception { } @Override - public Reader.Position proposeStopPosition(Reader.Progress proposedStopPosition) - throws Exception { - return getReadOperation().proposeStopPosition(proposedStopPosition); + public Reader.ForkResult requestFork(Reader.ForkRequest forkRequest) throws Exception { + return getReadOperation().requestFork(forkRequest); } ReadOperation getReadOperation() throws Exception { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java index 73b289ce9a227..d7f3f479f0ee3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java @@ -220,13 +220,9 @@ public Reader.Progress getProgress() { } /** - * Relays the request to update the stop position to {@code ReaderIterator}. - * - * @param proposedStopPosition the proposed stop position - * @return the new stop position updated in {@code ReaderIterator}, or - * {@code null} if the source iterator has not been initialized + * Relays the fork request to {@code ReaderIterator}. */ - public Reader.Position proposeStopPosition(Reader.Progress proposedStopPosition) { + public Reader.ForkResult requestFork(Reader.ForkRequest forkRequest) { synchronized (initializationStateLock) { if (isFinished()) { LOG.warn("Iterator is in the Finished state, returning null stop position."); @@ -234,10 +230,10 @@ public Reader.Position proposeStopPosition(Reader.Progress proposedStopPosition) } synchronized (sourceIteratorLock) { if (readerIterator == null) { - LOG.warn("Iterator has not been initialized, returning null stop position."); + LOG.warn("Iterator has not been initialized, refusing to fork at {}", forkRequest); return null; } - return readerIterator.updateStopPosition(proposedStopPosition); + return readerIterator.requestFork(forkRequest); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java index 7c9b783511c01..93a398907dd1e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java @@ -77,25 +77,38 @@ public interface ReaderIterator extends AutoCloseable { public Progress getProgress(); /** - * Attempts to update the stop position of the task with the proposed stop - * position and returns the actual new stop position. - * - *

    If the source finds the proposed one is not a convenient position to - * stop, it can pick a different stop position. The {@code ReaderIterator} - * should start returning {@code false} from {@code hasNext()} once it has - * passed its stop position. Subsequent stop position updates must be in - * non-increasing order within a task. - * - *

    This method is not required to be thread-safe, and it will not be + * Attempts to split the input in two parts: the "primary" part and the "residual" part. + * The current {@link ReaderIterator} keeps processing the primary part, while the residual part + * will be processed elsewhere (e.g. perhaps on a different worker). + *

    + * The primary and residual parts, if concatenated, must represent the same input as the + * current input of this {@link ReaderIterator} before this call. + *

    + * The boundary between the primary part and the residual part is specified in + * a framework-specific way using {@link ForkRequest}: e.g., if the framework supports the + * notion of positions, it might be a position at which the input is asked to split itself + * (which is not necessarily the same position at which it will split itself); it might + * be an approximate fraction of input, or something else. + *

    + * {@link ForkResult} encodes, in a framework-specific way, the information sufficient to + * construct a description of the resulting primary and residual inputs. For example, it might, + * again, be a position demarcating these parts, or it might be a pair of fully-specified input + * descriptions, or something else. + *

    + * After a successful call to {@link #requestFork}, subsequent calls should be interpreted + * relative to the new primary. + *

    + * This method is not required to be thread-safe, and it will not be * called concurrently to any other methods. + *

    + * This call should not affect the range of input represented by the {@link Reader} which + * produced this {@link ReaderIterator}. * - * @param proposedStopPosition a proposed position to stop - * iterating through the source - * @return the new stop position, or {@code null} on failure if the - * implementation does not support position updates(implementors are discouraged - * from throwing {@code UnsupportedOperationException} in this case). + * @return {@code null} if the {@link ForkRequest} cannot be honored (in that case the input + * represented by this {@link ReaderIterator} stays the same), or a {@link ForkResult} + * describing how the input was split into a primary and residual part. */ - public Position updateStopPosition(Progress proposedStopPosition); + public ForkResult requestFork(ForkRequest request); } /** An abstract base class for ReaderIterator implementations. */ @@ -116,7 +129,7 @@ public Progress getProgress() { } @Override - public Position updateStopPosition(Progress proposedStopPosition) { + public ForkResult requestFork(ForkRequest forkRequest) { return null; } } @@ -141,6 +154,34 @@ public interface Progress {} */ public interface Position {} + /** + * A framework-specific way to specify how {@link ReaderIterator#requestFork} should split + * the input into a primary and residual part. + */ + public interface ForkRequest {} + + /** + * A framework-specific way to specify how {@link ReaderIterator#requestFork} has split + * the input into a primary and residual part. + */ + public interface ForkResult {} + + /** + * A {@link ForkResult} which specifies the boundary between the primary and residual parts + * of the input using a {@link Position}. + */ + public static final class ForkResultWithPosition implements ForkResult { + private final Position acceptedPosition; + + public ForkResultWithPosition(Position acceptedPosition) { + this.acceptedPosition = acceptedPosition; + } + + public Position getAcceptedPosition() { + return acceptedPosition; + } + } + /** * Utility method to notify observers about a new element, which has * been read by this Reader, and its size in bytes. Normally, there diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java index 18ab4762d171d..117affab8d9a1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkExecutor.java @@ -83,12 +83,10 @@ public Reader.Progress getWorkerProgress() throws Exception { } /** - * Proposes that the worker changes the stop position for the current work. - * Returns the new position if accepted, otherwise {@code null}. + * See {@link Reader.ReaderIterator#requestFork}. Makes sense only for tasks which read input. */ - public Reader.Position proposeStopPosition(Reader.Progress proposedStopPosition) - throws Exception { - // By default, returns null indicating that no task splitting happens. + public Reader.ForkResult requestFork(Reader.ForkRequest forkRequest) throws Exception { + // By default, fork is unsupported. return null; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java index 4794fe3efb2ec..f212d42e469f1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java @@ -78,16 +78,11 @@ public abstract class WorkProgressUpdater { protected long progressReportIntervalMs; /** - * The stop position to report to the service in the next progress update, - * or {@code null} if there is nothing to report. - * In cases that there is no split request from service, or worker failed to - * split in response to the last received split request, the task stop - * position implicitly stays the same as it was before that last request - * (as a result of a prior split request), and on the next reportProgress - * we'll send the {@code null} as a stop position update, which is a no-op - * for the service. + * The {@link Reader.ForkResult} to report to the service in the next progress update, + * or {@code null} if there is nothing to report (if no fork happened since the last progress + * update). */ - protected Reader.Position stopPositionToService; + protected Reader.ForkResult forkResultToReport; public WorkProgressUpdater(WorkExecutor worker) { this.worker = worker; @@ -124,10 +119,10 @@ public void stopReportingProgress() throws Exception { executor.shutdownNow(); } - // We send a final progress report in case there was an unreported stop position update. - if (stopPositionToService != null) { - LOG.info("Sending final progress update with unreported stop position: {} " - + "for work item: {}", stopPositionToService, workString()); + // We send a final progress report in case there was an unreported fork. + if (forkResultToReport != null) { + LOG.info("Sending final progress update with unreported fork: {} " + + "for work item: {}", forkResultToReport, workString()); reportProgressHelper(); // This call can fail with an exception } @@ -215,8 +210,8 @@ protected long leaseRemainingTime(long leaseExpirationTimestamp) { } // Visible for testing. - public Reader.Position getStopPosition() { - return stopPositionToService; + public Reader.ForkResult getForkResultToReport() { + return forkResultToReport; } /** diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormatTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormatTest.java index 91d0003eea20c..34dc51bb3c041 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormatTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/BasicSerializableSourceFormatTest.java @@ -102,10 +102,9 @@ public boolean producesSortedKeys(PipelineOptions options) throws Exception { } @Override - public Reader createBasicReader( - PipelineOptions options, Coder coder, + public Reader createBasicReader(PipelineOptions options, Coder coder, @Nullable ExecutionContext executionContext) throws IOException { - return new RangeReader(from, to); + return new RangeReader(this); } @Override @@ -117,12 +116,10 @@ public Coder getDefaultOutputCoder() { } private class RangeReader implements Reader { - private int to; private int current; - public RangeReader(int from, int to) { - this.to = to; - this.current = from - 1; + public RangeReader(Read source) { + this.current = source.from - 1; } @Override @@ -149,8 +146,8 @@ public void testSplitAndReadShardsBack() throws Exception { DataflowPipelineOptions options = PipelineOptionsFactory.create().as(DataflowPipelineOptions.class); options.setNumWorkers(5); - com.google.api.services.dataflow.model.Source source = translateIOToCloudSource( - TestIO.fromRange(10, 20), options); + com.google.api.services.dataflow.model.Source source = + translateIOToCloudSource(TestIO.fromRange(10, 20), options); List> elems = CloudSourceUtils.readElemsFromSource(options, source); assertEquals(10, elems.size()); for (int i = 0; i < 10; ++i) { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java index 081e0b8c14ed9..0745e40faef03 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java @@ -16,10 +16,13 @@ package com.google.cloud.dataflow.sdk.runners.worker; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.approximateProgressAtIndex; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.approximateProgressAtPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.positionAtIndex; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.forkRequestToApproximateProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.toCloudPosition; import static com.google.cloud.dataflow.sdk.util.CloudCounterUtils.extractCounter; import static com.google.cloud.dataflow.sdk.util.CloudMetricUtils.extractCloudMetric; import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; @@ -93,13 +96,14 @@ public Reader.Progress getWorkerProgress() { } @Override - public Reader.Position proposeStopPosition(Reader.Progress suggestedStopPoint) { + public Reader.ForkResult requestFork(Reader.ForkRequest forkRequest) { @Nullable - ApproximateProgress progress = sourceProgressToCloudProgress(suggestedStopPoint); + ApproximateProgress progress = forkRequestToApproximateProgress(forkRequest); if (progress == null) { return null; } - return cloudPositionToReaderPosition(progress.getPosition()); + return new Reader.ForkResultWithPosition( + cloudPositionToReaderPosition(progress.getPosition())); } public void setWorkerProgress(ApproximateProgress progress) { @@ -125,6 +129,7 @@ public void setWorkerProgress(ApproximateProgress progress) { @Rule public final ExpectedException thrown = ExpectedException.none(); + @Mock private DataflowWorker.WorkUnitClient workUnitClient; private CounterSet counters; @@ -173,60 +178,60 @@ public void workProgressUpdaterUpdates() throws Exception { .thenReturn(generateServiceState(nowMillis + 2000, 1000, null)); setUpCounters(2); setUpMetrics(3); - setUpProgress(makeRecordIndexProgress(1L)); + setUpProgress(approximateProgressAtIndex(1L)); progressUpdater.startReportingProgress(); // The initial update should be sent after leaseRemainingTime / 2. verify(workUnitClient, timeout(600)) - .reportWorkItemStatus( - argThat(new ExpectedDataflowProgress().withCounters(2).withMetrics(3).withProgress( - makeRecordIndexProgress(1L)))); + .reportWorkItemStatus(argThat( + new ExpectedDataflowWorkItemStatus().withCounters(2).withMetrics(3).withProgress( + approximateProgressAtIndex(1L)))); progressUpdater.stopReportingProgress(); } // Verifies that ReportWorkItemStatusRequest contains correct progress report - // and actual stop position report. + // and actual fork result. @Test(timeout = 5000) public void workProgressUpdaterAdaptsProgressInterval() throws Exception { // Mock that the next reportProgress call will return a response that asks // us to truncate the task at index 3, and the next two will not ask us to // truncate at all. when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) - .thenReturn(generateServiceState(nowMillis + 2000, 1000, makeRecordIndexPosition(3L))) + .thenReturn(generateServiceState(nowMillis + 2000, 1000, positionAtIndex(3L))) .thenReturn(generateServiceState(nowMillis + 3000, 2000, null)) .thenReturn(generateServiceState(nowMillis + 4000, 3000, null)); setUpCounters(3); setUpMetrics(2); - setUpProgress(makeRecordIndexProgress(1L)); + setUpProgress(approximateProgressAtIndex(1L)); progressUpdater.startReportingProgress(); // The initial update should be sent after // leaseRemainingTime (1000) / 2 = 500. verify(workUnitClient, timeout(600)) - .reportWorkItemStatus( - argThat(new ExpectedDataflowProgress().withCounters(3).withMetrics(2).withProgress( - makeRecordIndexProgress(1L)))); + .reportWorkItemStatus(argThat( + new ExpectedDataflowWorkItemStatus().withCounters(3).withMetrics(2).withProgress( + approximateProgressAtIndex(1L)))); setUpCounters(5); setUpMetrics(6); - setUpProgress(makeRecordIndexProgress(2L)); + setUpProgress(approximateProgressAtIndex(2L)); // The second update should be sent after one second (2000 / 2). verify(workUnitClient, timeout(1100)) .reportWorkItemStatus(argThat( - new ExpectedDataflowProgress() + new ExpectedDataflowWorkItemStatus() .withCounters(5) .withMetrics(6) - .withProgress(makeRecordIndexProgress(2L)) - .withStopPosition(makeRecordIndexPosition(3L)))); + .withProgress(approximateProgressAtIndex(2L)) + .withForkAtPosition(positionAtIndex(3L)))); - // After the request is sent, reset stop position cache to null. - assertNull(progressUpdater.getStopPosition()); + // After the request is sent, reset cached fork result to null. + assertNull(progressUpdater.getForkResultToReport()); - setUpProgress(makeRecordIndexProgress(3L)); + setUpProgress(approximateProgressAtIndex(3L)); // The third update should be sent after one and half seconds (3000 / 2). verify(workUnitClient, timeout(1600)) - .reportWorkItemStatus( - argThat(new ExpectedDataflowProgress().withProgress(makeRecordIndexProgress(3L)))); + .reportWorkItemStatus(argThat( + new ExpectedDataflowWorkItemStatus().withProgress(approximateProgressAtIndex(3L)))); progressUpdater.stopReportingProgress(); } @@ -235,40 +240,41 @@ public void workProgressUpdaterAdaptsProgressInterval() throws Exception { @Test(timeout = 3000) public void workProgressUpdaterLastUpdate() throws Exception { when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) - .thenReturn(generateServiceState(nowMillis + 2000, 1000, makeRecordIndexPosition(2L))) + .thenReturn(generateServiceState(nowMillis + 2000, 1000, positionAtIndex(2L))) .thenReturn(generateServiceState(nowMillis + 3000, 2000, null)); - setUpProgress(makeRecordIndexProgress(1L)); + setUpProgress(approximateProgressAtIndex(1L)); progressUpdater.startReportingProgress(); // The initial update should be sent after leaseRemainingTime / 2 = 500 msec. Thread.sleep(600); verify(workUnitClient, timeout(200)) - .reportWorkItemStatus( - argThat(new ExpectedDataflowProgress().withProgress(makeRecordIndexProgress(1L)))); + .reportWorkItemStatus(argThat( + new ExpectedDataflowWorkItemStatus().withProgress(approximateProgressAtIndex(1L)))); - // The first update should include the new actual stop position. + // The first update should include the new fork result.. // Verify that the progressUpdater has recorded it. - assertEquals( - makeRecordIndexPosition(2L), - sourcePositionToCloudPosition(progressUpdater.getStopPosition())); + Reader.ForkResultWithPosition forkResult = + (Reader.ForkResultWithPosition) progressUpdater.getForkResultToReport(); + assertEquals(positionAtIndex(2L), toCloudPosition(forkResult.getAcceptedPosition())); - setUpProgress(makeRecordIndexProgress(2L)); + setUpProgress(approximateProgressAtIndex(2L)); // The second update should be sent after one second (2000 / 2). - Thread.sleep(200); // not enough time for an update so the latest stop position is not - // acknowledged. - // Check that the progressUpdater still has a pending stop position to send - assertEquals( - makeRecordIndexPosition(2L), - sourcePositionToCloudPosition(progressUpdater.getStopPosition())); - - progressUpdater.stopReportingProgress(); // should send the last update - // check that the progressUpdater is done with reporting its latest stop position - assertNull(progressUpdater.getStopPosition()); - - // Verify that the last update contained the latest stop position + + // Not enough time for an update so the latest fork result is not acknowledged. + Thread.sleep(200); + + // Check that the progressUpdater still has a pending fork result to send + forkResult = (Reader.ForkResultWithPosition) progressUpdater.getForkResultToReport(); + assertEquals(positionAtIndex(2L), toCloudPosition(forkResult.getAcceptedPosition())); + + progressUpdater.stopReportingProgress(); // Should send the last update. + // Check that the progressUpdater is done with reporting its latest fork result. + assertNull(progressUpdater.getForkResultToReport()); + + // Verify that the last update contained the latest fork result. verify(workUnitClient, timeout(1000)) - .reportWorkItemStatus( - argThat(new ExpectedDataflowProgress().withStopPosition(makeRecordIndexPosition(2L)))); + .reportWorkItemStatus(argThat( + new ExpectedDataflowWorkItemStatus().withForkAtPosition(positionAtIndex(2L)))); } private void setUpCounters(int n) { @@ -309,17 +315,6 @@ private void setUpProgress(ApproximateProgress progress) { worker.setWorkerProgress(progress); } - private com.google.api.services.dataflow.model.Position makeRecordIndexPosition(Long index) { - com.google.api.services.dataflow.model.Position position = - new com.google.api.services.dataflow.model.Position(); - position.setRecordIndex(index); - return position; - } - - private ApproximateProgress makeRecordIndexProgress(Long index) { - return new ApproximateProgress().setPosition(makeRecordIndexPosition(index)); - } - private WorkItemServiceState generateServiceState(long leaseExpirationTimestamp, int progressReportIntervalMs, Position suggestedStopPosition) throws IOException { WorkItemServiceState responseState = new WorkItemServiceState(); @@ -329,39 +324,43 @@ private WorkItemServiceState generateServiceState(long leaseExpirationTimestamp, toCloudDuration(Duration.millis(progressReportIntervalMs))); if (suggestedStopPosition != null) { - responseState.setSuggestedStopPosition(suggestedStopPosition); + responseState.setSuggestedStopPoint(approximateProgressAtPosition(suggestedStopPosition)); } return responseState; } - private static final class ExpectedDataflowProgress extends ArgumentMatcher { + private static final class ExpectedDataflowWorkItemStatus + extends ArgumentMatcher { @Nullable Integer counterCount; + @Nullable Integer metricCount; + @Nullable ApproximateProgress expectedProgress; + @Nullable - Position expectedStopPosition; + Position expectedForkPosition; - public ExpectedDataflowProgress withCounters(Integer counterCount) { + public ExpectedDataflowWorkItemStatus withCounters(Integer counterCount) { this.counterCount = counterCount; return this; } - public ExpectedDataflowProgress withMetrics(Integer metricCount) { + public ExpectedDataflowWorkItemStatus withMetrics(Integer metricCount) { this.metricCount = metricCount; return this; } - public ExpectedDataflowProgress withProgress(ApproximateProgress expectedProgress) { + public ExpectedDataflowWorkItemStatus withProgress(ApproximateProgress expectedProgress) { this.expectedProgress = expectedProgress; return this; } - public ExpectedDataflowProgress withStopPosition(Position expectedStopPosition) { - this.expectedStopPosition = expectedStopPosition; + public ExpectedDataflowWorkItemStatus withForkAtPosition(Position expectedForkPosition) { + this.expectedForkPosition = expectedForkPosition; return this; } @@ -381,12 +380,12 @@ public void describeTo(Description description) { if (this.expectedProgress != null) { values.add("progress " + this.expectedProgress); } - if (this.expectedStopPosition != null) { - values.add("stop position " + this.expectedStopPosition); + if (this.expectedForkPosition != null) { + values.add("fork position " + this.expectedForkPosition); } else { - values.add("no stop position present"); + values.add("no fork position present"); } - description.appendValueList("Dataflow progress with ", ", ", ".", values); + description.appendValueList("Dataflow WorkItemStatus with ", ", ", ".", values); } @Override @@ -431,10 +430,10 @@ private boolean matchProgress(WorkItemStatus status) { private boolean matchStopPosition(WorkItemStatus status) { Position actualStopPosition = status.getStopPosition(); - if (expectedStopPosition == null) { + if (expectedForkPosition == null) { return actualStopPosition == null; } - return expectedStopPosition.equals(actualStopPosition); + return expectedForkPosition.equals(actualStopPosition); } } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java index ff3abd3dbe426..c023123c4a6d6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java @@ -17,9 +17,18 @@ package com.google.cloud.dataflow.sdk.runners.worker; import static com.google.api.client.util.Base64.encodeBase64URLSafeString; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.forkRequestAtPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.positionFromForkResult; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.readerProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.toCloudPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.toForkRequest; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import com.google.api.services.dataflow.model.ApproximateProgress; import com.google.api.services.dataflow.model.Position; @@ -41,8 +50,8 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.common.collect.Lists; +import org.hamcrest.Matchers; import org.joda.time.Instant; -import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -56,6 +65,8 @@ import java.util.List; import java.util.NoSuchElementException; +import javax.annotation.Nullable; + /** * Tests for GroupingShuffleReader. */ @@ -84,57 +95,42 @@ private enum ValuesToRead { } private void runTestReadFromShuffle( - List>> input, - ValuesToRead valuesToRead) - throws Exception { - Coder>> sinkElemCoder = - WindowedValue.getFullCoder( - KvCoder.of(BigEndianIntegerCoder.of(), - StringUtf8Coder.of()), - IntervalWindow.getCoder()); + List>> input, ValuesToRead valuesToRead) throws Exception { + Coder>> sinkElemCoder = WindowedValue.getFullCoder( + KvCoder.of(BigEndianIntegerCoder.of(), StringUtf8Coder.of()), IntervalWindow.getCoder()); Coder>>> sourceElemCoder = WindowedValue.getFullCoder( - KvCoder.of( - BigEndianIntegerCoder.of(), - IterableCoder.of(StringUtf8Coder.of())), + KvCoder.of(BigEndianIntegerCoder.of(), IterableCoder.of(StringUtf8Coder.of())), IntervalWindow.getCoder()); // Write to shuffle with GROUP_KEYS ShuffleSink. ShuffleSink> shuffleSink = new ShuffleSink<>( - PipelineOptionsFactory.create(), - null, ShuffleSink.ShuffleKind.GROUP_KEYS, - sinkElemCoder); + PipelineOptionsFactory.create(), null, ShuffleSink.ShuffleKind.GROUP_KEYS, sinkElemCoder); TestShuffleWriter shuffleWriter = new TestShuffleWriter(); int kvCount = 0; List actualSizes = new ArrayList<>(); try (Sink.SinkWriter>> shuffleSinkWriter = - shuffleSink.writer(shuffleWriter)) { + shuffleSink.writer(shuffleWriter)) { for (KV> kvs : input) { Integer key = kvs.getKey(); for (String value : kvs.getValue()) { ++kvCount; actualSizes.add(shuffleSinkWriter.add( - WindowedValue.of(KV.of(key, value), - timestamp, - Lists.newArrayList(window)))); + WindowedValue.of(KV.of(key, value), timestamp, Lists.newArrayList(window)))); } } } List records = shuffleWriter.getRecords(); - Assert.assertEquals(kvCount, records.size()); - Assert.assertEquals(shuffleWriter.getSizes(), actualSizes); + assertEquals(kvCount, records.size()); + assertEquals(shuffleWriter.getSizes(), actualSizes); // Read from shuffle with GroupingShuffleReader. BatchModeExecutionContext context = new BatchModeExecutionContext(); - GroupingShuffleReader groupingShuffleReader = - new GroupingShuffleReader<>( - PipelineOptionsFactory.create(), - null, null, null, - sourceElemCoder, - context); + GroupingShuffleReader groupingShuffleReader = new GroupingShuffleReader<>( + PipelineOptionsFactory.create(), null, null, null, sourceElemCoder, context); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(groupingShuffleReader); @@ -147,17 +143,17 @@ private void runTestReadFromShuffle( List>> actual = new ArrayList<>(); try (Reader.ReaderIterator>>> iter = - groupingShuffleReader.iterator(shuffleReader)) { + groupingShuffleReader.iterator(shuffleReader)) { Iterable prevValuesIterable = null; Iterator prevValuesIterator = null; while (iter.hasNext()) { - Assert.assertTrue(iter.hasNext()); - Assert.assertTrue(iter.hasNext()); + assertTrue(iter.hasNext()); + assertTrue(iter.hasNext()); WindowedValue>> windowedValue = iter.next(); // Verify value is in an empty windows. - Assert.assertEquals(Long.MIN_VALUE, windowedValue.getTimestamp().getMillis()); - Assert.assertEquals(0, windowedValue.getWindows().size()); + assertEquals(Long.MIN_VALUE, windowedValue.getTimestamp().getMillis()); + assertEquals(0, windowedValue.getWindows().size()); KV> elem = windowedValue.getValue(); Integer key = elem.getKey(); @@ -175,21 +171,21 @@ private void runTestReadFromShuffle( if (valuesToRead.ordinal() >= ValuesToRead.READ_ONE_VALUE.ordinal()) { while (valuesIterator.hasNext()) { - Assert.assertTrue(valuesIterator.hasNext()); - Assert.assertTrue(valuesIterator.hasNext()); - Assert.assertEquals("BatchModeExecutionContext key", key, context.getKey()); + assertTrue(valuesIterator.hasNext()); + assertTrue(valuesIterator.hasNext()); + assertEquals("BatchModeExecutionContext key", key, context.getKey()); values.add(valuesIterator.next()); if (valuesToRead == ValuesToRead.READ_ONE_VALUE) { break; } } if (valuesToRead == ValuesToRead.READ_ALL_VALUES) { - Assert.assertFalse(valuesIterator.hasNext()); - Assert.assertFalse(valuesIterator.hasNext()); + assertFalse(valuesIterator.hasNext()); + assertFalse(valuesIterator.hasNext()); try { valuesIterator.next(); - Assert.fail("Expected NoSuchElementException"); + fail("Expected NoSuchElementException"); } catch (NoSuchElementException exn) { // As expected. } @@ -203,11 +199,11 @@ private void runTestReadFromShuffle( actual.add(KV.of(key, values)); } - Assert.assertFalse(iter.hasNext()); - Assert.assertFalse(iter.hasNext()); + assertFalse(iter.hasNext()); + assertFalse(iter.hasNext()); try { iter.next(); - Assert.fail("Expected NoSuchElementException"); + fail("Expected NoSuchElementException"); } catch (NoSuchElementException exn) { // As expected. } @@ -227,8 +223,8 @@ private void runTestReadFromShuffle( } expected.add(KV.of(key, values)); } - Assert.assertEquals(expected, actual); - Assert.assertEquals(expectedSizes, observer.getActualSizes()); + assertEquals(expected, actual); + assertEquals(expectedSizes, observer.getActualSizes()); } @Test @@ -261,7 +257,7 @@ public void testReadNonEmptyShuffleDataSkippingValues() throws Exception { runTestReadFromShuffle(KVS, ValuesToRead.SKIP_VALUES); } - static byte[] fabricatePosition(int shard, byte[] key) throws Exception { + static byte[] fabricatePosition(int shard, @Nullable byte[] key) throws Exception { ByteArrayOutputStream os = new ByteArrayOutputStream(); DataOutputStream dos = new DataOutputStream(os); dos.writeInt(shard); @@ -272,7 +268,7 @@ static byte[] fabricatePosition(int shard, byte[] key) throws Exception { } @Test - public void testReadFromEmptyShuffleDataAndUpdateStopPosition() throws Exception { + public void testReadFromEmptyShuffleDataAndRequestFork() throws Exception { BatchModeExecutionContext context = new BatchModeExecutionContext(); GroupingShuffleReader groupingShuffleReader = new GroupingShuffleReader<>( PipelineOptionsFactory.create(), null, null, null, @@ -283,31 +279,34 @@ public void testReadFromEmptyShuffleDataAndUpdateStopPosition() throws Exception TestShuffleReader shuffleReader = new TestShuffleReader(); try (Reader.ReaderIterator>>> iter = groupingShuffleReader.iterator(shuffleReader)) { - // Can update the stop position, the source range spans all interval - Position proposedStopPosition = new Position(); + // Can fork, the source range spans the entire interval. + Position proposedForkPosition = new Position(); String stop = encodeBase64URLSafeString(fabricatePosition(0, null)); - proposedStopPosition.setShufflePosition(stop); + proposedForkPosition.setShufflePosition(stop); - Assert.assertEquals( - stop, - sourcePositionToCloudPosition( - iter.updateStopPosition(cloudProgressToReaderProgress( - createApproximateProgress(proposedStopPosition)))).getShufflePosition()); + Reader.ForkResult forkResult = + iter.requestFork(toForkRequest(createApproximateProgress(proposedForkPosition))); + Reader.Position acceptedForkPosition = + ((Reader.ForkResultWithPosition) forkResult).getAcceptedPosition(); + assertEquals(stop, toCloudPosition(acceptedForkPosition).getShufflePosition()); - // Cannot update stop position to a position >= the current stop position + // Cannot fork at a position >= the current stop position stop = encodeBase64URLSafeString(fabricatePosition(1, null)); - proposedStopPosition.setShufflePosition(stop); + proposedForkPosition.setShufflePosition(stop); - Assert.assertEquals( - null, - iter.updateStopPosition( - cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); + try { + iter.requestFork(toForkRequest(createApproximateProgress(proposedForkPosition))); + fail("IllegalArgumentException expected"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), Matchers.containsString( + "Fork requested at a shuffle position beyond the end of the current range")); + } } } @Test - public void testReadFromShuffleDataAndFailToUpdateStopPosition() throws Exception { + public void testReadFromShuffleDataAndFailToFork() throws Exception { BatchModeExecutionContext context = new BatchModeExecutionContext(); final int kFirstShard = 0; @@ -331,16 +330,14 @@ public void testReadFromShuffleDataAndFailToUpdateStopPosition() throws Exceptio try (Reader.ReaderIterator>>> iter = groupingShuffleReader.iterator(shuffleReader)) { - Position proposedStopPosition = new Position(); - proposedStopPosition.setShufflePosition( - encodeBase64URLSafeString(fabricatePosition(kNumRecords + 1, null))); - - // Cannot update the stop position since the value provided is - // past the current stop position. - Assert.assertEquals( - null, - iter.updateStopPosition( - cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); + // Cannot fork since the value provided is past the current stop position. + try { + iter.requestFork(forkRequestAtPosition(makeShufflePosition(kNumRecords + 1, null))); + fail("IllegalArgumentException expected"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), Matchers.containsString( + "Fork requested at a shuffle position beyond the end of the current range")); + } int i = 0; for (; iter.hasNext(); ++i) { @@ -348,38 +345,31 @@ public void testReadFromShuffleDataAndFailToUpdateStopPosition() throws Exceptio if (i == 0) { // First record byte[] key = CoderUtils.encodeToByteArray(BigEndianIntegerCoder.of(), i); - proposedStopPosition.setShufflePosition( - encodeBase64URLSafeString(fabricatePosition(kFirstShard, key))); - // Cannot update stop position since it is identical with - // the position of the record that was just returned. - Assert.assertEquals( - null, - iter.updateStopPosition( - cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); - - proposedStopPosition.setShufflePosition( - encodeBase64URLSafeString(fabricatePosition(kFirstShard, null))); - // Cannot update stop position since it comes before current position - Assert.assertEquals( - null, - iter.updateStopPosition( - cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); + // Cannot fork since the fork position is identical with the position of the record + // that was just returned. + assertNull( + iter.requestFork(forkRequestAtPosition(makeShufflePosition(kFirstShard, key)))); + + // Cannot fork since the requested fork position comes before current position + assertNull( + iter.requestFork(forkRequestAtPosition(makeShufflePosition(kFirstShard, null)))); } } - Assert.assertEquals(kNumRecords, i); - - proposedStopPosition.setShufflePosition( - encodeBase64URLSafeString(fabricatePosition(kFirstShard, null))); - // Cannot update stop position since all input was consumed. - Assert.assertEquals( - null, - iter.updateStopPosition( - cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); + assertEquals(kNumRecords, i); + + // Cannot fork since all input was consumed. + assertNull( + iter.requestFork(forkRequestAtPosition(makeShufflePosition(kFirstShard, null)))); } } + private Position makeShufflePosition(int shard, byte[] key) throws Exception { + return new Position().setShufflePosition( + encodeBase64URLSafeString(fabricatePosition(shard, key))); + } + @Test - public void testReadFromShuffleAndUpdateStopPosition() throws Exception { + public void testReadFromShuffleAndFork() throws Exception { BatchModeExecutionContext context = new BatchModeExecutionContext(); GroupingShuffleReader groupingShuffleReader = new GroupingShuffleReader<>( PipelineOptionsFactory.create(), null, null, null, @@ -414,46 +404,45 @@ public void testReadFromShuffleAndUpdateStopPosition() throws Exception { int i = 0; try (Reader.ReaderIterator>>> iter = groupingShuffleReader.iterator(shuffleReader)) { - Position proposedStopPosition = new Position(); - - Assert.assertNull(iter.updateStopPosition( - cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); - - // Stop at the shard boundary - String stop = encodeBase64URLSafeString(fabricatePosition(kSecondShard, null)); - proposedStopPosition.setShufflePosition(stop); + assertNull(iter.requestFork(forkRequestAtPosition(new Position()))); - Assert.assertEquals( - stop, - sourcePositionToCloudPosition( - iter.updateStopPosition(cloudProgressToReaderProgress( - createApproximateProgress(proposedStopPosition)))).getShufflePosition()); + // Fork at the shard boundary + Reader.ForkResult forkResult = + iter.requestFork(forkRequestAtPosition(makeShufflePosition(kSecondShard, null))); + assertEquals( + encodeBase64URLSafeString(fabricatePosition(kSecondShard, null)), + positionFromForkResult(forkResult).getShufflePosition()); while (iter.hasNext()) { - Assert.assertTrue(iter.hasNext()); - Assert.assertTrue(iter.hasNext()); + // iter.hasNext() is supposed to be side-effect-free and give the same result if called + // repeatedly. Test that this is indeed the case. + assertTrue(iter.hasNext()); + assertTrue(iter.hasNext()); KV> elem = iter.next().getValue(); int key = elem.getKey(); - Assert.assertEquals(key, i); + assertEquals(key, i); Iterable valuesIterable = elem.getValue(); Iterator valuesIterator = valuesIterable.iterator(); int j = 0; while (valuesIterator.hasNext()) { - Assert.assertTrue(valuesIterator.hasNext()); - Assert.assertTrue(valuesIterator.hasNext()); + assertTrue(valuesIterator.hasNext()); + assertTrue(valuesIterator.hasNext()); int value = valuesIterator.next(); - Assert.assertEquals(value, i); + assertEquals(value, i); ++j; } - Assert.assertEquals(j, 1); + assertFalse(valuesIterator.hasNext()); + assertFalse(valuesIterator.hasNext()); + assertEquals(j, 1); ++i; } + assertFalse(iter.hasNext()); } - Assert.assertEquals(i, kNumRecords); + assertEquals(i, kNumRecords); } @Test @@ -484,34 +473,32 @@ public void testGetApproximateProgress() throws Exception { groupingShuffleReader.iterator(shuffleReader)) { Integer i = 0; while (readerIterator.hasNext()) { - Assert.assertTrue(readerIterator.hasNext()); - ApproximateProgress progress = sourceProgressToCloudProgress(readerIterator.getProgress()); - Assert.assertNotNull(progress.getPosition().getShufflePosition()); + assertTrue(readerIterator.hasNext()); + ApproximateProgress progress = readerProgressToCloudProgress(readerIterator.getProgress()); + assertNotNull(progress.getPosition().getShufflePosition()); // Compare returned position with the expected position. - Assert.assertEquals( + assertEquals( ByteArrayShufflePosition.of(positionsList.get(i)).encodeBase64(), progress.getPosition().getShufflePosition()); WindowedValue>> elem = readerIterator.next(); - Assert.assertEquals(i, elem.getValue().getKey()); + assertEquals(i, elem.getValue().getKey()); i++; } - Assert.assertFalse(readerIterator.hasNext()); + assertFalse(readerIterator.hasNext()); - // Cannot update stop position since all input was consumed. - Position proposedStopPosition = new Position(); + // Cannot fork since all input was consumed. + Position proposedForkPosition = new Position(); String stop = encodeBase64URLSafeString(fabricatePosition(0, null)); - proposedStopPosition.setShufflePosition(stop); - Assert.assertEquals( - null, - readerIterator.updateStopPosition( - cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); + proposedForkPosition.setShufflePosition(stop); + assertNull( + readerIterator.requestFork( + toForkRequest(createApproximateProgress(proposedForkPosition)))); } } - private ApproximateProgress createApproximateProgress( - com.google.api.services.dataflow.model.Position position) { + private ApproximateProgress createApproximateProgress(Position position) { return new ApproximateProgress().setPosition(position); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java index 0068e5e275172..bd6f02226a60e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java @@ -16,20 +16,28 @@ package com.google.cloud.dataflow.sdk.runners.worker; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.approximateProgressAtIndex; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.forkRequestAtIndex; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.positionAtIndex; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.positionFromForkResult; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.readerProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.toForkRequest; import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray; import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import com.google.api.services.dataflow.model.ApproximateProgress; -import com.google.api.services.dataflow.model.Position; import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; import com.google.cloud.dataflow.sdk.util.common.worker.Reader; -import org.junit.Assert; +import org.hamcrest.Matchers; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -62,14 +70,13 @@ void runTestReadInMemory(List elements, Long startIndex, Long endIndex, List actualElements = new ArrayList<>(); try (Reader.ReaderIterator iterator = inMemoryReader.iterator()) { for (long i = inMemoryReader.startIndex; iterator.hasNext(); i++) { - Assert.assertEquals( - new ApproximateProgress().setPosition(makeIndexPosition(i)), - sourceProgressToCloudProgress(iterator.getProgress())); + assertEquals( + approximateProgressAtIndex(i), readerProgressToCloudProgress(iterator.getProgress())); actualElements.add(iterator.next()); } } - Assert.assertEquals(expectedElements, actualElements); - Assert.assertEquals(expectedSizes, observer.getActualSizes()); + assertEquals(expectedElements, actualElements); + assertEquals(expectedSizes, observer.getActualSizes()); } @Test @@ -129,7 +136,7 @@ public void testReadNoElementsFromStartToEndEmptyRange() throws Exception { } @Test - public void testUpdatePosition() throws Exception { + public void testFork() throws Exception { List elements = Arrays.asList(33, 44, 55, 66, 77, 88); final long start = 1L; final long stop = 3L; @@ -139,52 +146,44 @@ public void testUpdatePosition() throws Exception { InMemoryReader inMemoryReader = new InMemoryReader<>(encodedElements(elements, coder), start, end, coder); - // Illegal proposed stop position. + // Illegal proposed fork position. try (Reader.ReaderIterator iterator = inMemoryReader.iterator()) { - Assert.assertNull( - iterator.updateStopPosition(cloudProgressToReaderProgress(new ApproximateProgress()))); - Assert.assertNull(iterator.updateStopPosition(cloudProgressToReaderProgress( - new ApproximateProgress().setPosition(makeIndexPosition(null))))); + assertNull(iterator.requestFork(toForkRequest(new ApproximateProgress()))); + assertNull(iterator.requestFork(forkRequestAtIndex(null))); } // Successful update. try (InMemoryReader.InMemoryReaderIterator iterator = (InMemoryReader.InMemoryReaderIterator) inMemoryReader.iterator()) { - Assert.assertEquals( - makeIndexPosition(stop), - sourcePositionToCloudPosition(iterator.updateStopPosition(cloudProgressToReaderProgress( - new ApproximateProgress().setPosition(makeIndexPosition(stop)))))); - Assert.assertEquals(stop, iterator.endPosition); - Assert.assertEquals(44, iterator.next().intValue()); - Assert.assertEquals(55, iterator.next().intValue()); - Assert.assertFalse(iterator.hasNext()); + Reader.ForkResult forkResult = iterator.requestFork(forkRequestAtIndex(stop)); + assertEquals(positionAtIndex(stop), positionFromForkResult(forkResult)); + assertEquals(stop, iterator.endPosition); + assertEquals(44, iterator.next().intValue()); + assertEquals(55, iterator.next().intValue()); + assertFalse(iterator.hasNext()); } - // Proposed stop position is before the current position, no update. + // Proposed fork position is before the current position, no update. try (InMemoryReader.InMemoryReaderIterator iterator = (InMemoryReader.InMemoryReaderIterator) inMemoryReader.iterator()) { - Assert.assertEquals(44, iterator.next().intValue()); - Assert.assertEquals(55, iterator.next().intValue()); - Assert.assertNull(iterator.updateStopPosition(cloudProgressToReaderProgress( - new ApproximateProgress().setPosition(makeIndexPosition(stop))))); - Assert.assertEquals((int) end, iterator.endPosition); - Assert.assertTrue(iterator.hasNext()); + assertEquals(44, iterator.next().intValue()); + assertEquals(55, iterator.next().intValue()); + assertNull(iterator.requestFork(forkRequestAtIndex(stop))); + assertEquals((int) end, iterator.endPosition); + assertTrue(iterator.hasNext()); } - // Proposed stop position is after the current stop (end) position, no update. + // Proposed fork position is after the current stop (end) position, no update. try (InMemoryReader.InMemoryReaderIterator iterator = (InMemoryReader.InMemoryReaderIterator) inMemoryReader.iterator()) { - Assert.assertNull(iterator.updateStopPosition(cloudProgressToReaderProgress( - new ApproximateProgress().setPosition(makeIndexPosition(end + 1))))); - Assert.assertEquals((int) end, iterator.endPosition); - } - } - - private Position makeIndexPosition(Long index) { - Position position = new Position(); - if (index != null) { - position.setRecordIndex(index); + try { + iterator.requestFork(forkRequestAtIndex(end + 1)); + fail("IllegalArgumentException expected"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), Matchers.containsString( + "Fork requested at an index beyond the end of the current range")); + } + assertEquals((int) end, iterator.endPosition); } - return position; } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderTestUtils.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderTestUtils.java new file mode 100644 index 0000000000000..96f8b8b432cc5 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderTestUtils.java @@ -0,0 +1,72 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.readerProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.toCloudPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.toForkRequest; + +import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.Position; +import com.google.cloud.dataflow.sdk.util.common.worker.Reader; + +import javax.annotation.Nullable; + +/** + * Helpers for testing {@code Reader} and related classes, especially + * {@link Reader.ReaderIterator#getProgress} and {@link Reader.ReaderIterator#requestFork}. + */ +public class ReaderTestUtils { + public static Position positionAtIndex(@Nullable Long index) { + return new Position().setRecordIndex(index); + } + + public static Position positionAtByteOffset(@Nullable Long byteOffset) { + return new Position().setByteOffset(byteOffset); + } + + public static ApproximateProgress approximateProgressAtPosition(@Nullable Position position) { + return new ApproximateProgress().setPosition(position); + } + + public static ApproximateProgress approximateProgressAtIndex(@Nullable Long index) { + return approximateProgressAtPosition(positionAtIndex(index)); + } + + public static ApproximateProgress approximateProgressAtByteOffset(@Nullable Long byteOffset) { + return approximateProgressAtPosition(positionAtByteOffset(byteOffset)); + } + + public static Reader.ForkRequest forkRequestAtPosition(@Nullable Position position) { + return toForkRequest(approximateProgressAtPosition(position)); + } + + public static Reader.ForkRequest forkRequestAtIndex(@Nullable Long index) { + return toForkRequest(approximateProgressAtIndex(index)); + } + + public static Reader.ForkRequest forkRequestAtByteOffset(@Nullable Long byteOffset) { + return toForkRequest(approximateProgressAtByteOffset(byteOffset)); + } + + public static Position positionFromForkResult(Reader.ForkResult forkResult) { + return toCloudPosition(((Reader.ForkResultWithPosition) forkResult).getAcceptedPosition()); + } + + public static Position positionFromProgress(Reader.Progress progress) { + return readerProgressToCloudProgress(progress).getPosition(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java index 95981a7eecccf..52c9c1cfd2866 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java @@ -16,13 +16,22 @@ package com.google.cloud.dataflow.sdk.runners.worker; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.forkRequestAtByteOffset; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.forkRequestAtPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.positionFromForkResult; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.readerProgressToCloudProgress; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.greaterThan; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.Position; import com.google.cloud.dataflow.sdk.TestUtils; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.TextualIntegerCoder; @@ -33,7 +42,7 @@ import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; -import org.junit.Assert; +import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -61,6 +70,7 @@ public class TextReaderTest { static { long sumLen = 0L; + for (String s : fileContent) { sumLen += s.length(); } @@ -83,11 +93,10 @@ private File initTestFile() throws IOException { @Test public void testReadEmptyFile() throws Exception { - TextReader textReader = - new TextReader<>(tmpFolder.newFile().getPath(), true, null, null, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFolder.newFile().getPath(), true, null, + null, StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); try (Reader.ReaderIterator iterator = textReader.iterator()) { - Assert.assertFalse(iterator.hasNext()); + assertFalse(iterator.hasNext()); } } @@ -136,66 +145,62 @@ public void testStartPosition() throws Exception { File tmpFile = initTestFile(); { - TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, 11L, null, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), false, 11L, null, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); try (Reader.ReaderIterator iterator = textReader.iterator()) { - Assert.assertEquals("Second line\r\n", iterator.next()); - Assert.assertEquals("Third line", iterator.next()); - Assert.assertFalse(iterator.hasNext()); + assertEquals("Second line\r\n", iterator.next()); + assertEquals("Third line", iterator.next()); + assertFalse(iterator.hasNext()); // The first '1' in the array represents the reading of '\n' between first and // second line, to confirm that we are reading from the beginning of a record. - Assert.assertEquals(Arrays.asList(1, 13, 10), observer.getActualSizes()); + assertEquals(Arrays.asList(1, 13, 10), observer.getActualSizes()); } } { - TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, 20L, null, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), false, 20L, null, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); try (Reader.ReaderIterator iterator = textReader.iterator()) { - Assert.assertEquals("Third line", iterator.next()); - Assert.assertFalse(iterator.hasNext()); + assertEquals("Third line", iterator.next()); + assertFalse(iterator.hasNext()); // The first '5' in the array represents the reading of a portion of the second // line, which had to be read to find the beginning of the third line. - Assert.assertEquals(Arrays.asList(5, 10), observer.getActualSizes()); + assertEquals(Arrays.asList(5, 10), observer.getActualSizes()); } } { - TextReader textReader = - new TextReader<>(tmpFile.getPath(), true, 0L, 20L, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), true, 0L, 20L, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); try (Reader.ReaderIterator iterator = textReader.iterator()) { - Assert.assertEquals("First line", iterator.next()); - Assert.assertEquals("Second line", iterator.next()); - Assert.assertFalse(iterator.hasNext()); - Assert.assertEquals(Arrays.asList(11, 13), observer.getActualSizes()); + assertEquals("First line", iterator.next()); + assertEquals("Second line", iterator.next()); + assertFalse(iterator.hasNext()); + assertEquals(Arrays.asList(11, 13), observer.getActualSizes()); } } { - TextReader textReader = - new TextReader<>(tmpFile.getPath(), true, 1L, 20L, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), true, 1L, 20L, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); try (Reader.ReaderIterator iterator = textReader.iterator()) { - Assert.assertEquals("Second line", iterator.next()); - Assert.assertFalse(iterator.hasNext()); + assertEquals("Second line", iterator.next()); + assertFalse(iterator.hasNext()); // The first '11' in the array represents the reading of the entire first // line, which had to be read to find the beginning of the second line. - Assert.assertEquals(Arrays.asList(11, 13), observer.getActualSizes()); + assertEquals(Arrays.asList(11, 13), observer.getActualSizes()); } } } @@ -214,34 +219,32 @@ public void testUtf8Handling() throws Exception { // 3L is after the first line if counting codepoints, but within // the first line if counting chars. So correct behavior is to return // just one line, since offsets are in chars, not codepoints. - TextReader textReader = - new TextReader<>(tmpFile.getPath(), true, 0L, 3L, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), true, 0L, 3L, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); try (Reader.ReaderIterator iterator = textReader.iterator()) { - Assert.assertArrayEquals("€".getBytes("UTF-8"), iterator.next().getBytes("UTF-8")); - Assert.assertFalse(iterator.hasNext()); - Assert.assertEquals(Arrays.asList(4), observer.getActualSizes()); + assertArrayEquals("€".getBytes("UTF-8"), iterator.next().getBytes("UTF-8")); + assertFalse(iterator.hasNext()); + assertEquals(Arrays.asList(4), observer.getActualSizes()); } } { // Starting location is mid-way into a codepoint. // Ensures we don't fail when skipping over an incomplete codepoint. - TextReader textReader = - new TextReader<>(tmpFile.getPath(), true, 2L, null, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), true, 2L, null, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); try (Reader.ReaderIterator iterator = textReader.iterator()) { - Assert.assertArrayEquals("¢".getBytes("UTF-8"), iterator.next().getBytes("UTF-8")); - Assert.assertFalse(iterator.hasNext()); + assertArrayEquals("¢".getBytes("UTF-8"), iterator.next().getBytes("UTF-8")); + assertFalse(iterator.hasNext()); // The first '3' in the array represents the reading of a portion of the first // line, which had to be read to find the beginning of the second line. - Assert.assertEquals(Arrays.asList(3, 3), observer.getActualSizes()); + assertEquals(Arrays.asList(3, 3), observer.getActualSizes()); } } } @@ -258,9 +261,8 @@ private void testNewlineHandling(String separator, boolean stripNewlines) throws } writer.close(); - TextReader textReader = - new TextReader<>(tmpFile.getPath(), stripNewlines, null, null, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), stripNewlines, null, null, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -272,16 +274,16 @@ private void testNewlineHandling(String separator, boolean stripNewlines) throws } if (stripNewlines) { - Assert.assertEquals(expected, actual); + assertEquals(expected, actual); } else { List unstripped = new LinkedList<>(); for (String s : expected) { unstripped.add(s + separator); } - Assert.assertEquals(unstripped, actual); + assertEquals(unstripped, actual); } - Assert.assertEquals(expectedSizes, observer.getActualSizes()); + assertEquals(expectedSizes, observer.getActualSizes()); } private void testStringPayload(String[] lines, String separator, boolean stripNewlines) @@ -296,19 +298,15 @@ private void testStringPayload(String[] lines, String separator, boolean stripNe } writer.close(); - TextReader textReader = - new TextReader<>(tmpFile.getPath(), stripNewlines, null, null, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); - ExecutorTestUtils.TestReaderObserver observer = - new ExecutorTestUtils.TestReaderObserver(textReader); - + TextReader textReader = new TextReader<>(tmpFile.getPath(), stripNewlines, null, null, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); List actual = new ArrayList<>(); try (Reader.ReaderIterator iterator = textReader.iterator()) { while (iterator.hasNext()) { actual.add(iterator.next()); } } - Assert.assertEquals(expected, actual); + assertEquals(expected, actual); } @Test @@ -327,9 +325,8 @@ public void testCloneIteratorWithEndPositionAndFinalBytesInBuffer() throws Excep writer.close(); Long fileSize = tmpFile.length(); - TextReader textReader = - new TextReader<>(tmpFile.getPath(), stripNewlines, null, fileSize, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), stripNewlines, null, + fileSize, StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); List actual = new ArrayList<>(); Reader.ReaderIterator iterator = textReader.iterator(); @@ -337,7 +334,7 @@ public void testCloneIteratorWithEndPositionAndFinalBytesInBuffer() throws Excep actual.add(iterator.next()); iterator = iterator.copy(); } - Assert.assertEquals(expected, actual); + assertEquals(expected, actual); } @Test @@ -354,9 +351,8 @@ public void testNonStringCoders() throws Exception { } writer.close(); - TextReader textReader = - new TextReader<>(tmpFile.getPath(), true, null, null, TextualIntegerCoder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), true, null, null, + TextualIntegerCoder.of(), TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); @@ -367,30 +363,29 @@ public void testNonStringCoders() throws Exception { } } - Assert.assertEquals(expected, actual); - Assert.assertEquals(expectedSizes, observer.getActualSizes()); + assertEquals(expected, actual); + assertEquals(expectedSizes, observer.getActualSizes()); } @Test public void testGetApproximatePosition() throws Exception { File tmpFile = initTestFile(); - TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, 0L, null, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), false, 0L, null, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); try (Reader.ReaderIterator iterator = textReader.iterator()) { - ApproximateProgress progress = sourceProgressToCloudProgress(iterator.getProgress()); - Assert.assertEquals(0L, progress.getPosition().getByteOffset().longValue()); + ApproximateProgress progress = readerProgressToCloudProgress(iterator.getProgress()); + assertEquals(0L, progress.getPosition().getByteOffset().longValue()); iterator.next(); - progress = sourceProgressToCloudProgress(iterator.getProgress()); - Assert.assertEquals(11L, progress.getPosition().getByteOffset().longValue()); + progress = readerProgressToCloudProgress(iterator.getProgress()); + assertEquals(11L, progress.getPosition().getByteOffset().longValue()); iterator.next(); - progress = sourceProgressToCloudProgress(iterator.getProgress()); - Assert.assertEquals(24L, progress.getPosition().getByteOffset().longValue()); + progress = readerProgressToCloudProgress(iterator.getProgress()); + assertEquals(24L, progress.getPosition().getByteOffset().longValue()); iterator.next(); - progress = sourceProgressToCloudProgress(iterator.getProgress()); - Assert.assertEquals(34L, progress.getPosition().getByteOffset().longValue()); - Assert.assertFalse(iterator.hasNext()); + progress = readerProgressToCloudProgress(iterator.getProgress()); + assertEquals(34L, progress.getPosition().getByteOffset().longValue()); + assertFalse(iterator.hasNext()); } } @@ -400,49 +395,36 @@ public void testUpdateStopPosition() throws Exception { final long stop = 14L; // in the middle of the second line File tmpFile = initTestFile(); - com.google.api.services.dataflow.model.Position proposedStopPosition = - new com.google.api.services.dataflow.model.Position(); - // Illegal proposed stop position, no update. { - TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, null, null, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); - ExecutorTestUtils.TestReaderObserver observer = - new ExecutorTestUtils.TestReaderObserver(textReader); + TextReader textReader = new TextReader<>(tmpFile.getPath(), false, null, null, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); try (TextReader.TextFileIterator iterator = (TextReader.TextFileIterator) textReader.iterator()) { - Assert.assertNull(iterator.updateStopPosition( - cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); + assertNull(iterator.requestFork(forkRequestAtPosition(new Position()))); } } - proposedStopPosition.setByteOffset(stop); - // Successful update. { - TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, null, null, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), false, null, null, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); try (TextReader.TextFileIterator iterator = (TextReader.TextFileIterator) textReader.iterator()) { - Assert.assertNull(iterator.getEndOffset()); - Assert.assertEquals( - stop, - sourcePositionToCloudPosition( - iterator.updateStopPosition( - cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))) - .getByteOffset() - .longValue()); - Assert.assertEquals(stop, iterator.getEndOffset().longValue()); - Assert.assertEquals(fileContent[0], iterator.next()); - Assert.assertEquals(fileContent[1], iterator.next()); - Assert.assertFalse(iterator.hasNext()); - Assert.assertEquals( + assertNull(iterator.getEndOffset()); + assertEquals( + Long.valueOf(stop), + positionFromForkResult(iterator.requestFork(forkRequestAtByteOffset(stop))) + .getByteOffset()); + assertEquals(stop, iterator.getEndOffset().longValue()); + assertEquals(fileContent[0], iterator.next()); + assertEquals(fileContent[1], iterator.next()); + assertFalse(iterator.hasNext()); + assertEquals( Arrays.asList(fileContent[0].length(), fileContent[1].length()), observer.getActualSizes()); } @@ -450,25 +432,23 @@ public void testUpdateStopPosition() throws Exception { // Proposed stop position is before the current position, no update. { - TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, null, null, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), false, null, null, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); try (TextReader.TextFileIterator iterator = (TextReader.TextFileIterator) textReader.iterator()) { - Assert.assertEquals(fileContent[0], iterator.next()); - Assert.assertEquals(fileContent[1], iterator.next()); - Assert.assertThat( - sourceProgressToCloudProgress(iterator.getProgress()).getPosition().getByteOffset(), + assertEquals(fileContent[0], iterator.next()); + assertEquals(fileContent[1], iterator.next()); + assertThat( + readerProgressToCloudProgress(iterator.getProgress()).getPosition().getByteOffset(), greaterThan(stop)); - Assert.assertNull(iterator.updateStopPosition( - cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); - Assert.assertNull(iterator.getEndOffset()); - Assert.assertTrue(iterator.hasNext()); - Assert.assertEquals(fileContent[2], iterator.next()); - Assert.assertEquals( + assertNull(iterator.requestFork(forkRequestAtByteOffset(stop))); + assertNull(iterator.getEndOffset()); + assertTrue(iterator.hasNext()); + assertEquals(fileContent[2], iterator.next()); + assertEquals( Arrays.asList( fileContent[0].length(), fileContent[1].length(), fileContent[2].length()), observer.getActualSizes()); @@ -477,20 +457,25 @@ public void testUpdateStopPosition() throws Exception { // Proposed stop position is after the current stop (end) position, no update. { - TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, null, end, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), false, null, end, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); ExecutorTestUtils.TestReaderObserver observer = new ExecutorTestUtils.TestReaderObserver(textReader); try (TextReader.TextFileIterator iterator = (TextReader.TextFileIterator) textReader.iterator()) { - Assert.assertEquals(fileContent[0], iterator.next()); - Assert.assertNull(iterator.updateStopPosition( - cloudProgressToReaderProgress(createApproximateProgress(proposedStopPosition)))); - Assert.assertEquals(end, iterator.getEndOffset().longValue()); - Assert.assertFalse(iterator.hasNext()); - Assert.assertEquals(Arrays.asList(fileContent[0].length()), observer.getActualSizes()); + assertEquals(fileContent[0], iterator.next()); + try { + iterator.requestFork(forkRequestAtByteOffset(stop)); + fail("IllegalArgumentException expected"); + } catch (IllegalArgumentException e) { + assertThat(e.getMessage(), Matchers.containsString( + "Fork requested at an offset beyond the end of the current range")); + } + + assertEquals(end, iterator.getEndOffset().longValue()); + assertFalse(iterator.hasNext()); + assertEquals(Arrays.asList(fileContent[0].length()), observer.getActualSizes()); } } } @@ -523,56 +508,48 @@ private void stopPositionTestInternal( StringBuilder accumulatedRead = new StringBuilder(); // Read from source without split attempts. - TextReader textReader = - new TextReader<>(tmpFile.getPath(), false, startOffset, endOffset, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + TextReader textReader = new TextReader<>(tmpFile.getPath(), false, startOffset, + endOffset, StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); try (TextReader.TextFileIterator iterator = (TextReader.TextFileIterator) textReader.iterator()) { while (iterator.hasNext()) { - accumulatedRead.append((String) iterator.next()); + accumulatedRead.append(iterator.next()); } readWithoutSplit = accumulatedRead.toString(); } // Read the first half of the split. - textReader = - new TextReader<>(tmpFile.getPath(), false, startOffset, stopOffset, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + textReader = new TextReader<>(tmpFile.getPath(), false, startOffset, stopOffset, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); accumulatedRead = new StringBuilder(); try (TextReader.TextFileIterator iterator = (TextReader.TextFileIterator) textReader.iterator()) { while (iterator.hasNext()) { - accumulatedRead.append((String) iterator.next()); + accumulatedRead.append(iterator.next()); } readWithSplit1 = accumulatedRead.toString(); } // Read the second half of the split. - textReader = - new TextReader<>(tmpFile.getPath(), false, stopOffset, endOffset, StringUtf8Coder.of(), - TextIO.CompressionType.UNCOMPRESSED); + textReader = new TextReader<>(tmpFile.getPath(), false, stopOffset, endOffset, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); accumulatedRead = new StringBuilder(); try (TextReader.TextFileIterator iterator = (TextReader.TextFileIterator) textReader.iterator()) { while (iterator.hasNext()) { - accumulatedRead.append((String) iterator.next()); + accumulatedRead.append(iterator.next()); } readWithSplit2 = accumulatedRead.toString(); } - Assert.assertEquals(readWithoutSplit, readWithSplit1 + readWithSplit2); - } - - private ApproximateProgress createApproximateProgress( - com.google.api.services.dataflow.model.Position position) { - return new ApproximateProgress().setPosition(position); + assertEquals(readWithoutSplit, readWithSplit1 + readWithSplit2); } - private OutputStream getOutputStreamForCompressionType(OutputStream stream, - CompressionType compressionType) throws IOException { + private OutputStream getOutputStreamForCompressionType( + OutputStream stream, CompressionType compressionType) throws IOException { switch (compressionType) { case GZIP: return new GZIPOutputStream(stream); @@ -582,13 +559,13 @@ private OutputStream getOutputStreamForCompressionType(OutputStream stream, case AUTO: return stream; default: - Assert.fail("Unrecognized stream type"); + fail("Unrecognized stream type"); } return stream; } - private File createFileWithCompressionType(String[] lines, String filename, - CompressionType compressionType) throws IOException { + private File createFileWithCompressionType( + String[] lines, String filename, CompressionType compressionType) throws IOException { File tmpFile = tmpFolder.newFile(filename); PrintStream writer = new PrintStream( getOutputStreamForCompressionType(new FileOutputStream(tmpFile), compressionType)); @@ -601,7 +578,7 @@ private File createFileWithCompressionType(String[] lines, String filename, private void testCompressionTypeHelper(String[] lines, String filename, CompressionType outputCompressionType, CompressionType inputCompressionType) - throws IOException { + throws IOException { File tmpFile = createFileWithCompressionType(lines, filename, outputCompressionType); List expected = new ArrayList<>(); @@ -609,9 +586,8 @@ private void testCompressionTypeHelper(String[] lines, String filename, expected.add(line); } - TextReader textReader = - new TextReader<>(tmpFile.getPath(), true, null, null, StringUtf8Coder.of(), - inputCompressionType); + TextReader textReader = new TextReader<>( + tmpFile.getPath(), true, null, null, StringUtf8Coder.of(), inputCompressionType); List actual = new ArrayList<>(); try (Reader.ReaderIterator iterator = textReader.iterator()) { @@ -619,7 +595,7 @@ private void testCompressionTypeHelper(String[] lines, String filename, actual.add(iterator.next()); } } - Assert.assertEquals(expected, actual); + assertEquals(expected, actual); tmpFile.delete(); } @@ -629,22 +605,20 @@ public void testCompressionTypeOneFile() throws IOException { // test AUTO compression type with different extensions testCompressionTypeHelper(contents, "test.gz", CompressionType.GZIP, CompressionType.AUTO); testCompressionTypeHelper(contents, "test.bz2", CompressionType.BZIP2, CompressionType.AUTO); - testCompressionTypeHelper(contents, "test.txt", CompressionType.UNCOMPRESSED, - CompressionType.AUTO); - testCompressionTypeHelper(contents, "test", CompressionType.UNCOMPRESSED, - CompressionType.AUTO); + testCompressionTypeHelper( + contents, "test.txt", CompressionType.UNCOMPRESSED, CompressionType.AUTO); + testCompressionTypeHelper(contents, "test", CompressionType.UNCOMPRESSED, CompressionType.AUTO); // test GZIP, BZIP2, and UNCOMPRESSED testCompressionTypeHelper(contents, "test.txt", CompressionType.GZIP, CompressionType.GZIP); testCompressionTypeHelper(contents, "test.txt", CompressionType.BZIP2, CompressionType.BZIP2); - testCompressionTypeHelper(contents, "test.gz", CompressionType.UNCOMPRESSED, - CompressionType.UNCOMPRESSED); + testCompressionTypeHelper( + contents, "test.gz", CompressionType.UNCOMPRESSED, CompressionType.UNCOMPRESSED); } @Test public void testCompressionTypeFileGlob() throws IOException { String[][] contents = { - {"Miserable pigeon", "Vulnerable sparrow", "Brazen crow"}, - {"Timid osprey", "Lazy vulture"}, + {"Miserable pigeon", "Vulnerable sparrow", "Brazen crow"}, {"Timid osprey", "Lazy vulture"}, {"Erratic finch", "Impressible parakeet"}, }; File[] files = { @@ -671,7 +645,7 @@ public void testCompressionTypeFileGlob() throws IOException { actual.add(iterator.next()); } } - Assert.assertThat(actual, containsInAnyOrder(expected.toArray())); + assertThat(actual, containsInAnyOrder(expected.toArray())); for (File file : files) { file.delete(); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java index 07944893ffebd..f13d0e02606c5 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java @@ -16,10 +16,14 @@ package com.google.cloud.dataflow.sdk.util.common.worker; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.approximateProgressAtIndex; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.forkRequestAtIndex; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.positionAtIndex; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.positionFromForkResult; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.positionFromProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.forkRequestToApproximateProgress; import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; import com.google.api.services.dataflow.model.ApproximateProgress; @@ -96,10 +100,10 @@ public Reader.Progress getProgress() { } @Override - public Reader.Position proposeStopPosition(Reader.Progress proposedStopPosition) { + public Reader.ForkResult requestFork(Reader.ForkRequest forkRequest) { // Fakes the return with the same position as proposed. - return cloudPositionToReaderPosition( - sourceProgressToCloudProgress(proposedStopPosition).getPosition()); + return new Reader.ForkResultWithPosition(cloudPositionToReaderPosition( + forkRequestToApproximateProgress(forkRequest).getPosition())); } public void setProgress(ApproximateProgress progress) { @@ -194,7 +198,7 @@ public void testGetReadOperation() throws Exception { new MapTaskExecutor(new ArrayList(), counterSet, stateSampler); try { - ReadOperation readOperation = executor.getReadOperation(); + executor.getReadOperation(); Assert.fail("Expected IllegalStateException."); } catch (IllegalStateException e) { // Exception expected @@ -208,7 +212,7 @@ public void testGetReadOperation() throws Exception { executor = new MapTaskExecutor(operations, counterSet, stateSampler); try { - ReadOperation readOperation = executor.getReadOperation(); + executor.getReadOperation(); Assert.fail("Expected IllegalStateException."); } catch (IllegalStateException e) { // Exception expected @@ -235,22 +239,11 @@ public void testGetProgressAndRequestSplit() throws Exception { MapTaskExecutor executor = new MapTaskExecutor(Arrays.asList(new Operation[] {operation}), counterSet, stateSampler); - operation.setProgress(new ApproximateProgress().setPosition(makePosition(1L))); + operation.setProgress(approximateProgressAtIndex(1L)); + Assert.assertEquals(positionAtIndex(1L), positionFromProgress(executor.getWorkerProgress())); Assert.assertEquals( - makePosition(1L), - sourceProgressToCloudProgress(executor.getWorkerProgress()).getPosition()); - Assert.assertEquals( - makePosition(1L), - sourcePositionToCloudPosition(executor.proposeStopPosition(cloudProgressToReaderProgress( - new ApproximateProgress().setPosition(makePosition(1L)))))); + positionAtIndex(1L), positionFromForkResult(executor.requestFork(forkRequestAtIndex(1L)))); executor.close(); } - - private com.google.api.services.dataflow.model.Position makePosition(long index) { - com.google.api.services.dataflow.model.Position position = - new com.google.api.services.dataflow.model.Position(); - position.setRecordIndex(index); - return position; - } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java index 8d754bd0d4368..9a778bdef25c6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java @@ -16,14 +16,15 @@ package com.google.cloud.dataflow.sdk.util.common.worker; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.forkRequestAtIndex; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.positionAtIndex; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourcePositionToCloudPosition; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.sourceProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.forkRequestToApproximateProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.readerProgressToCloudProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.toCloudPosition; import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN; import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.everyItem; -import static org.hamcrest.collection.IsIterableContainingInOrder.contains; import com.google.api.services.dataflow.model.ApproximateProgress; import com.google.api.services.dataflow.model.Position; @@ -39,103 +40,14 @@ import org.junit.runners.JUnit4; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import java.util.NoSuchElementException; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Exchanger; /** * Tests for ReadOperation. */ @RunWith(JUnit4.class) public class ReadOperationTest { - private static final long ITERATIONS = 3L; - - /** - * The test Reader for testing updating stop position and progress report. - * The number of read iterations is controlled by ITERATIONS. - */ - static class TestTextReader extends Reader { - @Override - public ReaderIterator iterator() { - return new TestTextReaderIterator(); - } - - class TestTextReaderIterator extends AbstractReaderIterator { - long offset = 0L; - List proposedPositions = new ArrayList<>(); - - @Override - public boolean hasNext() { - return offset < ITERATIONS; - } - - @Override - public String next() { - if (hasNext()) { - offset++; - return "hi"; - } else { - throw new AssertionError("No next Element."); - } - } - - @Override - public Progress getProgress() { - com.google.api.services.dataflow.model.Position currentPosition = - new com.google.api.services.dataflow.model.Position(); - currentPosition.setByteOffset(offset); - - ApproximateProgress progress = new ApproximateProgress(); - progress.setPosition(currentPosition); - - return cloudProgressToReaderProgress(progress); - } - - @Override - public Position updateStopPosition(Progress proposedStopPosition) { - proposedPositions.add(sourceProgressToCloudProgress(proposedStopPosition).getPosition()); - // Actually no update happens, returns null. - return null; - } - } - } - - /** - * The OutputReceiver for testing updating stop position and progress report. - * The offset of the Reader (iterator) will be advanced each time this - * Receiver processes a record. - */ - static class TestTextReceiver extends OutputReceiver { - ReadOperation readOperation = null; - com.google.api.services.dataflow.model.Position proposedStopPosition = null; - List progresses = new ArrayList<>(); - - public TestTextReceiver(CounterSet counterSet, String counterPrefix) { - super("test_receiver_out", counterPrefix, counterSet.getAddCounterMutator()); - } - - public void setReadOperation(ReadOperation readOp) { - this.readOperation = readOp; - } - - public void setProposedStopPosition(com.google.api.services.dataflow.model.Position position) { - this.proposedStopPosition = position; - } - - @Override - public void process(Object outputElem) throws Exception { - // Calls getProgress() and proposeStopPosition() in each iteration. - progresses.add(sourceProgressToCloudProgress(readOperation.getProgress())); - // We expect that call to proposeStopPosition is a no-op that does not - // update the stop position for every iteration. We will verify it is - // delegated to ReaderIterator after ReadOperation finishes. - Assert.assertNull(readOperation.proposeStopPosition( - cloudProgressToReaderProgress(makeApproximateProgress(proposedStopPosition)))); - } - } - @Test @SuppressWarnings("unchecked") public void testRunReadOperation() throws Exception { @@ -178,129 +90,180 @@ public void testRunReadOperation() throws Exception { } @Test - public void testGetProgressAndProposeStopPosition() throws Exception { - TestTextReader testTextReader = new TestTextReader(); + public void testGetProgress() throws Exception { + MockReaderIterator iterator = new MockReaderIterator(0, 5); CounterSet counterSet = new CounterSet(); String counterPrefix = "test-"; - StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); - TestTextReceiver receiver = new TestTextReceiver(counterSet, counterPrefix); - ReadOperation readOperation = new ReadOperation( - testTextReader, receiver, counterPrefix, counterSet.getAddCounterMutator(), stateSampler); - readOperation.setProgressUpdatePeriodMs(0); - receiver.setReadOperation(readOperation); - - Position proposedStopPosition = makePosition(3L); - receiver.setProposedStopPosition(proposedStopPosition); - - Assert.assertNull(readOperation.getProgress()); - Assert.assertNull(readOperation.proposeStopPosition( - cloudProgressToReaderProgress(makeApproximateProgress(proposedStopPosition)))); - - readOperation.start(); - - TestTextReader.TestTextReaderIterator testIterator = - (TestTextReader.TestTextReaderIterator) readOperation.readerIterator; - - Assert.assertEquals( - sourceProgressToCloudProgress(testIterator.getProgress()), - sourceProgressToCloudProgress(readOperation.getProgress())); - Assert.assertEquals( - sourcePositionToCloudPosition(testIterator.updateStopPosition( - cloudProgressToReaderProgress(makeApproximateProgress(proposedStopPosition)))), - sourcePositionToCloudPosition(readOperation.proposeStopPosition( - cloudProgressToReaderProgress(makeApproximateProgress(proposedStopPosition))))); - - // Verifies progress report and stop position updates. - Assert.assertEquals(testIterator.proposedPositions.size(), ITERATIONS + 2); - Assert.assertThat(testIterator.proposedPositions, everyItem(equalTo(makePosition(3L)))); - Assert.assertThat( - receiver.progresses, - contains( - makeApproximateProgress(1L), makeApproximateProgress(2L), makeApproximateProgress(3L))); - - readOperation.finish(); + final ReadOperation readOperation = new ReadOperation(new MockReader(iterator), + new OutputReceiver("out", "test-", counterSet.getAddCounterMutator()), counterPrefix, + counterSet.getAddCounterMutator(), + new StateSampler(counterPrefix, counterSet.getAddCounterMutator())); + // Update progress not continuously, but so that it's never more than 1 record stale. + readOperation.setProgressUpdatePeriodMs(150); - Assert.assertNull(readOperation.proposeStopPosition( - cloudProgressToReaderProgress(makeApproximateProgress(proposedStopPosition)))); + Thread thread = runReadLoopInThread(readOperation); + for (int i = 0; i < 5; ++i) { + Thread.sleep(300); // Wait for the operation to start and block. + // Ensure that getProgress() doesn't block while the next() method is blocked. + ApproximateProgress progress = readerProgressToCloudProgress(readOperation.getProgress()); + long observedIndex = progress.getPosition().getRecordIndex().longValue(); + Assert.assertTrue("Actual: " + observedIndex, i == observedIndex || i == observedIndex + 1); + iterator.offerNext(i); + } + thread.join(); } @Test - public void testGetProgressDoesNotBlock() throws Exception { - final BlockingQueue queue = new LinkedBlockingQueue<>(); - final Reader.ReaderIterator iterator = new Reader.AbstractReaderIterator() { - private int itemsReturned = 0; + public void testFork() throws Exception { + MockReaderIterator iterator = new MockReaderIterator(0, 10); + CounterSet counterSet = new CounterSet(); + MockOutputReceiver receiver = new MockOutputReceiver(counterSet.getAddCounterMutator()); + ReadOperation readOperation = new ReadOperation(new MockReader(iterator), receiver, "test-", + counterSet.getAddCounterMutator(), + new StateSampler("test-", counterSet.getAddCounterMutator())); + // Update progress on every iteration of the read loop. + readOperation.setProgressUpdatePeriodMs(0); - @Override - public boolean hasNext() throws IOException { - return itemsReturned < 5; - } + // An unstarted ReadOperation refuses fork requests. + Assert.assertNull( + readOperation.requestFork(forkRequestAtIndex(7L))); + + Thread thread = runReadLoopInThread(readOperation); + iterator.offerNext(0); // Await first next() and return 0 from it. + // Read loop is now blocked in process() (not next()). + Reader.ForkResultWithPosition fork = (Reader.ForkResultWithPosition) readOperation.requestFork( + forkRequestAtIndex(7L)); + Assert.assertNotNull(fork); + Assert.assertEquals(positionAtIndex(7L), toCloudPosition(fork.getAcceptedPosition())); + receiver.unblockProcess(); + iterator.offerNext(1); + receiver.unblockProcess(); + iterator.offerNext(2); + + // Should accept a fork at an earlier position than previously requested. + // Should reject a fork at a later position than previously requested. + // Note that here we're testing our own MockReaderIterator class, so it's kind of pointless, + // but we're also testing that ReadOperation correctly relays the request to the iterator. + fork = (Reader.ForkResultWithPosition) readOperation.requestFork(forkRequestAtIndex(5L)); + Assert.assertNotNull(fork); + Assert.assertEquals(positionAtIndex(5L), toCloudPosition(fork.getAcceptedPosition())); + fork = (Reader.ForkResultWithPosition) readOperation.requestFork(forkRequestAtIndex(5L)); + Assert.assertNull(fork); + receiver.unblockProcess(); + + iterator.offerNext(3); + receiver.unblockProcess(); + iterator.offerNext(4); + receiver.unblockProcess(); + + // Should return false from hasNext() and exit read loop now. + + thread.join(); + + // Operation is now finished. Check that it refuses a fork request. + Assert.assertNull(readOperation.requestFork(forkRequestAtIndex(5L))); + } + private Thread runReadLoopInThread(final ReadOperation readOperation) { + Thread thread = new Thread() { @Override - public Integer next() throws IOException { - ++itemsReturned; + public void run() { try { - return queue.take(); - } catch (InterruptedException e) { - throw new NoSuchElementException("interrupted"); + readOperation.start(); + readOperation.finish(); + } catch (Exception e) { + e.printStackTrace(); } } - - @Override - public Reader.Progress getProgress() { - return cloudProgressToReaderProgress(new ApproximateProgress().setPosition( - new Position().setRecordIndex((long) itemsReturned))); - } }; + thread.start(); + return thread; + } - Reader reader = new Reader() { - @Override - public ReaderIterator iterator() throws IOException { - return iterator; + private static class MockReaderIterator extends Reader.AbstractReaderIterator { + private int to; + private Exchanger exchanger = new Exchanger<>(); + private int current; + + public MockReaderIterator(int from, int to) { + this.current = from; + this.to = to; + } + + @Override + public boolean hasNext() throws IOException { + return current < to; + } + + @Override + public Integer next() throws IOException { + ++current; + try { + return exchanger.exchange(current); + } catch (InterruptedException e) { + throw new NoSuchElementException("interrupted"); } - }; + } - CounterSet counterSet = new CounterSet(); - String counterPrefix = "test-"; - StateSampler stateSampler = new StateSampler(counterPrefix, counterSet.getAddCounterMutator()); - TestTextReceiver receiver = new TestTextReceiver(counterSet, counterPrefix); - final ReadOperation readOperation = new ReadOperation( - reader, receiver, counterPrefix, counterSet.getAddCounterMutator(), stateSampler); - // Update progress not continuously, but so that it's never more than 1 record stale. - readOperation.setProgressUpdatePeriodMs(150); - receiver.setReadOperation(readOperation); + @Override + public Reader.Progress getProgress() { + return cloudProgressToReaderProgress( + new ApproximateProgress().setPosition(new Position().setRecordIndex((long) current))); + } - new Thread() { - @Override - public void run() { - try { - readOperation.start(); - readOperation.finish(); - } catch (Exception e) { - e.printStackTrace(); - } + @Override + public Reader.ForkResult requestFork(Reader.ForkRequest forkRequest) { + ApproximateProgress progress = forkRequestToApproximateProgress(forkRequest); + int index = progress.getPosition().getRecordIndex().intValue(); + if (index >= to) { + return null; + } else { + this.to = index; + return new Reader.ForkResultWithPosition( + cloudPositionToReaderPosition(progress.getPosition())); } - }.start(); + } - for (int i = 0; i < 5; ++i) { - Thread.sleep(100); // Wait for the operation to start and block. - // Ensure that getProgress() doesn't block. - ApproximateProgress progress = sourceProgressToCloudProgress(readOperation.getProgress()); - long observedIndex = progress.getPosition().getRecordIndex().longValue(); - Assert.assertTrue("Actual: " + observedIndex, i == observedIndex || i == observedIndex + 1); - queue.offer(i); + public int offerNext(int next) { + try { + return exchanger.exchange(next); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } } } - private static Position makePosition(long offset) { - return new Position().setByteOffset(offset); - } + private static class MockReader extends Reader { + private ReaderIterator iterator; + + private MockReader(ReaderIterator iterator) { + this.iterator = iterator; + } - private static ApproximateProgress makeApproximateProgress(long offset) { - return makeApproximateProgress(makePosition(offset)); + @Override + public ReaderIterator iterator() throws IOException { + return iterator; + } } - private static ApproximateProgress makeApproximateProgress( - com.google.api.services.dataflow.model.Position position) { - return new ApproximateProgress().setPosition(position); + private static class MockOutputReceiver extends OutputReceiver { + private Exchanger exchanger = new Exchanger<>(); + + MockOutputReceiver(CounterSet.AddCounterMutator mutator) { + super("out", "test-", mutator); + } + + @Override + public void process(Object elem) throws Exception { + exchanger.exchange(null); + } + + public void unblockProcess() { + try { + exchanger.exchange(null); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } } } From c06125d29229309b8e2d2070d7fa140c9a2e14f5 Mon Sep 17 00:00:00 2001 From: mattlang Date: Mon, 9 Feb 2015 07:31:11 -0800 Subject: [PATCH 0141/1541] Fix NullPointerException thrown by getOnly when a value is not present for a given tag. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85888387 --- .../sdk/transforms/join/CoGbkResult.java | 3 +- .../sdk/transforms/join/CoGroupByKeyTest.java | 94 +++++++++++++++++++ 2 files changed, 96 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java index cbacb5fec1385..56fc3fc84bc0a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java @@ -225,6 +225,7 @@ public CoGbkResult decode( return new CoGbkResult(schema, map); } + @Override public boolean equals(Object other) { if (!super.equals(other)) { return false; @@ -317,7 +318,7 @@ private V innerGetOnly( + " is not in the schema"); } List unions = valueMap.get(index); - if (unions.isEmpty()) { + if (unions == null || unions.isEmpty()) { if (useDefault) { return defaultValue; } else { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java index 6369d06aee522..13a74b419fda1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java @@ -17,8 +17,10 @@ package com.google.cloud.dataflow.sdk.transforms.join; import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; +import static org.hamcrest.core.IsEqual.equalTo; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; @@ -75,6 +77,98 @@ public void processElement(ProcessContext c) { })); } + /** + * Returns a PCollection> containing the result + * of a CoGbk over 2 PCollection>, where each PCollection + * has no duplicate keys and the key sets of each PCollection are + * intersecting but neither is a subset of the other. + */ + private PCollection> buildGetOnlyGbk( + Pipeline p, + TupleTag tag1, + TupleTag tag2) { + List> list1 = + Arrays.asList( + KV.of(1, "collection1-1"), + KV.of(2, "collection1-2")); + List> list2 = + Arrays.asList( + KV.of(2, "collection2-2"), + KV.of(3, "collection2-3")); + PCollection> collection1 = createInput(p, list1); + PCollection> collection2 = createInput(p, list2); + PCollection> coGbkResults = + KeyedPCollectionTuple.of(tag1, collection1) + .and(tag2, collection2) + .apply(CoGroupByKey.create()); + return coGbkResults; + } + + @Test + public void testCoGroupByKeyGetOnly() { + TupleTag tag1 = new TupleTag<>(); + TupleTag tag2 = new TupleTag<>(); + + DirectPipeline p = DirectPipeline.createForTest(); + + PCollection> coGbkResults = + buildGetOnlyGbk(p, tag1, tag2); + + EvaluationResults results = p.run(); + + List> finalResult = + results.getPCollection(coGbkResults); + + HashMap> collection1Matchers = + new HashMap>() { + { + put(1, equalTo("collection1-1")); + put(2, equalTo("collection1-2")); + } + }; + + HashMap> collection2Matchers = + new HashMap>() { + { + put(2, equalTo("collection2-2")); + put(3, equalTo("collection2-3")); + } + }; + + for (KV result : finalResult) { + int key = result.getKey(); + CoGbkResult row = result.getValue(); + checkGetOnlyForKey(key, collection1Matchers, row, tag1, "default"); + checkGetOnlyForKey(key, collection2Matchers, row, tag2, "default"); + } + } + + /** + * Check that a singleton value for a key in a CoGbkResult matches the + * expected value in a map. If no value exists for the key, check that + * a default value is given (if supplied) and that an + * {@link IllegalArgumentException} is thrown if no default is supplied. + */ + private void checkGetOnlyForKey( + K key, + HashMap> matchers, + CoGbkResult row, + TupleTag tag, + V defaultValue) { + if (matchers.containsKey(key)) { + assertThat(row.getOnly(tag), matchers.get(key)); + } else { + assertThat(row.getOnly(tag, defaultValue), equalTo(defaultValue)); + try { + row.getOnly(tag); + fail(); + } catch (IllegalArgumentException e) { + // if no value exists, an IllegalArgumentException should be thrown + } + + } + } + /** * Returns a PCollection> containing the * results of the CoGbk over 3 PCollection>, each of From 914fc7c984e88e94918c4331fd9fb87e2fbfcb4b Mon Sep 17 00:00:00 2001 From: kirpichov Date: Mon, 9 Feb 2015 10:44:43 -0800 Subject: [PATCH 0142/1541] Don't log the whole classpath at INFO, but log how many files will be staged and how to see what they are. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85904037 --- .../cloud/dataflow/sdk/runners/DataflowPipelineRunner.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index 86781cd60eff3..fe6079faad55f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -110,7 +110,10 @@ public static DataflowPipelineRunner fromOptions(PipelineOptions options) { dataflowOptions.setFilesToStage(detectClassPathResourcesToStage( DataflowPipelineRunner.class.getClassLoader())); LOG.info("PipelineOptions.filesToStage was not specified. " - + "Defaulting to files from the classpath: {}", dataflowOptions.getFilesToStage()); + + "Defaulting to files from the classpath: will stage {} files. " + + "Enable logging at DEBUG level to see which files will be staged.", + dataflowOptions.getFilesToStage().size()); + LOG.debug("Classpath elements: {}", dataflowOptions.getFilesToStage()); } // Verify jobName according to service requirements. From df106aafebe4dfeacfb05c654eb07c2d43a6f55b Mon Sep 17 00:00:00 2001 From: bchambers Date: Mon, 9 Feb 2015 11:00:29 -0800 Subject: [PATCH 0143/1541] Update AvroCoder to be serializable, and verify that in the tests. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85905701 --- .../cloud/dataflow/sdk/coders/AvroCoder.java | 29 +++++++++++++++++++ .../dataflow/sdk/coders/AvroCoderTest.java | 19 ++++++++++++ 2 files changed, 48 insertions(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java index db0000cd25b60..75216120ac0fb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java @@ -40,6 +40,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.io.Serializable; import java.util.List; /** @@ -82,6 +83,7 @@ */ @SuppressWarnings("serial") public class AvroCoder extends StandardCoder { + /** * Returns an {@code AvroCoder} instance for the provided element type. * @param the element type @@ -134,6 +136,12 @@ protected AvroCoder(Class type, Schema schema) { this.writer = createDatumWriter(); } + private Object writeReplace() { + // When serialized by Java, instances of AvroCoder should be replaced by + // a SerializedAvroCoderProxy. + return new SerializedAvroCoderProxy<>(type, schema.toString()); + } + @Override public void encode(T value, OutputStream outStream, Context context) throws IOException { @@ -199,4 +207,25 @@ public DatumWriter createDatumWriter() { public Schema getSchema() { return schema; } + + /** + * Proxy to use in place of serializing the AvroCoder. This allows the fields + * to remain final. + */ + private static class SerializedAvroCoderProxy implements Serializable { + private final Class type; + private final String schemaStr; + + public SerializedAvroCoderProxy(Class type, String schemaStr) { + this.type = type; + this.schemaStr = schemaStr; + } + + private Object readResolve() { + // When deserialized, instances of this object should be replaced by + // constructing an AvroCoder. + Schema.Parser parser = new Schema.Parser(); + return new AvroCoder(type, parser.parse(schemaStr)); + } + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java index e3da71bcdff08..234fb046b2737 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java @@ -24,6 +24,7 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; import com.google.cloud.dataflow.sdk.values.PCollection; import org.apache.avro.Schema; @@ -37,6 +38,7 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; +import java.io.Serializable; /** * Tests for AvroCoder. @@ -180,4 +182,21 @@ public void testDefaultCoder() throws Exception { .containsInAnyOrder("hello", "world"); p.run(); } + + @Test + public void testAvroCoderJavaSerializable() throws Exception { + AvroCoder coder = AvroCoder.of(Pojo.class); + + // Cast the coder to serializable to test that it is serializable using + // Java serialization. + SerializableUtils.ensureSerializable((Serializable) coder); + } + + @Test + public void testAvroCoderJsonSerializable() throws Exception { + AvroCoder coder = AvroCoder.of(Pojo.class); + + // Check that the coder is serializable using the regular JSON approach. + SerializableUtils.ensureSerializable(coder); + } } From 02d4fb4357c9af517edbe0d3a66425dab67cff45 Mon Sep 17 00:00:00 2001 From: peihe Date: Mon, 9 Feb 2015 11:27:47 -0800 Subject: [PATCH 0144/1541] Testing: Run CoGroupByKeyTest on service, and add a windowing test case. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85908565 --- .../sdk/transforms/join/CoGroupByKeyTest.java | 151 +++++++++++++++++- 1 file changed, 145 insertions(+), 6 deletions(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java index 13a74b419fda1..69524fb623755 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import com.google.api.client.util.Preconditions; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; import com.google.cloud.dataflow.sdk.coders.KvCoder; @@ -34,12 +35,18 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.DoFnTester; import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.collect.Iterables; import org.hamcrest.Matcher; +import org.joda.time.Duration; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -62,8 +69,21 @@ public class CoGroupByKeyTest implements Serializable { */ private PCollection> createInput( Pipeline p, List> list) { - return p - .apply(Create.of(list)) + return createInput(p, list, new ArrayList()); + } + + /** + * Converts the given list with timestamps into a PCollection. + */ + private PCollection> createInput( + Pipeline p, List> list, List timestamps) { + PCollection> input; + if (timestamps.isEmpty()) { + input = p.apply(Create.of(list)); + } else { + input = p.apply(Create.timestamped(list, timestamps)); + } + return input // Create doesn't infer coders for parameterized types. .setCoder( KvCoder.of(BigEndianIntegerCoder.of(), StringUtf8Coder.of())) @@ -227,6 +247,59 @@ private PCollection> buildPurchasesCoGbk( return coGbkResults; } + /** + * Returns a PCollection> containing the + * results of the CoGbk over 2 PCollection>, each of + * which correlates a customer id to clicks, purchases, respectively. + */ + private PCollection> buildPurchasesCoGbkWithWindowing( + Pipeline p, + TupleTag clicksTag, + TupleTag purchasesTag) { + List> idToClick = + Arrays.asList( + KV.of(1, "Click t0"), + KV.of(2, "Click t2"), + KV.of(1, "Click t4"), + KV.of(1, "Click t6"), + KV.of(2, "Click t8")); + + List> idToPurchases = + Arrays.asList( + KV.of(1, "Boat t1"), + KV.of(1, "Shoesi t2"), + KV.of(1, "Pens t3"), + KV.of(2, "House t4"), + KV.of(2, "Suit t5"), + KV.of(1, "Car t6"), + KV.of(1, "Book t7"), + KV.of(2, "House t8"), + KV.of(2, "Shoes t9"), + KV.of(2, "House t10")); + + PCollection> clicksTable = + createInput( + p, + idToClick, + Arrays.asList(0L, 2L, 4L, 6L, 8L)) + .apply(Window.>into( + FixedWindows.>of(new Duration(4)))); + + PCollection> purchasesTable = + createInput( + p, + idToPurchases, + Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L)) + .apply(Window.>into( + FixedWindows.>of(new Duration(4)))); + + PCollection> coGbkResults = + KeyedPCollectionTuple.of(clicksTag, clicksTable) + .and(purchasesTag, purchasesTable) + .apply(CoGroupByKey.create()); + return coGbkResults; + } + @Test public void testCoGroupByKey() { TupleTag namesTag = new TupleTag<>(); @@ -307,6 +380,41 @@ private void checkValuesMatch( } } + /** + * A DoFn used in testCoGroupByKeyWithWindowing(), to test processing the + * results of a CoGroupByKey. + */ + private static class ClickOfPurchaseFn extends + DoFn, KV> { + private final TupleTag clicksTag; + + private final TupleTag purchasesTag; + + private ClickOfPurchaseFn( + TupleTag clicksTag, + TupleTag purchasesTag) { + this.clicksTag = clicksTag; + this.purchasesTag = purchasesTag; + } + + @Override + public void processElement(ProcessContext c) { + Preconditions.checkState(c.windows().size() == 1); + BoundedWindow w = c.windows().iterator().next(); + KV e = c.element(); + CoGbkResult row = e.getValue(); + Iterable clicks = row.getAll(clicksTag); + Iterable purchases = row.getAll(purchasesTag); + for (String click : clicks) { + for (String purchase : purchases) { + c.output(KV.of(click + ":" + purchase, + c.timestamp().getMillis() + ":" + w.maxTimestamp().getMillis())); + } + } + } + } + + /** * A DoFn used in testCoGroupByKeyHandleResults(), to test processing the * results of a CoGroupByKey. @@ -353,10 +461,7 @@ public void processElement(ProcessContext c) { Iterable purchases = row.getAll(purchasesTag); - int purchaseCount = 0; - for (String purchase : purchases) { - purchaseCount++; - } + int purchaseCount = Iterables.size(purchases); for (String address : addressList) { c.output(KV.of(address, purchaseCount)); @@ -416,6 +521,7 @@ public void testConsumingDoFn() { */ @SuppressWarnings("unchecked") @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) public void testCoGroupByKeyHandleResults() { TupleTag namesTag = new TupleTag<>(); TupleTag addressesTag = new TupleTag<>(); @@ -440,4 +546,37 @@ public void testCoGroupByKeyHandleResults() { KV.of("383 Jackson Street", 1)); p.run(); } + + /** + * Tests the pipeline end-to-end with FixedWindows. + */ + @SuppressWarnings("unchecked") + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testCoGroupByKeyWithWindowing() { + TupleTag clicksTag = new TupleTag<>(); + TupleTag purchasesTag = new TupleTag<>(); + + Pipeline p = TestPipeline.create(); + + PCollection> coGbkResults = + buildPurchasesCoGbkWithWindowing(p, clicksTag, purchasesTag); + + PCollection> + clickOfPurchase = coGbkResults.apply(ParDo.of( + new ClickOfPurchaseFn(clicksTag, purchasesTag))); + DataflowAssert.that(clickOfPurchase) + .containsInAnyOrder( + KV.of("Click t0:Boat t1", "3:3"), + KV.of("Click t0:Shoesi t2", "3:3"), + KV.of("Click t0:Pens t3", "3:3"), + KV.of("Click t4:Car t6", "7:7"), + KV.of("Click t4:Book t7", "7:7"), + KV.of("Click t6:Car t6", "7:7"), + KV.of("Click t6:Book t7", "7:7"), + KV.of("Click t8:House t8", "11:11"), + KV.of("Click t8:Shoes t9", "11:11"), + KV.of("Click t8:House t10", "11:11")); + p.run(); + } } From e76eff5e0cd3e00321b73a71aace8879e7c30143 Mon Sep 17 00:00:00 2001 From: davor Date: Mon, 9 Feb 2015 11:35:52 -0800 Subject: [PATCH 0145/1541] Dataflow SDK: Change email address to send Travis build status notifications. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85909441 --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 4815f28e12944..dc499659491a4 100644 --- a/.travis.yml +++ b/.travis.yml @@ -3,7 +3,7 @@ language: java notifications: email: recipients: - - dataflow-sdk+travis@google.com + - dataflow-sdk-build-notifications+travis@google.com on_success: change on_failure: always From a392b2c79a642f04a2c2e0a1eb4bc0c72810dc73 Mon Sep 17 00:00:00 2001 From: millsd Date: Mon, 9 Feb 2015 12:29:54 -0800 Subject: [PATCH 0146/1541] Uses slf4j logging in the streaming worker to enable Cloud Logging. Removes the use of MDC because it doesn't actually provide thread-local formatting parameters (http://bugzilla.slf4j.org/show_bug.cgi?id=325, https://github.com/qos-ch/slf4j/pull/99). [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85914693 --- .../runners/worker/DataflowWorkerHarness.java | 7 +- .../worker/StreamingDataflowWorker.java | 86 ++++++++++--------- .../DataflowWorkerLoggingFormatter.java | 57 ++++++++++-- .../worker/DataflowWorkerHarnessTest.java | 12 +-- .../worker/StreamingDataflowWorkerTest.java | 7 +- .../DataflowWorkerLoggingFormatterTest.java | 38 ++++---- ...a => RestoreDataflowLoggingFormatter.java} | 26 +++--- ... RestoreDataflowLoggingFormatterTest.java} | 25 +++--- 8 files changed, 154 insertions(+), 104 deletions(-) rename sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/{RestoreMappedDiagnosticContext.java => RestoreDataflowLoggingFormatter.java} (51%) rename sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/{RestoreMappedDiagnosticContextTest.java => RestoreDataflowLoggingFormatterTest.java} (59%) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java index d069b63d67a6a..4b0879e22c99d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java @@ -44,7 +44,6 @@ import org.joda.time.format.ISODateTimeFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.slf4j.MDC; import java.io.IOException; import java.lang.Thread.UncaughtExceptionHandler; @@ -142,8 +141,8 @@ public Boolean call() throws Exception { } static DataflowWorker create(DataflowWorkerHarnessOptions options) { - MDC.put(DataflowWorkerLoggingFormatter.MDC_DATAFLOW_JOB_ID, options.getJobId()); - MDC.put(DataflowWorkerLoggingFormatter.MDC_DATAFLOW_WORKER_ID, options.getWorkerId()); + DataflowWorkerLoggingFormatter.setJobId(options.getJobId()); + DataflowWorkerLoggingFormatter.setWorkerId(options.getWorkerId()); options.setAppName(APPLICATION_NAME); // Configure standard IO factories. @@ -223,7 +222,7 @@ public WorkItem getWorkItem() throws IOException { return null; } - MDC.put(DataflowWorkerLoggingFormatter.MDC_DATAFLOW_WORK_ID, Long.toString(work.getId())); + DataflowWorkerLoggingFormatter.setWorkId(Long.toString(work.getId())); // Looks like the work's a'ight. return work; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java index 9234d31c7e7fa..48d7ebd30623b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -18,8 +18,10 @@ import com.google.api.services.dataflow.model.MapTask; import com.google.api.services.dataflow.model.MetricUpdate; -import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.DataflowWorkerHarnessOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingFormatter; +import com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingInitializer; import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; import com.google.cloud.dataflow.sdk.runners.worker.windmill.WindmillServerStub; import com.google.cloud.dataflow.sdk.util.BoundedQueueExecutor; @@ -37,6 +39,9 @@ import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.AbstractHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; @@ -51,8 +56,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -import java.util.logging.Level; -import java.util.logging.Logger; import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; @@ -61,7 +64,7 @@ * Implements a Streaming Dataflow worker. */ public class StreamingDataflowWorker { - private static final Logger LOG = Logger.getLogger(StreamingDataflowWorker.class.getName()); + private static final Logger LOG = LoggerFactory.getLogger(StreamingDataflowWorker.class); static final int MAX_THREAD_POOL_SIZE = 100; static final long THREAD_EXPIRATION_TIME_SEC = 60; static final int MAX_THREAD_POOL_QUEUE_SIZE = 100; @@ -87,7 +90,8 @@ static MapTask parseMapTask(String input) throws IOException { } public static void main(String[] args) throws Exception { - LOG.setLevel(Level.INFO); + new DataflowWorkerLoggingInitializer().initialize(); + String hostport = System.getProperty("windmill.hostport"); if (hostport == null) { throw new Exception("-Dwindmill.hostport must be set to the location of the windmill server"); @@ -124,7 +128,7 @@ public static void main(String[] args) throws Exception { private Thread commitThread; private AtomicBoolean running; private StateFetcher stateFetcher; - private DataflowPipelineOptions options; + private DataflowWorkerHarnessOptions options; private long clientId; private Server statusServer; private AtomicReference lastException; @@ -132,11 +136,12 @@ public static void main(String[] args) throws Exception { /** Regular constructor. */ public StreamingDataflowWorker( List mapTasks, WindmillServerStub server) { - initialize(mapTasks, server); - options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + options = PipelineOptionsFactory.as(DataflowWorkerHarnessOptions.class); options.setAppName("StreamingWorkerHarness"); options.setStreaming(true); + initialize(mapTasks, server); + if (System.getProperties().containsKey("path_validator_class")) { try { options.setPathValidatorClass((Class) Class.forName( @@ -157,9 +162,9 @@ public StreamingDataflowWorker( /** The constructor that takes PipelineOptions. Should be used only by unit tests. */ StreamingDataflowWorker( - List mapTasks, WindmillServerStub server, DataflowPipelineOptions options) { - initialize(mapTasks, server); + List mapTasks, WindmillServerStub server, DataflowWorkerHarnessOptions options) { this.options = options; + initialize(mapTasks, server); } public void start() { @@ -202,8 +207,7 @@ public void stop() { } commitThread.join(); } catch (Exception e) { - LOG.warning("Exception while shutting down: " + e); - e.printStackTrace(); + LOG.warn("Exception while shutting down: ", e); } } @@ -219,8 +223,7 @@ private void initialize(List mapTasks, WindmillServerStub server) { private final Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() { public void uncaughtException(Thread thread, Throwable e) { - LOG.severe("Uncaught exception: " + e); - e.printStackTrace(); + LOG.error("Uncaught exception: ", e); System.exit(1); } }; @@ -239,6 +242,9 @@ public Thread newThread(Runnable r) { this.stateFetcher = new StateFetcher(server); this.clientId = new Random().nextLong(); this.lastException = new AtomicReference<>(); + + DataflowWorkerLoggingFormatter.setJobId(options.getJobId()); + DataflowWorkerLoggingFormatter.setWorkerId(options.getWorkerId()); } public void runStatusServer(int statusPort) { @@ -246,17 +252,17 @@ public void runStatusServer(int statusPort) { statusServer.setHandler(new StatusHandler()); try { statusServer.start(); - LOG.info("Status server started on port " + statusPort); + LOG.info("Status server started on port {}", statusPort); statusServer.join(); } catch (Exception e) { - LOG.warning("Status server failed to start: " + e); + LOG.warn("Status server failed to start: ", e); } } private void addComputation(MapTask mapTask) { String computation = mapTask.getSystemName(); if (!instructionMap.containsKey(computation)) { - LOG.info("Adding config for " + computation + ": " + mapTask); + LOG.info("Adding config for {}: {}", computation, mapTask); outputMap.put(computation, new ConcurrentLinkedQueue()); instructionMap.put(computation, mapTask); mapTaskExecutors.put( @@ -284,8 +290,8 @@ private void dispatchLoop() { // Also force a GC to try to get under the memory threshold if possible. while (rt.freeMemory() < rt.totalMemory() * PUSHBACK_THRESHOLD) { if (lastPushbackLog < (lastPushbackLog = System.currentTimeMillis()) - 60 * 1000) { - LOG.warning("In pushback, not accepting new work. Free Memory: " - + rt.freeMemory() + "MB / " + rt.totalMemory() + "MB"); + LOG.warn("In pushback, not accepting new work. Free Memory: {}MB / {}MB", + rt.freeMemory(), rt.totalMemory()); System.gc(); } sleep(10); @@ -320,12 +326,12 @@ public void run() { private void process( final String computation, final Windmill.WorkItem work) { - LOG.log(Level.FINE, "Starting processing for " + computation + ":\n{0}", work); + LOG.debug("Starting processing for {}:\n{}", computation, work); MapTask mapTask = instructionMap.get(computation); if (mapTask == null) { - LOG.info("Received work for unknown computation: " + computation - + ". Known computations are " + instructionMap.keySet()); + LOG.info("Received work for unknown computation: {}. Known computations are {}", + computation, instructionMap.keySet()); return; } @@ -338,6 +344,8 @@ private void process( MapTaskExecutor worker = null; try { + DataflowWorkerLoggingFormatter.setWorkId( + work.getKey().toStringUtf8() + "-" + Long.toString(work.getWorkToken())); WorkerAndContext workerAndContext = mapTaskExecutors.get(computation).poll(); if (workerAndContext == null) { context = new StreamingModeExecutionContext(computation, stateFetcher); @@ -359,28 +367,31 @@ private void process( mapTaskExecutors.get(computation).offer(new WorkerAndContext(worker, context)); worker = null; context = null; + + Windmill.WorkItemCommitRequest output = outputBuilder.build(); + outputMap.get(computation).add(output); + LOG.debug("Processing done for work token: {}", work.getWorkToken()); } catch (Throwable t) { if (worker != null) { try { worker.close(); } catch (Exception e) { - LOG.warning("Failed to close worker: " + e.getMessage()); - e.printStackTrace(); + LOG.warn("Failed to close worker: ", e); } } t = t instanceof UserCodeException ? t.getCause() : t; if (t instanceof KeyTokenInvalidException) { - LOG.fine("Execution of work for " + computation + " for key " + work.getKey().toStringUtf8() + LOG.debug("Execution of work for " + computation + + " for key " + work.getKey().toStringUtf8() + " failed due to token expiration, will not retry locally."); } else { - LOG.warning("Execution of work for " + computation + " for key " - + work.getKey().toStringUtf8() + " failed, retrying." - + "\nError: " + t.getMessage()); - t.printStackTrace(); + LOG.error("Execution of work for {} for key {} failed, retrying.", + computation, work.getKey().toStringUtf8()); + LOG.error("\nError: ", t); lastException.set(t); - LOG.fine("Failed work: " + work); + LOG.debug("Failed work: {}", work); reportFailure(computation, work, t); // Try again, but go to the end of the queue to avoid a tight loop. sleep(60000); @@ -390,12 +401,9 @@ public void run() { } }); } - return; + } finally { + DataflowWorkerLoggingFormatter.setWorkId(null); } - - Windmill.WorkItemCommitRequest output = outputBuilder.build(); - outputMap.get(computation).add(output); - LOG.fine("Processing done for work token: " + work.getWorkToken()); } private void commitLoop() { @@ -423,7 +431,7 @@ private void commitLoop() { } if (commitRequestBuilder.getRequestsCount() > 0) { Windmill.CommitWorkRequest commitRequest = commitRequestBuilder.build(); - LOG.log(Level.FINE, "Commit: {0}", commitRequest); + LOG.debug("Commit: {}", commitRequest); commitWork(commitRequest); } if (remainingCommitBytes > 0) { @@ -451,8 +459,8 @@ private void getConfig(String computation) { try { addComputation(parseMapTask(serializedMapTask)); } catch (IOException e) { - LOG.warning("Parsing MapTask failed: " + serializedMapTask); - e.printStackTrace(); + LOG.warn("Parsing MapTask failed: {}", serializedMapTask); + LOG.warn("Error: ", e); } } } @@ -472,7 +480,7 @@ private void buildCounters(CounterSet counterSet, } else if (cloudKind.equals(Counter.AggregationKind.MIN.name())) { kind = Windmill.Counter.Kind.MIN; } else { - LOG.log(Level.FINE, "Unhandled counter type: " + metricUpdate.getKind()); + LOG.debug("Unhandled counter type: {}", metricUpdate.getKind()); return; } Windmill.Counter.Builder counterBuilder = builder.addCounterUpdatesBuilder(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java index 911b3fe2e5e68..b6bb0ce960173 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java @@ -22,7 +22,6 @@ import org.joda.time.format.DateTimeFormatter; import org.joda.time.format.ISODateTimeFormat; -import org.slf4j.MDC; import java.io.PrintWriter; import java.io.StringWriter; @@ -39,9 +38,53 @@ public class DataflowWorkerLoggingFormatter extends Formatter { private static final DateTimeFormatter DATE_FORMATTER = ISODateTimeFormat.dateTime().withZoneUTC(); - public static final String MDC_DATAFLOW_JOB_ID = "dataflow.jobId"; - public static final String MDC_DATAFLOW_WORKER_ID = "dataflow.workerId"; - public static final String MDC_DATAFLOW_WORK_ID = "dataflow.workId"; + + private static final InheritableThreadLocal jobId = new InheritableThreadLocal<>(); + private static final InheritableThreadLocal workerId = new InheritableThreadLocal<>(); + private static final InheritableThreadLocal workId = new InheritableThreadLocal<>(); + + /** + * Sets the Job ID of the current thread, which will be inherited by child threads. + */ + public static void setJobId(String newJobId) { + jobId.set(newJobId); + } + + /** + * Sets the Worker ID of the current thread, which will be inherited by child threads. + */ + public static void setWorkerId(String newWorkerId) { + workerId.set(newWorkerId); + } + + /** + * Sets the Work ID of the current thread, which will be inherited by child threads. + */ + public static void setWorkId(String newWorkId) { + workId.set(newWorkId); + } + + /** + * Gets the Job ID of the current thread. + */ + public static String getJobId() { + return jobId.get(); + } + + /** + * Gets the Worker ID of the current thread. + */ + public static String getWorkerId() { + return workerId.get(); + } + + /** + * Gets the Work ID of the current thread. + */ + public static String getWorkId() { + return workId.get(); + } + @Override public String format(LogRecord record) { @@ -49,9 +92,9 @@ public String format(LogRecord record) { return DATE_FORMATTER.print(record.getMillis()) + " " + MoreObjects.firstNonNull(LEVELS.get(record.getLevel()), record.getLevel().getName()) - + " " + MoreObjects.firstNonNull(MDC.get(MDC_DATAFLOW_JOB_ID), "unknown") - + " " + MoreObjects.firstNonNull(MDC.get(MDC_DATAFLOW_WORKER_ID), "unknown") - + " " + MoreObjects.firstNonNull(MDC.get(MDC_DATAFLOW_WORK_ID), "unknown") + + " " + MoreObjects.firstNonNull(jobId.get(), "unknown") + + " " + MoreObjects.firstNonNull(workerId.get(), "unknown") + + " " + MoreObjects.firstNonNull(workId.get(), "unknown") + " " + record.getThreadID() + " " + record.getLoggerName() + " " + record.getMessage() + System.lineSeparator() diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java index e167aab7cf1ba..6dc9424e9848c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java @@ -35,7 +35,8 @@ import com.google.api.services.dataflow.model.WorkItem; import com.google.cloud.dataflow.sdk.options.DataflowWorkerHarnessOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import com.google.cloud.dataflow.sdk.testing.RestoreMappedDiagnosticContext; +import com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingFormatter; +import com.google.cloud.dataflow.sdk.testing.RestoreDataflowLoggingFormatter; import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties; import com.google.cloud.dataflow.sdk.util.TestCredential; import com.google.cloud.dataflow.sdk.util.Transport; @@ -52,7 +53,6 @@ import org.junit.runners.JUnit4; import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import org.slf4j.MDC; import java.io.IOException; @@ -60,7 +60,7 @@ @RunWith(JUnit4.class) public class DataflowWorkerHarnessTest { @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); - @Rule public TestRule restoreMDC = new RestoreMappedDiagnosticContext(); + @Rule public TestRule restoreLogging = new RestoreDataflowLoggingFormatter(); @Rule public ExpectedException expectedException = ExpectedException.none(); @Mock private MockHttpTransport transport; @Mock private MockLowLevelHttpRequest request; @@ -108,8 +108,8 @@ public void testCreationOfWorkerHarness() throws Exception { DataflowWorkerHarnessOptions options = PipelineOptionsFactory.createFromSystemProperties(); options.setGcpCredential(new TestCredential()); assertNotNull(DataflowWorkerHarness.create(options)); - assertEquals("jobId", MDC.get("dataflow.jobId")); - assertEquals("workerId", MDC.get("dataflow.workerId")); + assertEquals("jobId", DataflowWorkerLoggingFormatter.getJobId()); + assertEquals("workerId", DataflowWorkerLoggingFormatter.getWorkerId()); } @Test @@ -138,7 +138,7 @@ public void testCloudServiceCall() throws Exception { actualRequest.getWorkerCapabilities()); assertEquals(ImmutableList.of("map_task", "seq_map_task", "remote_source_task"), actualRequest.getWorkItemTypes()); - assertEquals("1234", MDC.get("dataflow.workId")); + assertEquals("1234", DataflowWorkerLoggingFormatter.getWorkId()); } @Test diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java index 9c5c5cbd24f0c..536ba0b7fa133 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java @@ -33,7 +33,7 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.ListCoder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; -import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.DataflowWorkerHarnessOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; import com.google.cloud.dataflow.sdk.runners.worker.windmill.WindmillServerStub; @@ -317,8 +317,9 @@ private Windmill.WorkItemCommitRequest makeExpectedOutput(int index, long timest Arrays.asList(DEFAULT_WINDOW))); } - private DataflowPipelineOptions createTestingPipelineOptions() { - DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + private DataflowWorkerHarnessOptions createTestingPipelineOptions() { + DataflowWorkerHarnessOptions options = + PipelineOptionsFactory.as(DataflowWorkerHarnessOptions.class); options.setAppName("StreamingWorkerHarnessTest"); options.setStreaming(true); return options; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java index 4fd2b5f4688d4..2bf711c84bbca 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java @@ -18,15 +18,13 @@ import static org.junit.Assert.assertEquals; -import com.google.cloud.dataflow.sdk.testing.RestoreMappedDiagnosticContext; -import com.google.common.collect.ImmutableMap; +import com.google.cloud.dataflow.sdk.testing.RestoreDataflowLoggingFormatter; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestRule; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.slf4j.MDC; import java.util.logging.Level; import java.util.logging.LogRecord; @@ -34,7 +32,7 @@ /** Unit tests for {@link DataflowWorkerLoggingFormatter}. */ @RunWith(JUnit4.class) public class DataflowWorkerLoggingFormatterTest { - @Rule public TestRule restoreMDC = new RestoreMappedDiagnosticContext(); + @Rule public TestRule restoreMDC = new RestoreDataflowLoggingFormatter(); @Test public void testWithUnsetValuesInMDC() { @@ -47,10 +45,10 @@ public void testWithUnsetValuesInMDC() { @Test public void testWithMessage() { - MDC.setContextMap(ImmutableMap.of( - "dataflow.jobId", "testJobId", - "dataflow.workerId", "testWorkerId", - "dataflow.workId", "testWorkId")); + DataflowWorkerLoggingFormatter.setJobId("testJobId"); + DataflowWorkerLoggingFormatter.setWorkerId("testWorkerId"); + DataflowWorkerLoggingFormatter.setWorkId("testWorkId"); + assertEquals( "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName " + "test.message" + System.lineSeparator(), @@ -60,10 +58,10 @@ public void testWithMessage() { @Test public void testWithMessageAndException() { - MDC.setContextMap(ImmutableMap.of( - "dataflow.jobId", "testJobId", - "dataflow.workerId", "testWorkerId", - "dataflow.workId", "testWorkId")); + DataflowWorkerLoggingFormatter.setJobId("testJobId"); + DataflowWorkerLoggingFormatter.setWorkerId("testWorkerId"); + DataflowWorkerLoggingFormatter.setWorkId("testWorkId"); + assertEquals( "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName " + "test.message" + System.lineSeparator() @@ -77,10 +75,10 @@ public void testWithMessageAndException() { @Test public void testWithException() { - MDC.setContextMap(ImmutableMap.of( - "dataflow.jobId", "testJobId", - "dataflow.workerId", "testWorkerId", - "dataflow.workId", "testWorkId")); + DataflowWorkerLoggingFormatter.setJobId("testJobId"); + DataflowWorkerLoggingFormatter.setWorkerId("testWorkerId"); + DataflowWorkerLoggingFormatter.setWorkId("testWorkId"); + assertEquals( "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName null" + System.lineSeparator() @@ -94,10 +92,10 @@ public void testWithException() { @Test public void testWithoutExceptionOrMessage() { - MDC.setContextMap(ImmutableMap.of( - "dataflow.jobId", "testJobId", - "dataflow.workerId", "testWorkerId", - "dataflow.workId", "testWorkId")); + DataflowWorkerLoggingFormatter.setJobId("testJobId"); + DataflowWorkerLoggingFormatter.setWorkerId("testWorkerId"); + DataflowWorkerLoggingFormatter.setWorkId("testWorkId"); + assertEquals( "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName null" + System.lineSeparator(), diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContext.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreDataflowLoggingFormatter.java similarity index 51% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContext.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreDataflowLoggingFormatter.java index f0bdb9e217044..6f5309e332e18 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContext.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreDataflowLoggingFormatter.java @@ -16,32 +16,32 @@ package com.google.cloud.dataflow.sdk.testing; -import com.google.common.base.MoreObjects; -import com.google.common.collect.ImmutableMap; +import com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingFormatter; import org.junit.rules.ExternalResource; -import org.slf4j.MDC; - -import java.util.Map; /** - * Saves and restores the current MDC for tests. + * Saves and restores the current thread-local logging parameters for tests. */ -public class RestoreMappedDiagnosticContext extends ExternalResource { - private Map previousValue; +public class RestoreDataflowLoggingFormatter extends ExternalResource { + private String previousJobId; + private String previousWorkerId; + private String previousWorkId; - public RestoreMappedDiagnosticContext() { + public RestoreDataflowLoggingFormatter() { } @Override protected void before() throws Throwable { - previousValue = MoreObjects.firstNonNull( - MDC.getCopyOfContextMap(), - ImmutableMap.of()); + previousJobId = DataflowWorkerLoggingFormatter.getJobId(); + previousWorkerId = DataflowWorkerLoggingFormatter.getWorkerId(); + previousWorkId = DataflowWorkerLoggingFormatter.getWorkId(); } @Override protected void after() { - MDC.setContextMap(previousValue); + DataflowWorkerLoggingFormatter.setJobId(previousJobId); + DataflowWorkerLoggingFormatter.setWorkerId(previousWorkerId); + DataflowWorkerLoggingFormatter.setWorkId(previousWorkId); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContextTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreDataflowLoggingFormatterTest.java similarity index 59% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContextTest.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreDataflowLoggingFormatterTest.java index c88f275f4bf65..46ff22912ef42 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreMappedDiagnosticContextTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/RestoreDataflowLoggingFormatterTest.java @@ -19,33 +19,34 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingFormatter; + import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestRule; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.slf4j.MDC; -/** Tests for {@link RestoreMappedDiagnosticContext}. */ +/** Tests for {@link RestoreDataflowLoggingFormatter}. */ @RunWith(JUnit4.class) -public class RestoreMappedDiagnosticContextTest { - @Rule public TestRule restoreMappedDiagnosticContext = new RestoreMappedDiagnosticContext(); +public class RestoreDataflowLoggingFormatterTest { + @Rule public TestRule restoreDataflowLoggingFormatter = new RestoreDataflowLoggingFormatter(); /* * Since these tests can run out of order, both test A and B verify that they * could insert their property and that the other does not exist. */ @Test - public void testThatMDCIsClearedA() { - MDC.put("TestA", "TestA"); - assertNotNull(MDC.get("TestA")); - assertNull(MDC.get("TestB")); + public void testLoggingParamsClearedA() { + DataflowWorkerLoggingFormatter.setJobId("job"); + assertNotNull(DataflowWorkerLoggingFormatter.getJobId()); + assertNull(DataflowWorkerLoggingFormatter.getWorkerId()); } @Test - public void testThatMDCIsClearedB() { - MDC.put("TestB", "TestB"); - assertNotNull(MDC.get("TestB")); - assertNull(MDC.get("TestA")); + public void testLoggingParamsClearedB() { + DataflowWorkerLoggingFormatter.setWorkerId("worker"); + assertNotNull(DataflowWorkerLoggingFormatter.getWorkerId()); + assertNull(DataflowWorkerLoggingFormatter.getJobId()); } } From ea18500b103ac76c7bc83ac6533dac689bf84a39 Mon Sep 17 00:00:00 2001 From: earhart Date: Mon, 9 Feb 2015 13:21:33 -0800 Subject: [PATCH 0147/1541] Add the report index to work item status messages sent to the Dataflow service. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85919759 --- .../worker/DataflowWorkProgressUpdater.java | 16 +++++++++- .../sdk/runners/worker/DataflowWorker.java | 17 +++++----- .../DataflowWorkProgressUpdaterTest.java | 31 ++++++++++++++++--- 3 files changed, 52 insertions(+), 12 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java index 66c1fd1a2c862..ca0b1667dae15 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java @@ -54,12 +54,16 @@ public class DataflowWorkProgressUpdater extends WorkProgressUpdater { /** Options specifying information about the pipeline run by the worker.*/ private final DataflowWorkerHarnessOptions options; + /** The index to use for the next report sent for the updater's work item. */ + private long nextReportIndex; + public DataflowWorkProgressUpdater(WorkItem workItem, WorkExecutor worker, DataflowWorker.WorkUnitClient workUnitClient, DataflowWorkerHarnessOptions options) { super(worker); this.workItem = workItem; this.workUnitClient = workUnitClient; this.options = options; + this.nextReportIndex = 1; } @Override @@ -76,13 +80,15 @@ protected long getWorkUnitLeaseExpirationTimestamp() { protected void reportProgressHelper() throws Exception { WorkItemStatus status = buildStatus(workItem, false/*completed*/, worker.getOutputCounters(), worker.getOutputMetrics(), options, worker.getWorkerProgress(), forkResultToReport, - null/*sourceOperationResponse*/, null/*errors*/); + null/*sourceOperationResponse*/, null/*errors*/, + getNextReportIndex()); status.setRequestedLeaseDuration(toCloudDuration(Duration.millis(requestedLeaseDurationMs))); WorkItemServiceState result = workUnitClient.reportWorkItemStatus(status); if (result != null) { // Resets state after a successful progress report. forkResultToReport = null; + nextReportIndex++; progressReportIntervalMs = nextProgressReportInterval( fromCloudDuration(workItem.getReportStatusInterval()).getMillis(), @@ -105,4 +111,12 @@ private long getLeaseExpirationTimestamp(WorkItem workItem) { private long getLeaseExpirationTimestamp(WorkItemServiceState workItemServiceState) { return fromCloudTime(workItemServiceState.getLeaseExpireTime()).getMillis(); } + + /** + * Returns the index to use for the next work item report for the work + * progress updater's work item. + */ + long getNextReportIndex() { + return nextReportIndex; + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java index 3a46be9c173cd..f1fe0d2f6ec01 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java @@ -38,7 +38,6 @@ import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.cloud.dataflow.sdk.util.common.worker.SourceFormat; import com.google.cloud.dataflow.sdk.util.common.worker.WorkExecutor; -import com.google.cloud.dataflow.sdk.util.common.worker.WorkProgressUpdater; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -121,7 +120,7 @@ private boolean doWork(WorkItem workItem) throws IOException { throw new RuntimeException("unknown kind of work item: " + workItem.toString()); } - WorkProgressUpdater progressUpdater = + DataflowWorkProgressUpdater progressUpdater = new DataflowWorkProgressUpdater(workItem, worker, workUnitClient, options); progressUpdater.startReportingProgress(); @@ -156,7 +155,8 @@ private boolean doWork(WorkItem workItem) throws IOException { ((SourceOperationExecutor) worker).getResponse()) : null; reportStatus( - options, "Success", workItem, counters, metrics, operationResponse, null/*errors*/); + options, "Success", workItem, counters, metrics, operationResponse, null/*errors*/, + progressUpdater.getNextReportIndex()); return true; @@ -190,7 +190,7 @@ private void handleWorkError(WorkItem workItem, WorkExecutor worker, Throwable e reportStatus(options, "Failure", workItem, worker == null ? null : worker.getOutputCounters(), worker == null ? null : worker.getOutputMetrics(), null/*sourceOperationResponse*/, - error == null ? null : Collections.singletonList(error)); + error == null ? null : Collections.singletonList(error), 0); } /** @@ -219,11 +219,12 @@ private static String buildCloudStackTrace(Throwable t) { private void reportStatus(DataflowWorkerHarnessOptions options, String status, WorkItem workItem, @Nullable CounterSet counters, @Nullable Collection> metrics, - @Nullable SourceFormat.OperationResponse operationResponse, @Nullable List errors) + @Nullable SourceFormat.OperationResponse operationResponse, @Nullable List errors, + long finalReportIndex) throws IOException { LOG.info("{} processing work item {}", status, uniqueId(workItem)); WorkItemStatus workItemStatus = buildStatus(workItem, true/*completed*/, counters, metrics, - options, null, null, operationResponse, errors); + options, null, null, operationResponse, errors, finalReportIndex); workUnitClient.reportWorkItemStatus(workItemStatus); } @@ -231,10 +232,12 @@ static WorkItemStatus buildStatus(WorkItem workItem, boolean completed, @Nullable CounterSet counters, @Nullable Collection> metrics, DataflowWorkerHarnessOptions options, @Nullable Reader.Progress progress, @Nullable Reader.ForkResult forkResult, - @Nullable SourceFormat.OperationResponse operationResponse, @Nullable List errors) { + @Nullable SourceFormat.OperationResponse operationResponse, @Nullable List errors, + long finalReportIndex) { WorkItemStatus status = new WorkItemStatus(); status.setWorkItemId(Long.toString(workItem.getId())); status.setCompleted(completed); + status.setReportIndex(finalReportIndex); List counterUpdates = null; List metricUpdates = null; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java index 0745e40faef03..3146c0e72c379 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java @@ -209,7 +209,7 @@ public void workProgressUpdaterAdaptsProgressInterval() throws Exception { verify(workUnitClient, timeout(600)) .reportWorkItemStatus(argThat( new ExpectedDataflowWorkItemStatus().withCounters(3).withMetrics(2).withProgress( - approximateProgressAtIndex(1L)))); + approximateProgressAtIndex(1L)).withReportIndex(1L))); setUpCounters(5); setUpMetrics(6); @@ -221,7 +221,8 @@ public void workProgressUpdaterAdaptsProgressInterval() throws Exception { .withCounters(5) .withMetrics(6) .withProgress(approximateProgressAtIndex(2L)) - .withForkAtPosition(positionAtIndex(3L)))); + .withForkAtPosition(positionAtIndex(3L)) + .withReportIndex(2L))); // After the request is sent, reset cached fork result to null. assertNull(progressUpdater.getForkResultToReport()); @@ -231,9 +232,12 @@ public void workProgressUpdaterAdaptsProgressInterval() throws Exception { // The third update should be sent after one and half seconds (3000 / 2). verify(workUnitClient, timeout(1600)) .reportWorkItemStatus(argThat( - new ExpectedDataflowWorkItemStatus().withProgress(approximateProgressAtIndex(3L)))); + new ExpectedDataflowWorkItemStatus().withProgress(approximateProgressAtIndex(3L)) + .withReportIndex(3L))); progressUpdater.stopReportingProgress(); + + assertEquals(4L, progressUpdater.getNextReportIndex()); } // Verifies that a last update is sent when there is an unacknowledged split request. @@ -344,6 +348,9 @@ private static final class ExpectedDataflowWorkItemStatus @Nullable Position expectedForkPosition; + @Nullable + Long expectedReportIndex; + public ExpectedDataflowWorkItemStatus withCounters(Integer counterCount) { this.counterCount = counterCount; return this; @@ -364,6 +371,11 @@ public ExpectedDataflowWorkItemStatus withForkAtPosition(Position expectedForkPo return this; } + public ExpectedDataflowWorkItemStatus withReportIndex(Long reportIndex) { + this.expectedReportIndex = reportIndex; + return this; + } + @Override public void describeTo(Description description) { List values = new ArrayList<>(); @@ -385,13 +397,17 @@ public void describeTo(Description description) { } else { values.add("no fork position present"); } + if (this.expectedReportIndex != null) { + values.add("reportIndex " + this.expectedReportIndex); + } description.appendValueList("Dataflow WorkItemStatus with ", ", ", ".", values); } @Override public boolean matches(Object status) { WorkItemStatus st = (WorkItemStatus) status; - return matchCountersAndMetrics(st) && matchProgress(st) && matchStopPosition(st); + return matchCountersAndMetrics(st) && matchProgress(st) && matchStopPosition(st) + && matchReportIndex(st); } private boolean matchCountersAndMetrics(WorkItemStatus status) { @@ -435,5 +451,12 @@ private boolean matchStopPosition(WorkItemStatus status) { } return expectedForkPosition.equals(actualStopPosition); } + + private boolean matchReportIndex(WorkItemStatus status) { + if (expectedReportIndex == null) { + return true; + } + return expectedReportIndex.equals(status.getReportIndex()); + } } } From fcd3a5291da5d204e2bf530e62afdcd744308233 Mon Sep 17 00:00:00 2001 From: samuelw Date: Mon, 9 Feb 2015 18:56:47 -0800 Subject: [PATCH 0148/1541] Fixed logic inversion and added test of exception handling retries. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85949368 --- .../worker/StreamingDataflowWorker.java | 38 +++++++++++-------- .../worker/StreamingDataflowWorkerTest.java | 24 ++++++++++-- 2 files changed, 44 insertions(+), 18 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java index 48d7ebd30623b..c9167b13da975 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -392,14 +392,19 @@ private void process( LOG.error("\nError: ", t); lastException.set(t); LOG.debug("Failed work: {}", work); - reportFailure(computation, work, t); - // Try again, but go to the end of the queue to avoid a tight loop. - sleep(60000); - executor.forceExecute(new Runnable() { - public void run() { - process(computation, work); - } - }); + if (reportFailure(computation, work, t)) { + // Try again, after some delay and at the end of the queue to avoid a tight loop. + sleep(10000); + executor.forceExecute(new Runnable() { + public void run() { + process(computation, work); + } + }); + } else { + // If we failed to report the error, the item is invalid and should + // not be retried internally. It will be retried at the higher level. + LOG.debug("Aborting processing due to exception reporting failure"); + } } } finally { DataflowWorkerLoggingFormatter.setWorkId(null); @@ -537,13 +542,16 @@ private Windmill.Exception buildExceptionReport(Throwable t) { return builder.build(); } - private void reportFailure(String computation, Windmill.WorkItem work, Throwable t) { - windmillServer.reportStats(Windmill.ReportStatsRequest.newBuilder() - .setComputationId(computation) - .setKey(work.getKey()) - .setWorkToken(work.getWorkToken()) - .addExceptions(buildExceptionReport(t)) - .build()); + // Returns true if reporting the exception is successful and the work should be retried. + private boolean reportFailure(String computation, Windmill.WorkItem work, Throwable t) { + Windmill.ReportStatsResponse response = + windmillServer.reportStats(Windmill.ReportStatsRequest.newBuilder() + .setComputationId(computation) + .setKey(work.getKey()) + .setWorkToken(work.getWorkToken()) + .addExceptions(buildExceptionReport(t)) + .build()); + return !response.getFailed(); } private static class WorkerAndContext { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java index 536ba0b7fa133..4005140fcd1d9 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java @@ -75,6 +75,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; /** Unit tests for {@link StreamingDataflowWorker}. */ @RunWith(JUnit4.class) @@ -88,12 +89,14 @@ private static class FakeWindmillServer extends WindmillServerStub { private Map commitsReceived; private LinkedBlockingQueue exceptions; private int commitsRequested = 0; + private AtomicInteger expectedExceptionCount; public FakeWindmillServer() { workToOffer = new ConcurrentLinkedQueue(); dataToOffer = new ConcurrentLinkedQueue(); commitsReceived = new ConcurrentHashMap(); exceptions = new LinkedBlockingQueue<>(); + expectedExceptionCount = new AtomicInteger(); } public void addWorkToOffer(Windmill.GetWorkResponse work) { @@ -142,16 +145,22 @@ public Windmill.ReportStatsResponse reportStats(Windmill.ReportStatsRequest requ for (Windmill.Exception exception : request.getExceptionsList()) { try { exceptions.put(exception); - } catch (InterruptedException e) {} + } catch (InterruptedException expected) {} + } + + if (expectedExceptionCount.getAndDecrement() > 0) { + return Windmill.ReportStatsResponse.newBuilder().build(); + } else { + return Windmill.ReportStatsResponse.newBuilder() + .setFailed(true).build(); } - return Windmill.ReportStatsResponse.newBuilder().build(); } public Map waitForAndGetCommits(int numCommits) { while (commitsReceived.size() < commitsRequested + numCommits) { try { Thread.sleep(1000); - } catch (InterruptedException e) {} + } catch (InterruptedException expected) {} } commitsRequested += numCommits; @@ -159,6 +168,10 @@ public Map waitForAndGetCommits(int numCom return commitsReceived; } + public void setExpectedExceptionCount(int i) { + expectedExceptionCount.getAndAdd(i); + } + public Windmill.Exception getException() throws InterruptedException { return exceptions.take(); } @@ -614,6 +627,7 @@ public void processElement(ProcessContext c) throws Exception { makeSinkInstruction(StringUtf8Coder.of(), 1)); FakeWindmillServer server = new FakeWindmillServer(); + server.setExpectedExceptionCount(1); server.addWorkToOffer(buildInput( "work {" + " computation_id: \"computation\"" + @@ -649,6 +663,10 @@ public void processElement(ProcessContext c) throws Exception { Assert.assertThat(exception.getCause().getStackFrames(1), JUnitMatchers.containsString("processElement")); Assert.assertFalse(exception.getCause().hasCause()); + + // The server should retry the work since reporting the exception succeeded. + // Make next retry should fail because we only expected 1 exception. + exception = server.getException(); } private static class TestTimerFn From 8378460a9f932906a11f54f9ecae2e73df182472 Mon Sep 17 00:00:00 2001 From: lcwik Date: Tue, 10 Feb 2015 09:19:08 -0800 Subject: [PATCH 0149/1541] Setup worker harness to only log to file to prevent duplication of log records sent to Cloud Logging and Monitoring. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85994616 --- .../DataflowWorkerLoggingInitializer.java | 6 ------ .../DataflowWorkerLoggingInitializerTest.java | 18 ++++-------------- 2 files changed, 4 insertions(+), 20 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java index e89d6d73b94c2..a513dc75ee359 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java @@ -20,7 +20,6 @@ import java.io.File; import java.io.IOException; -import java.util.logging.ConsoleHandler; import java.util.logging.FileHandler; import java.util.logging.Formatter; import java.util.logging.Handler; @@ -69,10 +68,6 @@ void initialize(LogManager logManager) { fileHandler.setFormatter(formatter); fileHandler.setLevel(logLevel); - ConsoleHandler consoleHandler = new ConsoleHandler(); - consoleHandler.setFormatter(formatter); - consoleHandler.setLevel(logLevel); - // Reset the global log manager, get the root logger and remove the default log handlers. logManager.reset(); Logger rootLogger = logManager.getLogger(ROOT_LOGGER_NAME); @@ -81,7 +76,6 @@ void initialize(LogManager logManager) { } rootLogger.setLevel(logLevel); - rootLogger.addHandler(consoleHandler); rootLogger.addHandler(fileHandler); } catch (SecurityException | IOException e) { throw new ExceptionInInitializerError(e); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java index 68a6b58a2d9bc..50cc1e2d3ed18 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java @@ -17,7 +17,6 @@ package com.google.cloud.dataflow.sdk.runners.worker.logging; import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; @@ -35,7 +34,6 @@ import org.mockito.MockitoAnnotations; import java.util.List; -import java.util.logging.ConsoleHandler; import java.util.logging.FileHandler; import java.util.logging.Handler; import java.util.logging.Level; @@ -68,12 +66,11 @@ public void testWithDefaults() { verify(mockRootLogger).getHandlers(); verify(mockRootLogger).removeHandler(mockHandler); verify(mockRootLogger).setLevel(Level.INFO); - verify(mockRootLogger, times(2)).addHandler(argument.capture()); + verify(mockRootLogger).addHandler(argument.capture()); verifyNoMoreInteractions(mockLogManager, mockRootLogger); List handlers = argument.getAllValues(); - assertTrue(isConsoleHandler(handlers.get(0), Level.INFO)); - assertTrue(isFileHandler(handlers.get(1), Level.INFO)); + assertTrue(isFileHandler(handlers.get(0), Level.INFO)); } @Test @@ -87,18 +84,11 @@ public void testWithOverrides() { verify(mockRootLogger).getHandlers(); verify(mockRootLogger).removeHandler(mockHandler); verify(mockRootLogger).setLevel(Level.WARNING); - verify(mockRootLogger, times(2)).addHandler(argument.capture()); + verify(mockRootLogger).addHandler(argument.capture()); verifyNoMoreInteractions(mockLogManager, mockRootLogger); List handlers = argument.getAllValues(); - assertTrue(isConsoleHandler(handlers.get(0), Level.WARNING)); - assertTrue(isFileHandler(handlers.get(1), Level.WARNING)); - } - - private boolean isConsoleHandler(Handler handler, Level level) { - return handler instanceof ConsoleHandler - && level.equals(handler.getLevel()) - && handler.getFormatter() instanceof DataflowWorkerLoggingFormatter; + assertTrue(isFileHandler(handlers.get(0), Level.WARNING)); } private boolean isFileHandler(Handler handler, Level level) { From 1294029bd8b2311f4a2dacda8d74aa024f0ba0fb Mon Sep 17 00:00:00 2001 From: lcwik Date: Tue, 10 Feb 2015 10:07:25 -0800 Subject: [PATCH 0150/1541] Remove logic for getting credentials from gcloud binary since official application default credentials can now pass along credentials from the currently logged in user. See https://developers.google.com/accounts/docs/application-default-credentials for more details. Users will need to update to the latest version of gcloud by running 'gcloud components update'. See https://cloud.google.com/sdk/gcloud/ for more details. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=85998709 --- .../dataflow/sdk/options/GcpOptions.java | 5 - .../cloud/dataflow/sdk/util/Credentials.java | 63 +++------- .../dataflow/sdk/util/GCloudCredential.java | 113 ------------------ 3 files changed, 18 insertions(+), 163 deletions(-) delete mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GCloudCredential.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java index 0d824405b9075..571dba49cd0e2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java @@ -96,11 +96,6 @@ public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions { String getServiceAccountName(); void setServiceAccountName(String value); - @Description("The path to the gcloud binary. " - + " Default is to search the system path.") - String getGCloudPath(); - void setGCloudPath(String value); - /** * Directory for storing dataflow credentials. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java index e37275cdb936f..c416eae3d844e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java @@ -28,7 +28,6 @@ import com.google.api.client.json.JsonFactory; import com.google.api.client.json.jackson2.JacksonFactory; import com.google.api.client.util.Preconditions; -import com.google.api.client.util.Strings; import com.google.api.client.util.store.FileDataStoreFactory; import com.google.cloud.dataflow.sdk.options.GcpOptions; @@ -50,14 +49,15 @@ public class Credentials { private static final Logger LOG = LoggerFactory.getLogger(Credentials.class); - /** OAuth 2.0 scopes used by a local worker (not on GCE). - * The scope cloud-platform provides access to all Cloud Platform resources. - * cloud-platform isn't sufficient yet for talking to datastore so we request - * those resources separately. - * - * Note that trusted scope relationships don't apply to OAuth tokens, so for - * services we access directly (GCS) as opposed to through the backend - * (BigQuery, GCE), we need to explicitly request that scope. + /** + * OAuth 2.0 scopes used by a local worker (not on GCE). + * The scope cloud-platform provides access to all Cloud Platform resources. + * cloud-platform isn't sufficient yet for talking to datastore so we request + * those resources separately. + *

    + * Note that trusted scope relationships don't apply to OAuth tokens, so for + * services we access directly (GCS) as opposed to through the backend + * (BigQuery, GCE), we need to explicitly request that scope. */ private static final List SCOPES = Arrays.asList( "https://www.googleapis.com/auth/cloud-platform", @@ -74,8 +74,8 @@ public String getRedirectUri() { /** * Initializes OAuth2 credentials. - * - * This can use 4 different mechanisms for obtaining a credential: + *

    + * This can use 3 different mechanisms for obtaining a credential: *

      *
    1. * It can fetch the @@ -83,11 +83,6 @@ public String getRedirectUri() { * application default credentials. *
    2. *
    3. - * It can run the gcloud tool in a subprocess to obtain a credential. - * This is the preferred mechanism. The property "gcloud_path" can be - * used to specify where we search for gcloud data. - *
    4. - *
    5. * The user can specify a client secrets file and go through the OAuth2 * webflow. The credential will then be cached in the user's home * directory for reuse. Provide the property "secrets_file" to use this @@ -101,8 +96,8 @@ public String getRedirectUri() { *
    * The default mechanism is to use the * - * application default credentials falling back to gcloud. The other options can be - * used by providing the corresponding properties. + * application default credentials. The other options can be used by providing the + * corresponding properties. */ public static Credential getCredential(GcpOptions options) throws IOException, GeneralSecurityException { @@ -124,11 +119,12 @@ public static Credential getCredential(GcpOptions options) try { return GoogleCredential.getApplicationDefault().createScoped(SCOPES); } catch (IOException e) { - LOG.debug("Failed to get application default credentials, falling back to gcloud."); + throw new RuntimeException("Unable to get application default credentials. Please see " + + "https://developers.google.com/accounts/docs/application-default-credentials " + + "for details on how to specify credentials. This version of the SDK is " + + "dependent on the gcloud core component version 2015.02.05 or newer to " + + "be able to get credentials from the currently authorized user via gcloud auth.", e); } - - String gcloudPath = options.getGCloudPath(); - return getCredentialFromGCloud(gcloudPath); } /** @@ -149,29 +145,6 @@ private static Credential getCredentialFromFile( return credential; } - /** - * Loads OAuth2 credential from GCloud utility. - */ - private static Credential getCredentialFromGCloud(String gcloudPath) - throws IOException, GeneralSecurityException { - GCloudCredential credential; - HttpTransport transport = GoogleNetHttpTransport.newTrustedTransport(); - if (Strings.isNullOrEmpty(gcloudPath)) { - credential = new GCloudCredential(transport); - } else { - credential = new GCloudCredential(gcloudPath, transport); - } - - try { - credential.refreshToken(); - } catch (IOException e) { - throw new RuntimeException("Could not obtain credential using gcloud", e); - } - - LOG.info("Got user credential from GCloud"); - return credential; - } - /** * Loads OAuth2 credential from client secrets, which may require an * interactive authorization prompt. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GCloudCredential.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GCloudCredential.java deleted file mode 100644 index a3a3fd2eb5bf1..0000000000000 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GCloudCredential.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * Copyright (C) 2014 Google Inc. - * - * Licensed 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 com.google.cloud.dataflow.sdk.util; - -import com.google.api.client.auth.oauth2.BearerToken; -import com.google.api.client.auth.oauth2.Credential; -import com.google.api.client.auth.oauth2.TokenResponse; -import com.google.api.client.http.HttpTransport; -import com.google.api.client.util.IOUtils; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.util.Arrays; - -/** - * A credential object which uses the GCloud command line tool to get - * an access token. - */ -public class GCloudCredential extends Credential { - private static final String DEFAULT_GCLOUD_BINARY = "gcloud"; - private final String binary; - - public GCloudCredential(HttpTransport transport) { - this(DEFAULT_GCLOUD_BINARY, transport); - } - - /** - * Path to the GCloud binary. - */ - public GCloudCredential(String binary, HttpTransport transport) { - super(new Builder(BearerToken.authorizationHeaderAccessMethod()) - .setTransport(transport)); - - this.binary = binary; - } - - private String readStream(InputStream stream) throws IOException { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - IOUtils.copy(stream, baos); - return baos.toString("UTF-8"); - } - - @Override - protected TokenResponse executeRefreshToken() throws IOException { - TokenResponse response = new TokenResponse(); - - ProcessBuilder builder = new ProcessBuilder(); - // ProcessBuilder will search the path automatically for the binary - // GCLOUD_BINARY. - builder.command(Arrays.asList(binary, "auth", "print-access-token")); - Process process = builder.start(); - - try { - process.waitFor(); - } catch (InterruptedException e) { - throw new RuntimeException( - "Could not obtain an access token using gcloud; timed out waiting " + - "for gcloud."); - } - - if (process.exitValue() != 0) { - String output; - try { - output = readStream(process.getErrorStream()); - } catch (IOException e) { - throw new RuntimeException( - "Could not obtain an access token using gcloud."); - } - - throw new RuntimeException( - "Could not obtain an access token using gcloud. Result of " + - "invoking gcloud was:\n" + output); - } - - String output; - try { - output = readStream(process.getInputStream()); - } catch (IOException e) { - throw new RuntimeException( - "Could not obtain an access token using gcloud. We encountered an " + - "an error trying to read stdout.", e); - } - String[] lines = output.split("\n"); - - if (lines.length != 1) { - throw new RuntimeException( - "Could not obtain an access token using gcloud. Result of " + - "invoking gcloud was:\n" + output); - } - - // Access token should be good for 5 minutes. - Long expiresInSeconds = 5L * 60; - response.setExpiresInSeconds(expiresInSeconds); - response.setAccessToken(output.trim()); - - return response; - } -} From 0f05e56cb04600b3851ffee5c93b22dd01fe7e86 Mon Sep 17 00:00:00 2001 From: relax Date: Tue, 10 Feb 2015 15:47:40 -0800 Subject: [PATCH 0151/1541] Allows users of PubsubIO to specify which pubsub labels are used to propagate record timestamps and record ids. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86030744 --- .../cloud/dataflow/sdk/io/PubsubIO.java | 193 +++++++++++++++++- .../runners/dataflow/PubsubIOTranslator.java | 15 ++ .../dataflow/sdk/util/PropertyNames.java | 3 + 3 files changed, 202 insertions(+), 9 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index 4a379920249e7..2414feeb34f9f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -153,6 +153,10 @@ public static Bound named(String name) { *
  9. Must end with a letter or a number.
  10. *
  11. Cannot begin with 'goog' prefix.
  12. * + * + * Dataflow will start reading data published on this topic from the time the pipeline is + * started. Any data published on the topic before the pipeline is started will not be read + * by Dataflow. */ public static Bound topic(String topic) { return new Bound().topic(topic); @@ -180,6 +184,44 @@ public static Bound subscription(String subscription) { return new Bound().subscription(subscription); } + /** + * Creates and returns a PubsubIO.Read PTransform where record timestamps are expected + * to be provided using the PubSub labeling API. The {@code } parameter + * specifies the label name. The label value sent to PubsSub is a numerical value representing + * the number of milliseconds since the Unix epoch. For example, if using the joda time classes, + * org.joda.time.Instant.getMillis() returns the correct value for this label. + * + *

    If {@code } is not provided, the system will generate record timestamps + * the first time it sees each record. All windowing will be done relative to these timestamps. + * Windows are closed based on an estimate of when this source has finished producing data for + * a timestamp range, which means that late data can arrive after a window has been closed. The + * {#dropLateData} field allows you to control what to do with late data. + */ + public static Bound timestampLabel(String timestampLabel) { + return new Bound().timestampLabel(timestampLabel); + } + + /** + * If true, then late-arriving data from this source will be dropped. + */ + public static Bound dropLateData(boolean dropLateData) { + return new Bound().dropLateData(dropLateData); + } + + /** + * Creates and returns a PubSubIO.Read PTransform where unique record identifiers are + * expected to be provided using the PubSub labeling API. The {@code } parameter + * specifies the label name. The label value sent to PubSub can be any string value that + * uniquely identifies this record. + * + *

    If idLabel is not provided, Dataflow cannot guarantee that no duplicate data will be + * delivered on the PubSub stream. In this case, deduplication of the stream will be + * stricly best effort. + */ + public static Bound idLabel(String idLabel) { + return new Bound().idLabel(idLabel); + } + /** * A PTransform that reads from a PubSub source and returns * a unbounded PCollection containing the items from the stream. @@ -191,10 +233,22 @@ public static class Bound String topic; /** The Pubsub subscription to read from. */ String subscription; + /** The Pubsub label to read timestamps from. */ + String timestampLabel; + Boolean dropLateData; + /** This is set for backwards compatibility with old services. If dropLateData is not + * explicitly called, then we won't forward that parameter to the service. */ + Boolean dropLateDataExplicit; + /** The Pubsub label to read ids from. */ + String idLabel; + + Bound() { + this.dropLateData = true; + this.dropLateDataExplicit = false; + } - Bound() {} - - Bound(String name, String subscription, String topic) { + Bound(String name, String subscription, String topic, String timestampLabel, + boolean dropLateData, boolean dropLateDataExplicit, String idLabel) { super(name); if (subscription != null) { Validator.validateSubscriptionName(subscription); @@ -204,18 +258,63 @@ public static class Bound } this.subscription = subscription; this.topic = topic; + this.timestampLabel = timestampLabel; + this.dropLateData = dropLateData; + this.dropLateDataExplicit = dropLateDataExplicit; + this.idLabel = idLabel; } + /** + * Returns a new TextIO.Read PTransform that's like this one but with the given + * step name. Does not modify the object. + */ public Bound named(String name) { - return new Bound(name, subscription, topic); + return new Bound(name, subscription, topic, timestampLabel, dropLateData, + dropLateDataExplicit, idLabel); } + /** + * Returns a new TextIO.Read PTransform that's like this one but reading from the + * given subscription. Does not modify the object. + */ public Bound subscription(String subscription) { - return new Bound(name, subscription, topic); + return new Bound(name, subscription, topic, timestampLabel, dropLateData, + dropLateDataExplicit, idLabel); } + /** + * Returns a new TextIO.Read PTransform that's like this one but reading from the + * give topic. Does not modify the object. + */ public Bound topic(String topic) { - return new Bound(name, subscription, topic); + return new Bound(name, subscription, topic, timestampLabel, dropLateData, + dropLateDataExplicit, idLabel); + } + + /** + * Returns a new TextIO.Read PTransform that's like this one but reading timestamps + * from the given PubSub label. Does not modify the object. + */ + public Bound timestampLabel(String timestampLabel) { + return new Bound(name, subscription, topic, timestampLabel, dropLateData, + dropLateDataExplicit, idLabel); + } + + /** + * Returns a new TextIO.Read PTransform that's like this one but with the specified + * setting for dropLateData. Does not modify the object. + */ + public Bound dropLateData(boolean dropLateData) { + return new Bound(name, subscription, topic, timestampLabel, dropLateData, true, idLabel); + } + + /** + * Returns a new TextIO.Read PTransform that's like this one but reading unique ids + * from the given PubSub label. Does not modify the object. + */ + public Bound idLabel(String idLabel) { + return new Bound(name, subscription, topic, timestampLabel, dropLateData, + dropLateDataExplicit, idLabel); } @Override @@ -250,6 +349,22 @@ public String getSubscription() { return subscription; } + public String getTimestampLabel() { + return timestampLabel; + } + + public boolean getDropLateData() { + return dropLateData; + } + + public boolean getDropLateDataExplicit() { + return dropLateDataExplicit; + } + + public String getIdLabel() { + return idLabel; + } + static { // TODO: Figure out how to make this work under // DirectPipelineRunner. @@ -278,6 +393,30 @@ public static Bound topic(String topic) { return new Bound().topic(topic); } + /** + * If specified, Dataflow will add a Pubsub label to each output record specifying the logical + * timestamp of the record. {@code } determines the label name. The label value + * is a numerical value representing the number of milliseconds since the Unix epoch. For + * example, if using the joda time classes, the org.joda.time.Instant(long) constructor can be + * used to parse this value. If the output from this sink is being read by another Dataflow + * source, then PubsubIO.Read.timestampLabel can be used to ensure that the other source reads + * these timestamps from the appropriate label. + */ + public static Bound timestampLabel(String timestampLabel) { + return new Bound().timestampLabel(timestampLabel); + } + + /** + * If specified, Dataflow will add a Pubsub label to each output record containing a unique + * identifier for that record. {@code } determines the label name. The label value + * is an opaque string value. This is useful if the the output from this sink is being read + * by another Dataflow source, in which case PubsubIO.Read.idLabel can be used to ensure that + * the other source reads these ids from the appropriate label. + */ + public static Bound idLabel(String idLabel) { + return new Bound().idLabel(idLabel); + } + /** * A PTransfrom that writes a unbounded {@code PCollection} * to a PubSub stream. @@ -287,23 +426,51 @@ public static class Bound extends PTransform, PDone> { /** The Pubsub topic to publish to. */ String topic; + String timestampLabel; + String idLabel; Bound() {} - Bound(String name, String topic) { + Bound(String name, String topic, String timestampLabel, String idLabel) { super(name); if (topic != null) { Validator.validateTopicName(topic); this.topic = topic; } + this.timestampLabel = timestampLabel; + this.idLabel = idLabel; } + /** + * Returns a new TextIO.Write PTransform that's like this one but with the given step + * name. Does not modify the object. + */ public Bound named(String name) { - return new Bound(name, topic); + return new Bound(name, topic, timestampLabel, idLabel); } + /** + * Returns a new TextIO.Write PTransform that's like this one but writing to the given + * topic. Does not modify the object. + */ public Bound topic(String topic) { - return new Bound(name, topic); + return new Bound(name, topic, timestampLabel, idLabel); + } + + /** + * Returns a new TextIO.Write PTransform that's like this one but publishing timestamps + * to the given PubSub label. Does not modify the object. + */ + public Bound timestampLabel(String timestampLabel) { + return new Bound(name, topic, timestampLabel, idLabel); + } + + /** + * Returns a new TextIO.Write PTransform that's like this one but publishing record ids + * to the given PubSub label. Does not modify the object. + */ + public Bound idLabel(String idLabel) { + return new Bound(name, topic, timestampLabel, idLabel); } @Override @@ -327,6 +494,14 @@ public String getTopic() { return topic; } + public String getTimestampLabel() { + return timestampLabel; + } + + public String getIdLabel() { + return idLabel; + } + static { // TODO: Figure out how to make this work under // DirectPipelineRunner. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java index 706397bddd37b..8b297d6f31e37 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/PubsubIOTranslator.java @@ -57,6 +57,15 @@ private void translateReadHelper( if (transform.getSubscription() != null) { context.addInput(PropertyNames.PUBSUB_SUBSCRIPTION, transform.getSubscription()); } + if (transform.getTimestampLabel() != null) { + context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel()); + } + if (transform.getDropLateDataExplicit()) { + context.addInput(PropertyNames.PUBSUB_DROP_LATE_DATA, transform.getDropLateData()); + } + if (transform.getIdLabel() != null) { + context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel()); + } context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); // TODO: Orderedness? } @@ -83,6 +92,12 @@ private void translateWriteHelper( context.addStep(transform, "ParallelWrite"); context.addInput(PropertyNames.FORMAT, "pubsub"); context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic()); + if (transform.getTimestampLabel() != null) { + context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel()); + } + if (transform.getIdLabel() != null) { + context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel()); + } context.addEncodingInput( WindowedValue.getValueOnlyCoder(transform.getInput().getCoder())); context.addInput(PropertyNames.PARALLEL_INPUT, transform.getInput()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java index 0afe5ae411901..a22f7893a334c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PropertyNames.java @@ -68,7 +68,10 @@ public class PropertyNames { public static final String OUTPUT_NAME = "output_name"; public static final String PARALLEL_INPUT = "parallel_input"; public static final String PHASE = "phase"; + public static final String PUBSUB_DROP_LATE_DATA = "pubsub_drop_late_data"; + public static final String PUBSUB_ID_LABEL = "pubsub_id_label"; public static final String PUBSUB_SUBSCRIPTION = "pubsub_subscription"; + public static final String PUBSUB_TIMESTAMP_LABEL = "pubsub_timestamp_label"; public static final String PUBSUB_TOPIC = "pubsub_topic"; public static final String SCALAR_FIELD_NAME = "value"; public static final String SERIALIZED_FN = "serialized_fn"; From 5a11734d638b53662c036ead93094c5a7fdd9994 Mon Sep 17 00:00:00 2001 From: lcwik Date: Wed, 11 Feb 2015 14:54:15 -0800 Subject: [PATCH 0152/1541] Re-add support for getting credentials from gcloud for older versions of gcloud. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86122002 --- .../dataflow/sdk/options/GcpOptions.java | 5 + .../cloud/dataflow/sdk/util/Credentials.java | 63 +++++++--- .../dataflow/sdk/util/GCloudCredential.java | 113 ++++++++++++++++++ 3 files changed, 163 insertions(+), 18 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GCloudCredential.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java index 571dba49cd0e2..0d824405b9075 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java @@ -96,6 +96,11 @@ public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions { String getServiceAccountName(); void setServiceAccountName(String value); + @Description("The path to the gcloud binary. " + + " Default is to search the system path.") + String getGCloudPath(); + void setGCloudPath(String value); + /** * Directory for storing dataflow credentials. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java index c416eae3d844e..e37275cdb936f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java @@ -28,6 +28,7 @@ import com.google.api.client.json.JsonFactory; import com.google.api.client.json.jackson2.JacksonFactory; import com.google.api.client.util.Preconditions; +import com.google.api.client.util.Strings; import com.google.api.client.util.store.FileDataStoreFactory; import com.google.cloud.dataflow.sdk.options.GcpOptions; @@ -49,15 +50,14 @@ public class Credentials { private static final Logger LOG = LoggerFactory.getLogger(Credentials.class); - /** - * OAuth 2.0 scopes used by a local worker (not on GCE). - * The scope cloud-platform provides access to all Cloud Platform resources. - * cloud-platform isn't sufficient yet for talking to datastore so we request - * those resources separately. - *

    - * Note that trusted scope relationships don't apply to OAuth tokens, so for - * services we access directly (GCS) as opposed to through the backend - * (BigQuery, GCE), we need to explicitly request that scope. + /** OAuth 2.0 scopes used by a local worker (not on GCE). + * The scope cloud-platform provides access to all Cloud Platform resources. + * cloud-platform isn't sufficient yet for talking to datastore so we request + * those resources separately. + * + * Note that trusted scope relationships don't apply to OAuth tokens, so for + * services we access directly (GCS) as opposed to through the backend + * (BigQuery, GCE), we need to explicitly request that scope. */ private static final List SCOPES = Arrays.asList( "https://www.googleapis.com/auth/cloud-platform", @@ -74,8 +74,8 @@ public String getRedirectUri() { /** * Initializes OAuth2 credentials. - *

    - * This can use 3 different mechanisms for obtaining a credential: + * + * This can use 4 different mechanisms for obtaining a credential: *

      *
    1. * It can fetch the @@ -83,6 +83,11 @@ public String getRedirectUri() { * application default credentials. *
    2. *
    3. + * It can run the gcloud tool in a subprocess to obtain a credential. + * This is the preferred mechanism. The property "gcloud_path" can be + * used to specify where we search for gcloud data. + *
    4. + *
    5. * The user can specify a client secrets file and go through the OAuth2 * webflow. The credential will then be cached in the user's home * directory for reuse. Provide the property "secrets_file" to use this @@ -96,8 +101,8 @@ public String getRedirectUri() { *
    * The default mechanism is to use the * - * application default credentials. The other options can be used by providing the - * corresponding properties. + * application default credentials falling back to gcloud. The other options can be + * used by providing the corresponding properties. */ public static Credential getCredential(GcpOptions options) throws IOException, GeneralSecurityException { @@ -119,12 +124,11 @@ public static Credential getCredential(GcpOptions options) try { return GoogleCredential.getApplicationDefault().createScoped(SCOPES); } catch (IOException e) { - throw new RuntimeException("Unable to get application default credentials. Please see " - + "https://developers.google.com/accounts/docs/application-default-credentials " - + "for details on how to specify credentials. This version of the SDK is " - + "dependent on the gcloud core component version 2015.02.05 or newer to " - + "be able to get credentials from the currently authorized user via gcloud auth.", e); + LOG.debug("Failed to get application default credentials, falling back to gcloud."); } + + String gcloudPath = options.getGCloudPath(); + return getCredentialFromGCloud(gcloudPath); } /** @@ -145,6 +149,29 @@ private static Credential getCredentialFromFile( return credential; } + /** + * Loads OAuth2 credential from GCloud utility. + */ + private static Credential getCredentialFromGCloud(String gcloudPath) + throws IOException, GeneralSecurityException { + GCloudCredential credential; + HttpTransport transport = GoogleNetHttpTransport.newTrustedTransport(); + if (Strings.isNullOrEmpty(gcloudPath)) { + credential = new GCloudCredential(transport); + } else { + credential = new GCloudCredential(gcloudPath, transport); + } + + try { + credential.refreshToken(); + } catch (IOException e) { + throw new RuntimeException("Could not obtain credential using gcloud", e); + } + + LOG.info("Got user credential from GCloud"); + return credential; + } + /** * Loads OAuth2 credential from client secrets, which may require an * interactive authorization prompt. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GCloudCredential.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GCloudCredential.java new file mode 100644 index 0000000000000..a3a3fd2eb5bf1 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GCloudCredential.java @@ -0,0 +1,113 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.api.client.auth.oauth2.BearerToken; +import com.google.api.client.auth.oauth2.Credential; +import com.google.api.client.auth.oauth2.TokenResponse; +import com.google.api.client.http.HttpTransport; +import com.google.api.client.util.IOUtils; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; + +/** + * A credential object which uses the GCloud command line tool to get + * an access token. + */ +public class GCloudCredential extends Credential { + private static final String DEFAULT_GCLOUD_BINARY = "gcloud"; + private final String binary; + + public GCloudCredential(HttpTransport transport) { + this(DEFAULT_GCLOUD_BINARY, transport); + } + + /** + * Path to the GCloud binary. + */ + public GCloudCredential(String binary, HttpTransport transport) { + super(new Builder(BearerToken.authorizationHeaderAccessMethod()) + .setTransport(transport)); + + this.binary = binary; + } + + private String readStream(InputStream stream) throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + IOUtils.copy(stream, baos); + return baos.toString("UTF-8"); + } + + @Override + protected TokenResponse executeRefreshToken() throws IOException { + TokenResponse response = new TokenResponse(); + + ProcessBuilder builder = new ProcessBuilder(); + // ProcessBuilder will search the path automatically for the binary + // GCLOUD_BINARY. + builder.command(Arrays.asList(binary, "auth", "print-access-token")); + Process process = builder.start(); + + try { + process.waitFor(); + } catch (InterruptedException e) { + throw new RuntimeException( + "Could not obtain an access token using gcloud; timed out waiting " + + "for gcloud."); + } + + if (process.exitValue() != 0) { + String output; + try { + output = readStream(process.getErrorStream()); + } catch (IOException e) { + throw new RuntimeException( + "Could not obtain an access token using gcloud."); + } + + throw new RuntimeException( + "Could not obtain an access token using gcloud. Result of " + + "invoking gcloud was:\n" + output); + } + + String output; + try { + output = readStream(process.getInputStream()); + } catch (IOException e) { + throw new RuntimeException( + "Could not obtain an access token using gcloud. We encountered an " + + "an error trying to read stdout.", e); + } + String[] lines = output.split("\n"); + + if (lines.length != 1) { + throw new RuntimeException( + "Could not obtain an access token using gcloud. Result of " + + "invoking gcloud was:\n" + output); + } + + // Access token should be good for 5 minutes. + Long expiresInSeconds = 5L * 60; + response.setExpiresInSeconds(expiresInSeconds); + response.setAccessToken(output.trim()); + + return response; + } +} From eea6c20cfb30a7388ecea4f89cc4001ac96bcfec Mon Sep 17 00:00:00 2001 From: robertwb Date: Wed, 11 Feb 2015 15:16:33 -0800 Subject: [PATCH 0153/1541] Internal support for demultiplexing reiterables. No externally visible change. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86124146 --- .../sdk/util/common/TaggedReiteratorList.java | 152 +++++++++++++ .../util/common/TaggedReiteratorListTest.java | 209 ++++++++++++++++++ 2 files changed, 361 insertions(+) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/TaggedReiteratorList.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/TaggedReiteratorListTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/TaggedReiteratorList.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/TaggedReiteratorList.java new file mode 100644 index 0000000000000..c11f5098306bb --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/TaggedReiteratorList.java @@ -0,0 +1,152 @@ +/******************************************************************************* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import java.util.AbstractList; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * Provides a view a of re-iterable of tagged values, with monotonically + * increasing tags, as a list of tagged re-iterables. + * + * This class, and the returned iterators, are not threadsafe. + */ +public class TaggedReiteratorList extends AbstractList> { + + /** + * Interface for extracting the tag and value from an opaque element. + */ + public interface TagExtractor { + public int getTag(T elem); + public Object getValue(T elem); + } + + private final TagExtractor extractor; + + private final List> starts; + + private final int size; + + public TaggedReiteratorList(Reiterator taggedReiterator, + TagExtractor extractor) { + this(taggedReiterator, extractor, -1); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + public TaggedReiteratorList(Reiterator taggedReiterator, + TagExtractor extractor, + int size) { + starts = new ArrayList<>(); + starts.add(new PeekingReiterator((Reiterator) taggedReiterator)); + this.extractor = (TagExtractor) extractor; + this.size = size; + } + + @Override + public Reiterator get(int tag) { + return new SubIterator(tag); + } + + @Override + public int size() { + if (size == -1) { + throw new UnsupportedOperationException(); + } else { + return size; + } + } + + private PeekingReiterator getStart(int tag) { + if (tag >= starts.size()) { + PeekingReiterator start = getStart(tag - 1); + while (start.hasNext() && extractor.getTag(start.peek()) < tag) { + start.next(); + } + starts.add(start); + } + // Use the stored value, store a copy. + return starts.set(tag, starts.get(tag).copy()); + } + + private static final PeekingReiterator EMPTY_TAIL = + new PeekingReiterator( + new Reiterator() { + public boolean hasNext() { return false; } + public Object next() { throw new NoSuchElementException(); } + public void remove() { throw new IllegalArgumentException(); } + public Reiterator copy() { throw new IllegalArgumentException(); } + }); + + private class SubIterator implements Reiterator { + + private final int tag; + private PeekingReiterator iterator; + + private SubIterator(int tag) { + this(tag, null); + } + + private SubIterator(int tag, PeekingReiterator iterator) { + this.tag = tag; + this.iterator = iterator; + } + + @Override + public boolean hasNext() { + if (iterator == null) { + iterator = getStart(tag); + } + if (iterator.hasNext() && extractor.getTag(iterator.peek()) == tag) { + return true; + } else { + if (iterator != EMPTY_TAIL) { + // Set up for the common case that we're iterating over the + // next tag soon. + if (starts.size() > tag + 1) { + starts.set(tag + 1, iterator); + } else { + starts.add(tag + 1, iterator); + } + iterator = EMPTY_TAIL; + } + return false; + } + } + + @Override + public Object next() { + if (hasNext()) { + return extractor.getValue(iterator.next()); + } else { + throw new NoSuchElementException(); + } + } + + @Override + public void remove() { + throw new IllegalArgumentException(); + } + + @Override + public Reiterator copy() { + return new SubIterator( + tag, iterator == null || iterator == EMPTY_TAIL ? iterator : iterator.copy()); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/TaggedReiteratorListTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/TaggedReiteratorListTest.java new file mode 100644 index 0000000000000..a3682c25105c2 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/TaggedReiteratorListTest.java @@ -0,0 +1,209 @@ +/******************************************************************************* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +/** + * Tests for {@link TaggedReiteratorList}. + */ +@RunWith(JUnit4.class) +public class TaggedReiteratorListTest { + + @Test + public void testSingleIterator() { + TaggedReiteratorList iter = create(new String[] {"a", "b", "c"}); + assertEquals(iter.get(0), "a", "b", "c"); + assertEquals(iter.get(0), "a", "b", "c"); + assertEquals(iter.get(1) /*empty*/); + assertEquals(iter.get(0), "a", "b", "c"); + } + + @Test + public void testSequentialAccess() { + TaggedReiteratorList iter = create(3, new String[] {"a", "b", "c"}); + for (int i = 0; i < 2; i++) { + assertEquals(iter.get(0), "a0", "b0", "c0"); + assertEquals(iter.get(1), "a1", "b1", "c1"); + assertEquals(iter.get(2), "a2", "b2", "c2"); + } + for (int i = 0; i < 2; i++) { + assertEquals(iter.get(2), "a2", "b2", "c2"); + assertEquals(iter.get(1), "a1", "b1", "c1"); + assertEquals(iter.get(0), "a0", "b0", "c0"); + } + } + + @Test + public void testRandomAccess() { + TaggedReiteratorList iter = create(6, new String[] {"a", "b"}); + assertEquals(iter.get(3), "a3", "b3"); + assertEquals(iter.get(1), "a1", "b1"); + assertEquals(iter.get(5), "a5", "b5"); + assertEquals(iter.get(0), "a0", "b0"); + assertEquals(iter.get(4), "a4", "b4"); + assertEquals(iter.get(4), "a4", "b4"); + assertEquals(iter.get(1), "a1", "b1"); + } + + @Test + public void testPartialIteration() { + TaggedReiteratorList iter = create(6, new String[] {"a", "b", "c"}); + Iterator get0 = iter.get(0); + Iterator get1 = iter.get(1); + Iterator get3 = iter.get(3); + assertEquals(asList(get0, 1), "a0"); + assertEquals(asList(get1, 2), "a1", "b1"); + assertEquals(asList(get3, 3), "a3", "b3", "c3"); + Iterator get2 = iter.get(2); + Iterator get0Again = iter.get(0); + assertEquals(asList(get0, 1), "b0"); + assertEquals(get2, "a2", "b2", "c2"); + assertEquals(get0Again, "a0", "b0", "c0"); + assertEquals(asList(get0), "c0"); + Iterator get4 = iter.get(4); + assertEquals(get4, "a4", "b4", "c4"); + assertEquals(get4 /*empty*/); + assertEquals(iter.get(4), "a4", "b4", "c4"); + } + + @Test + public void testNextIteration() { + TaggedReiteratorList iter = create(2, new String[] {"a", "b", "c"}); + Reiterator get0 = iter.get(0); + assertEquals(get0, "a0", "b0", "c0"); + Iterator get1 = iter.get(1); + Assert.assertEquals(get1.next(), "a1"); + assertEquals(get0.copy() /*empty*/); + Assert.assertEquals(get1.next(), "b1"); + assertEquals(iter.get(1), "a1", "b1", "c1"); + } + + @Test + public void testEmpties() { + TaggedReiteratorList iter = create(new String[] {}, + new String[] {"a", "b", "c"}, + new String[] {}, + new String[] {}, + new String[] {"d"}); + assertEquals(iter.get(2) /*empty*/); + assertEquals(iter.get(1), "a", "b", "c"); + assertEquals(iter.get(2) /*empty*/); + assertEquals(iter.get(0) /*empty*/); + assertEquals(iter.get(2) /*empty*/); + assertEquals(iter.get(4), "d"); + assertEquals(iter.get(3) /*empty*/); + } + + ///////////////////////////////////////////////////////////////////////////// + // Helpers + + private TaggedReiteratorList create(String[]... values) { + ArrayList taggedValues = new ArrayList<>(); + for (int tag = 0; tag < values.length; tag++) { + for (String value : values[tag]) { + taggedValues.add(new TaggedValue(tag, value)); + } + } + return new TaggedReiteratorList( + new TestReiterator(taggedValues.toArray(new TaggedValue[0])), + new TaggedValueExtractor()); + } + + private TaggedReiteratorList create(int repeat, String... values) { + ArrayList taggedValues = new ArrayList<>(); + for (int tag = 0; tag < repeat; tag++) { + for (String value : values) { + taggedValues.add(new TaggedValue(tag, value + tag)); + } + } + return new TaggedReiteratorList( + new TestReiterator(taggedValues.toArray(new TaggedValue[0])), + new TaggedValueExtractor()); + } + + private List asList(Iterator iter) { + return asList(iter, Integer.MAX_VALUE); + } + + private List asList(Iterator iter, int limit) { + List list = new ArrayList<>(); + for (int i = 0; i < limit && iter.hasNext(); i++) { + list.add(iter.next()); + } + return list; + } + + private void assertEquals(Iterator actual, Object... expected) { + assertEquals(asList(actual), expected); + } + + private void assertEquals(List actual, Object... expected) { + Assert.assertEquals(Arrays.asList(expected), actual); + } + + private static class TestReiterator implements Reiterator { + private final TaggedValue[] values; + private int pos = 0; + public TestReiterator(TaggedValue... values) { + this(values, 0); + } + private TestReiterator(TaggedValue[] values, int pos) { + this.values = values; + this.pos = pos; + } + public boolean hasNext() { + return pos < values.length; + } + public TaggedValue next() { + return values[pos++]; + } + public void remove() { + throw new IllegalArgumentException(); + } + public TestReiterator copy() { + return new TestReiterator(values, pos); + } + } + + private static class TaggedValueExtractor + implements TaggedReiteratorList.TagExtractor { + public int getTag(TaggedValue elem) { + return elem.tag; + } + public String getValue(TaggedValue elem) { + return elem.value; + } + } + + private static class TaggedValue { + public final int tag; + public final String value; + public TaggedValue(int tag, String value) { + this.tag = tag; + this.value = value; + } + } +} From d1ec460f2324edbda3288ef73a6c119d12085132 Mon Sep 17 00:00:00 2001 From: peihe Date: Wed, 11 Feb 2015 16:04:47 -0800 Subject: [PATCH 0154/1541] Test: add testSessionsCombine to cover GroupAlsoByWindowsDoFn.processElementViaWindowSet(). [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86128434 --- .../dataflow/sdk/transforms/CombineTest.java | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java index 403190b6115aa..284ae327eabcf 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java @@ -37,6 +37,7 @@ import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; @@ -190,7 +191,7 @@ private void runTestAccumulatingCombine(KV[] table, @Test @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) - public void testWindowedCombine() { + public void testFixedWindowsCombine() { Pipeline p = TestPipeline.create(); PCollection> input = @@ -215,6 +216,32 @@ public void testWindowedCombine() { p.run(); } + @Test + @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) + public void testSessionsCombine() { + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.timestamped(Arrays.asList(TABLE), + Arrays.asList(0L, 4L, 7L, 10L, 16L))) + .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())) + .apply(Window.>into(Sessions.withGapDuration(Duration.millis(5)))); + + PCollection sum = input + .apply(Values.create()) + .apply(Combine.globally(new SumInts())); + + PCollection> sumPerKey = input + .apply(Combine.perKey(new SumInts())); + + DataflowAssert.that(sum).containsInAnyOrder(7, 13); + DataflowAssert.that(sumPerKey).containsInAnyOrder( + KV.of("a", 6), + KV.of("b", 1), + KV.of("b", 13)); + p.run(); + } + @Test @Category(com.google.cloud.dataflow.sdk.testing.RunnableOnService.class) public void testWindowedCombineEmpty() { From 2a71b265199a0a9bd61d37c5497678dfa5ba9596 Mon Sep 17 00:00:00 2001 From: mattlang Date: Wed, 11 Feb 2015 17:02:12 -0800 Subject: [PATCH 0155/1541] Do not try to decompress gzip input streams that are already decompressed. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86133530 --- .../google/cloud/dataflow/sdk/io/TextIO.java | 14 +++++++++- .../cloud/dataflow/sdk/io/TextIOTest.java | 26 +++++++++++++++++++ 2 files changed, 39 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java index 487dcc371af31..40c24353c1d7b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java @@ -32,11 +32,13 @@ import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PDone; import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.common.primitives.Ints; import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; import java.io.IOException; import java.io.InputStream; +import java.io.PushbackInputStream; import java.util.List; import java.util.regex.Pattern; import java.util.zip.GZIPInputStream; @@ -647,7 +649,17 @@ public static enum CompressionType implements FileBasedReader.DecompressingStrea GZIP(".gz") { @Override public InputStream createInputStream(InputStream inputStream) throws IOException { - return new GZIPInputStream(inputStream); + // Determine if the input stream is gzipped. The input stream returned from the + // GCS connector may already be decompressed, and no action is required. + PushbackInputStream stream = new PushbackInputStream(inputStream, 2); + byte[] headerBytes = new byte[2]; + int bytesRead = stream.read(headerBytes); + stream.unread(headerBytes, 0, bytesRead); + int header = Ints.fromBytes((byte) 0, (byte) 0, headerBytes[1], headerBytes[0]); + if (header == GZIPInputStream.GZIP_MAGIC) { + return new GZIPInputStream(stream); + } + return stream; } }, /** diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java index ead2e779eb026..ee54668db71ed 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java @@ -423,6 +423,7 @@ public void testCompressedRead() throws Exception { } } + DirectPipeline p = DirectPipeline.createForTest(); TextIO.Read.Bound read = @@ -434,4 +435,29 @@ public void testCompressedRead() throws Exception { assertThat(results.getPCollection(output), containsInAnyOrder(expected.toArray())); tmpFile.delete(); } + + @Test + public void testGZIPReadWhenUncompressed() throws Exception { + String[] lines = {"Meritorious condor", "Obnoxious duck"}; + File tmpFile = tmpFolder.newFile("test"); + String filename = tmpFile.getPath(); + + List expected = new ArrayList<>(); + try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) { + for (String line : lines) { + writer.println(line); + expected.add(line); + } + } + + DirectPipeline p = DirectPipeline.createForTest(); + TextIO.Read.Bound read = + TextIO.Read.from(filename).withCompressionType(CompressionType.GZIP); + PCollection output = p.apply(read); + + EvaluationResults results = p.run(); + + assertThat(results.getPCollection(output), containsInAnyOrder(expected.toArray())); + tmpFile.delete(); + } } From 21c8f75e2f43225db9972622078dd07e92396ad3 Mon Sep 17 00:00:00 2001 From: millsd Date: Thu, 12 Feb 2015 13:28:19 -0800 Subject: [PATCH 0156/1541] Fix units in log message in the StreamingDataflowWorker [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86208755 --- .../dataflow/sdk/runners/worker/StreamingDataflowWorker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java index c9167b13da975..f16200038f326 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -291,7 +291,7 @@ private void dispatchLoop() { while (rt.freeMemory() < rt.totalMemory() * PUSHBACK_THRESHOLD) { if (lastPushbackLog < (lastPushbackLog = System.currentTimeMillis()) - 60 * 1000) { LOG.warn("In pushback, not accepting new work. Free Memory: {}MB / {}MB", - rt.freeMemory(), rt.totalMemory()); + rt.freeMemory() / 1e6, rt.totalMemory() / 1e6); System.gc(); } sleep(10); From 544de16ca6b827be61db469d3bf14093c885a72a Mon Sep 17 00:00:00 2001 From: sisk Date: Fri, 13 Feb 2015 05:41:17 -0800 Subject: [PATCH 0157/1541] Removing log lines from workers that aren't relevant to users, also fix up logs that should stay. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86265043 --- .../dataflow/sdk/coders/CoderRegistry.java | 22 +++++++++---------- .../sdk/runners/worker/DataflowWorker.java | 13 +++++++---- .../runners/worker/DataflowWorkerHarness.java | 4 ++-- .../util/common/worker/MapTaskExecutor.java | 8 +++---- .../common/worker/WorkProgressUpdater.java | 10 ++++----- 5 files changed, 31 insertions(+), 26 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java index d2b61293c1dc6..37128918b5d71 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java @@ -161,20 +161,20 @@ public void registerCoder(Class clazz, coderClazz.getDeclaredMethod("of", factoryMethodArgTypes); } catch (NoSuchMethodException | SecurityException exn) { throw new IllegalArgumentException( - "cannot register Coder " + coderClazz + ": " + "Cannot register Coder " + coderClazz + ": " + "does not have an accessible method named 'of' with " + numTypeParameters + " arguments of Coder type", exn); } if (!Modifier.isStatic(factoryMethod.getModifiers())) { throw new IllegalArgumentException( - "cannot register Coder " + coderClazz + ": " + "Cannot register Coder " + coderClazz + ": " + "method named 'of' with " + numTypeParameters + " arguments of Coder type is not static"); } if (!coderClazz.isAssignableFrom(factoryMethod.getReturnType())) { throw new IllegalArgumentException( - "cannot register Coder " + coderClazz + ": " + "Cannot register Coder " + coderClazz + ": " + "method named 'of' with " + numTypeParameters + " arguments of Coder type does not return a " + coderClazz); } @@ -184,7 +184,7 @@ public void registerCoder(Class clazz, } } catch (SecurityException exn) { throw new IllegalArgumentException( - "cannot register Coder " + coderClazz + ": " + "Cannot register Coder " + coderClazz + ": " + "method named 'of' with " + numTypeParameters + " arguments of Coder type is not accessible", exn); @@ -200,7 +200,7 @@ public void registerCoder(Class clazz, "getInstanceComponents", clazz); } catch (NoSuchMethodException | SecurityException exn) { - LOG.warn("cannot find getInstanceComponents for class {}. This may limit the ability to" + LOG.warn("Cannot find getInstanceComponents for class {}. This may limit the ability to" + " infer a Coder for values of this type.", coderClazz, exn); } } @@ -212,7 +212,7 @@ public void registerCoder(Class rawClazz, CoderFactory coderFactory) { if (coderFactoryMap.put(rawClazz, coderFactory) != null) { throw new IllegalArgumentException( - "cannot register multiple default Coder factories for " + rawClazz); + "Cannot register multiple default Coder factories for " + rawClazz); } } @@ -497,7 +497,7 @@ public Coder create(List> typeArgumentCoders) { NullPointerException | ExceptionInInitializerError exn) { throw new IllegalStateException( - "error when invoking Coder factory method " + coderFactoryMethod, + "Error when invoking Coder factory method " + coderFactoryMethod, exn); } } @@ -506,7 +506,7 @@ public Coder create(List> typeArgumentCoders) { public List getInstanceComponents(Object value) { if (getComponentsMethod == null) { throw new IllegalStateException( - "no suitable static getInstanceComponents method available for " + "No suitable static getInstanceComponents method available for " + "Coder " + coderClazz); } @@ -521,7 +521,7 @@ public List getInstanceComponents(Object value) { | NullPointerException | ExceptionInInitializerError exn) { throw new IllegalStateException( - "error when invoking Coder getComponents method " + getComponentsMethod, + "Error when invoking Coder getComponents method " + getComponentsMethod, exn); } } @@ -540,7 +540,7 @@ static CoderFactory defaultCoderFactory(Class coderClazz, final Method coderF CoderFactory getDefaultCoderFactory(Class clazz) { CoderFactory coderFactory = coderFactoryMap.get(clazz); if (coderFactory == null) { - LOG.debug("No Coder registered for {}", clazz); + LOG.info("No Coder registered for {}", clazz); } return coderFactory; } @@ -582,7 +582,7 @@ Coder getDefaultCoder(Type type, Map> typeCoderBindings) { return null; } else { throw new RuntimeException( - "internal error: unexpected kind of Type: " + type); + "Internal error: unexpected kind of Type: " + type); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java index f1fe0d2f6ec01..fcd6ccd618a4e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java @@ -101,7 +101,7 @@ public boolean getAndPerformWork() throws IOException { * @throws IOException Only if the WorkUnitClient fails. */ private boolean doWork(WorkItem workItem) throws IOException { - LOG.info("Executing: {}", workItem); + LOG.debug("Executing: {}", workItem); WorkExecutor worker = null; try { @@ -132,13 +132,13 @@ private boolean doWork(WorkItem workItem) throws IOException { // Log all counter values for debugging purposes. CounterSet counters = worker.getOutputCounters(); for (Counter counter : counters) { - LOG.info("COUNTER {}.", counter); + LOG.trace("COUNTER {}.", counter); } // Log all metrics for debugging purposes. Collection> metrics = worker.getOutputMetrics(); for (Metric metric : metrics) { - LOG.info("METRIC {}: {}", metric.getName(), metric.getValue()); + LOG.trace("METRIC {}: {}", metric.getName(), metric.getValue()); } // stopReportingProgress can throw an exception if the final progress @@ -222,7 +222,12 @@ private void reportStatus(DataflowWorkerHarnessOptions options, String status, W @Nullable SourceFormat.OperationResponse operationResponse, @Nullable List errors, long finalReportIndex) throws IOException { - LOG.info("{} processing work item {}", status, uniqueId(workItem)); + String message = "{} processing work item {}"; + if (null != errors && errors.size() > 0) { + LOG.warn(message, status, uniqueId(workItem)); + } else { + LOG.debug(message, status, uniqueId(workItem)); + } WorkItemStatus workItemStatus = buildStatus(workItem, true/*completed*/, counters, metrics, options, null, null, operationResponse, errors, finalReportIndex); workUnitClient.reportWorkItemStatus(workItemStatus); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java index 4b0879e22c99d..3ac2895afcba4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java @@ -130,11 +130,11 @@ public Boolean call() throws Exception { } long endTime = DateTimeUtils.currentTimeMillis(); - LOG.info("processWork() start time: {}, end time: {}", + LOG.debug("Parallel worker thread processing start time: {}, end time: {}", ISODateTimeFormat.dateTime().print(startTime), ISODateTimeFormat.dateTime().print(endTime)); for (long completionTime : completionTimes) { - LOG.info("Duration: {}ms Wasted Time: {}ms", + LOG.debug("Worker thread execution time {}ms, idle time waiting for other work threads: {}ms", completionTime - startTime, endTime - completionTime); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java index 717c44aeec422..1d2595feda39c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java @@ -53,13 +53,13 @@ public MapTaskExecutor( @Override public void execute() throws Exception { - LOG.debug("executing map task"); + LOG.debug("Executing map task"); // Start operations, in reverse-execution-order, so that a // consumer is started before a producer might output to it. // Starting a root operation such as a ReadOperation does the work // of processing the input dataset. - LOG.debug("starting operations"); + LOG.debug("Starting operations"); ListIterator iterator = operations.listIterator(operations.size()); while (iterator.hasPrevious()) { Operation op = iterator.previous(); @@ -69,12 +69,12 @@ public void execute() throws Exception { // Finish operations, in forward-execution-order, so that a // producer finishes outputting to its consumers before those // consumers are themselves finished. - LOG.debug("finishing operations"); + LOG.debug("Finishing operations"); for (Operation op : operations) { op.finish(); } - LOG.debug("map task execution complete"); + LOG.debug("Map task execution complete"); // TODO: support for success / failure ports? } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java index f212d42e469f1..90dfb7727cdf5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/WorkProgressUpdater.java @@ -101,7 +101,7 @@ public void startReportingProgress() { nextProgressReportInterval(leaseRemainingTime / 2, leaseRemainingTime); requestedLeaseDurationMs = DEFAULT_LEASE_DURATION_MILLIS; - LOG.info("Started reporting progress for work item: {}", workString()); + LOG.debug("Started reporting progress for work item: {}", workString()); scheduleNextUpdate(); } @@ -121,12 +121,12 @@ public void stopReportingProgress() throws Exception { // We send a final progress report in case there was an unreported fork. if (forkResultToReport != null) { - LOG.info("Sending final progress update with unreported fork: {} " + LOG.debug("Sending final progress update with unreported fork: {} " + "for work item: {}", forkResultToReport, workString()); reportProgressHelper(); // This call can fail with an exception } - LOG.info("Stopped reporting progress for work item: {}", workString()); + LOG.debug("Stopped reporting progress for work item: {}", workString()); } /** @@ -183,11 +183,11 @@ public void run() { * Reports the current work progress to the worker service. */ private void reportProgress() { - LOG.info("Updating progress on work item {}", workString()); + LOG.debug("Updating progress on work item {}", workString()); try { reportProgressHelper(); } catch (Throwable e) { - LOG.warn("Error reporting work progress update: ", e); + LOG.warn("Error reporting workitem progress update to Dataflow service: ", e); } finally { scheduleNextUpdate(); } From 47de70d885b96f92c7bfe3916a5acfbe4dbb6303 Mon Sep 17 00:00:00 2001 From: tudorm Date: Fri, 13 Feb 2015 10:21:23 -0800 Subject: [PATCH 0158/1541] Add an extra check to ease reasoning about the PeekingReiterator. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86282507 --- .../cloud/dataflow/sdk/util/common/PeekingReiterator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/PeekingReiterator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/PeekingReiterator.java index 8789c4e84c8df..ed779da0b4cca 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/PeekingReiterator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/PeekingReiterator.java @@ -36,7 +36,7 @@ public PeekingReiterator(Reiterator iterator) { } PeekingReiterator(PeekingReiterator it) { - this.iterator = checkNotNull(it).iterator.copy(); + this.iterator = checkNotNull(checkNotNull(it).iterator.copy()); this.nextElement = it.nextElement; this.nextElementComputed = it.nextElementComputed; } From 926dda2257e467e5cdb381189ece728755ce0bfe Mon Sep 17 00:00:00 2001 From: millsd Date: Fri, 13 Feb 2015 11:03:29 -0800 Subject: [PATCH 0159/1541] Construct StreamingDataflowWorker options with createFromSystemProperties [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86286736 --- .../sdk/runners/worker/StreamingDataflowWorker.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java index f16200038f326..6567a0cda2442 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -136,7 +136,7 @@ public static void main(String[] args) throws Exception { /** Regular constructor. */ public StreamingDataflowWorker( List mapTasks, WindmillServerStub server) { - options = PipelineOptionsFactory.as(DataflowWorkerHarnessOptions.class); + options = PipelineOptionsFactory.createFromSystemProperties(); options.setAppName("StreamingWorkerHarness"); options.setStreaming(true); @@ -627,12 +627,10 @@ private void printMetrics(PrintWriter response) { private void printResources(PrintWriter response) { Runtime rt = Runtime.getRuntime(); - int mb = 1024 * 1024; - response.append("

    Resources

    \n"); - response.append("Total Memory: " + rt.totalMemory() / mb + "MB
    \n"); - response.append("Used Memory: " + (rt.totalMemory() - rt.freeMemory()) / mb + "MB
    \n"); - response.append("Max Memory: " + rt.maxMemory() / mb + "MB
    \n"); + response.append("Total Memory: " + rt.totalMemory() / 1e6 + "MB
    \n"); + response.append("Used Memory: " + (rt.totalMemory() - rt.freeMemory()) / 1e6 + "MB
    \n"); + response.append("Max Memory: " + rt.maxMemory() / 1e6 + "MB
    \n"); } private void printSpecs(PrintWriter response) { From 162c25429b7509177e4a5a744d38d81b74924d43 Mon Sep 17 00:00:00 2001 From: millsd Date: Fri, 13 Feb 2015 12:55:46 -0800 Subject: [PATCH 0160/1541] Change streaming worker pushback code to only stop pulling work if it is within a threshold of max available memory instead of current total memory. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86296819 --- .../worker/StreamingDataflowWorker.java | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java index 6567a0cda2442..0f75fb9cbb42a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -70,8 +70,9 @@ public class StreamingDataflowWorker { static final int MAX_THREAD_POOL_QUEUE_SIZE = 100; static final long MAX_COMMIT_BYTES = 32 << 20; static final int DEFAULT_STATUS_PORT = 8081; - // Memory threshold under which no new work will be processed. Set to 0 to disable pushback. - static final double PUSHBACK_THRESHOLD = 0.1; + // Memory threshold over which no new work will be processed. + // Set to a value >= 1 to disable pushback. + static final double PUSHBACK_THRESHOLD_RATIO = 0.9; static final String WINDMILL_SERVER_CLASS_NAME = "com.google.cloud.dataflow.sdk.runners.worker.windmill.WindmillServer"; @@ -288,10 +289,15 @@ private void dispatchLoop() { // If free memory is less than a percentage of total memory, block // until current work drains and memory is released. // Also force a GC to try to get under the memory threshold if possible. - while (rt.freeMemory() < rt.totalMemory() * PUSHBACK_THRESHOLD) { + long currentMemorySize = rt.totalMemory(); + long memoryUsed = currentMemorySize - rt.freeMemory(); + long maxMemory = rt.maxMemory(); + + while (memoryUsed > maxMemory * PUSHBACK_THRESHOLD_RATIO) { if (lastPushbackLog < (lastPushbackLog = System.currentTimeMillis()) - 60 * 1000) { - LOG.warn("In pushback, not accepting new work. Free Memory: {}MB / {}MB", - rt.freeMemory() / 1e6, rt.totalMemory() / 1e6); + LOG.warn( + "In pushback, not accepting new work. Using {}MB / {}MB ({}MB currently used by JVM)", + memoryUsed >> 20, maxMemory >> 20, currentMemorySize >> 20); System.gc(); } sleep(10); @@ -628,9 +634,9 @@ private void printMetrics(PrintWriter response) { private void printResources(PrintWriter response) { Runtime rt = Runtime.getRuntime(); response.append("

    Resources

    \n"); - response.append("Total Memory: " + rt.totalMemory() / 1e6 + "MB
    \n"); - response.append("Used Memory: " + (rt.totalMemory() - rt.freeMemory()) / 1e6 + "MB
    \n"); - response.append("Max Memory: " + rt.maxMemory() / 1e6 + "MB
    \n"); + response.append("Total Memory: " + (rt.totalMemory() >> 20) + "MB
    \n"); + response.append("Used Memory: " + ((rt.totalMemory() - rt.freeMemory()) >> 20) + "MB
    \n"); + response.append("Max Memory: " + (rt.maxMemory() >> 20) + "MB
    \n"); } private void printSpecs(PrintWriter response) { From 20be3253eda5e813913d8b0e6484b4c59661c4f8 Mon Sep 17 00:00:00 2001 From: lcwik Date: Fri, 13 Feb 2015 14:28:46 -0800 Subject: [PATCH 0161/1541] Have coders which depend on state information during encoding/decoding or that can't serialize all their state information fail when being used within the DirectPipelineRunner, DataflowPipelineRunner, and our test utilities. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86304960 --- .../dataflow/sdk/util/SerializableUtils.java | 8 +- .../dataflow/sdk/coders/AvroCoderTest.java | 17 +- .../dataflow/sdk/coders/CoderProperties.java | 38 ++-- .../sdk/coders/CoderPropertiesTest.java | 184 ++++++++++++++++++ .../sdk/util/SerializableUtilsTest.java | 155 ++++++++++++--- 5 files changed, 346 insertions(+), 56 deletions(-) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderPropertiesTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SerializableUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SerializableUtils.java index 9ee09c8608ab7..10080f144861f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SerializableUtils.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SerializableUtils.java @@ -92,7 +92,11 @@ public static T ensureSerializable(T value) { * @return the serialized Coder, as a {@link CloudObject} */ public static CloudObject ensureSerializable(Coder coder) { - CloudObject cloudObject = coder.asCloudObject(); + // Make sure that Coders are java serializable as well since + // they are regularly captured within DoFn's. + Coder copy = (Coder) ensureSerializable((Serializable) coder); + + CloudObject cloudObject = copy.asCloudObject(); Coder decoded; try { @@ -114,7 +118,7 @@ public static CloudObject ensureSerializable(Coder coder) { } /** - * Serializes an arbitrary T with the given Coder and verifies + * Serializes an arbitrary T with the given {@code Coder} and verifies * that it can be correctly deserialized. */ public static T ensureSerializableByCoder( diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java index 234fb046b2737..43023a1f63dd9 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java @@ -38,11 +38,8 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; -import java.io.Serializable; -/** - * Tests for AvroCoder. - */ +/** Tests for {@link AvroCoder}. */ @RunWith(JUnit4.class) @SuppressWarnings("serial") public class AvroCoderTest { @@ -53,6 +50,7 @@ private static class Pojo { public int count; // Empty constructor required for Avro decoding. + @SuppressWarnings("unused") public Pojo() { } @@ -184,16 +182,7 @@ public void testDefaultCoder() throws Exception { } @Test - public void testAvroCoderJavaSerializable() throws Exception { - AvroCoder coder = AvroCoder.of(Pojo.class); - - // Cast the coder to serializable to test that it is serializable using - // Java serialization. - SerializableUtils.ensureSerializable((Serializable) coder); - } - - @Test - public void testAvroCoderJsonSerializable() throws Exception { + public void testAvroCoderIsSerializable() throws Exception { AvroCoder coder = AvroCoder.of(Pojo.class); // Check that the coder is serializable using the regular JSON approach. diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java index 5fbaf1f5a59e6..1b9200a9164de 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java @@ -21,11 +21,13 @@ import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; -import static org.junit.Assume.assumeThat; import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.util.Serializer; import com.google.common.collect.Iterables; +import org.hamcrest.CoreMatchers; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -36,6 +38,10 @@ /** * Properties for use in {@link Coder} tests. These are implemented with junit assertions * rather than as predicates for the sake of error messages. + *

    + * We serialize and deserialize the coder to make sure that any state information required by the + * coder is preserved. This causes tests written such that coders which lose information during + * serialization or change state during encoding/decoding will fail. */ public class CoderProperties { @@ -46,7 +52,7 @@ public class CoderProperties { Coder.Context.OUTER, Coder.Context.NESTED); /** - * Verifies that for the given {@link Coder}, and values of + * Verifies that for the given {@link Coder Coder}, and values of * type {@code T}, if the values are equal then the encoded bytes are equal, * in any {@link Coder.Context}. */ @@ -59,20 +65,22 @@ public static void coderDeterministic( } /** - * Verifies that for the given {@link Coder}, {@link Coder.Context}, and values of + * Verifies that for the given {@link Coder Coder}, {@link Coder.Context}, and values of * type {@code T}, if the values are equal then the encoded bytes are equal. */ public static void coderDeterministicInContext( Coder coder, Coder.Context context, T value1, T value2) throws Exception { - assumeThat(value1, equalTo(value2)); + assertThat("Expected that the coder is deterministic", + coder.isDeterministic(), CoreMatchers.is(true)); + assertThat("Expected that the passed in values are equal()", value1, equalTo(value2)); assertThat( encode(coder, context, value1), equalTo(encode(coder, context, value2))); } /** - * Verifies that for the given {@link Coder}, + * Verifies that for the given {@link Coder Coder}, * and value of type {@code T}, encoding followed by decoding yields an * equal value of type {@code T}, in any {@link Coder.Context}. */ @@ -85,7 +93,7 @@ public static void coderDecodeEncodeEqual( } /** - * Verifies that for the given {@link Coder}, {@link Coder.Context}, + * Verifies that for the given {@link Coder Coder}, {@link Coder.Context}, * and value of type {@code T}, encoding followed by decoding yields an * equal value of type {@code T}. */ @@ -96,7 +104,7 @@ public static void coderDecodeEncodeEqualInContext( } /** - * Verifies that for the given {@link Coder>}, + * Verifies that for the given {@link Coder Coder>}, * and value of type {@code Collection}, encoding followed by decoding yields an * equal value of type {@code Collection}, in any {@link Coder.Context}. */ @@ -109,7 +117,7 @@ public static > void coderDecodeEncodeContentsEqual( } /** - * Verifies that for the given {@link Coder>}, + * Verifies that for the given {@link Coder Coder>}, * and value of type {@code Collection}, encoding followed by decoding yields an * equal value of type {@code Collection}, in the given {@link Coder.Context}. */ @@ -128,7 +136,7 @@ public static > void coderDecodeEncodeContentsEqualI } /** - * Verifies that for the given {@link Coder>}, + * Verifies that for the given {@link Coder Coder>}, * and value of type {@code Collection}, encoding followed by decoding yields an * equal value of type {@code Collection}, in any {@link Coder.Context}. */ @@ -142,7 +150,7 @@ public static > void coderDecodeEncodeContentsInSameOr } /** - * Verifies that for the given {@link Coder>}, + * Verifies that for the given {@link Coder Coder>}, * and value of type {@code Iterable}, encoding followed by decoding yields an * equal value of type {@code Collection}, in the given {@link Coder.Context}. */ @@ -168,15 +176,21 @@ public static void coderSerializable(Coder coder) { private static byte[] encode( Coder coder, Coder.Context context, T value) throws CoderException, IOException { + @SuppressWarnings("unchecked") + Coder deserializedCoder = Serializer.deserialize(coder.asCloudObject(), Coder.class); + ByteArrayOutputStream os = new ByteArrayOutputStream(); - coder.encode(value, os, context); + deserializedCoder.encode(value, os, context); return os.toByteArray(); } private static T decode( Coder coder, Coder.Context context, byte[] bytes) throws CoderException, IOException { + @SuppressWarnings("unchecked") + Coder deserializedCoder = Serializer.deserialize(coder.asCloudObject(), Coder.class); + ByteArrayInputStream is = new ByteArrayInputStream(bytes); - return coder.decode(is, context); + return deserializedCoder.decode(is, context); } private static T decodeEncode(Coder coder, Coder.Context context, T value) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderPropertiesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderPropertiesTest.java new file mode 100644 index 0000000000000..66edb00204979 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderPropertiesTest.java @@ -0,0 +1,184 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +import com.google.common.base.Strings; + +import org.hamcrest.CoreMatchers; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** Unit tests for {@link CoderProperties}. */ +@RunWith(JUnit4.class) +public class CoderPropertiesTest { + @Rule public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testGoodCoderIsDeterministic() throws Exception { + CoderProperties.coderDeterministic(StringUtf8Coder.of(), "TestData", "TestData"); + } + + /** A coder which says it is not deterministic but actually is. */ + private static class NonDeterministicCoder extends CustomCoder { + @Override + public void encode(String value, OutputStream outStream, Context context) + throws CoderException, IOException { + StringUtf8Coder.of().encode(value, outStream, context); + } + + @Override + public String decode(InputStream inStream, Context context) + throws CoderException, IOException { + return StringUtf8Coder.of().decode(inStream, context); + } + } + + @Test + public void testNonDeterministicCoder() throws Exception { + try { + CoderProperties.coderDeterministic(new NonDeterministicCoder(), "TestData", "TestData"); + fail("Expected AssertionError"); + } catch (AssertionError error) { + assertThat(error.getMessage(), + CoreMatchers.containsString("Expected that the coder is deterministic")); + } + } + + @Test + public void testPassingInNonEqualValuesWithDeterministicCoder() throws Exception { + try { + CoderProperties.coderDeterministic(StringUtf8Coder.of(), "AAA", "BBB"); + fail("Expected AssertionError"); + } catch (AssertionError error) { + assertThat(error.getMessage(), + CoreMatchers.containsString("Expected that the passed in values")); + } + } + + /** A coder which is non-deterministic because it adds a string to the value. */ + private static class BadDeterminsticCoder extends CustomCoder { + public BadDeterminsticCoder() { + } + + @Override + public void encode(String value, OutputStream outStream, Context context) + throws IOException, CoderException { + StringUtf8Coder.of().encode(value + System.nanoTime(), outStream, context); + } + + @Override + public String decode(InputStream inStream, Context context) + throws CoderException, IOException { + return StringUtf8Coder.of().decode(inStream, context); + } + + @Override + public boolean isDeterministic() { + return true; + } + } + + @Test + public void testBadCoderIsNotDeterministic() throws Exception { + try { + CoderProperties.coderDeterministic(new BadDeterminsticCoder(), "TestData", "TestData"); + fail("Expected AssertionError"); + } catch (AssertionError error) { + assertThat(error.getMessage(), + CoreMatchers.containsString("<84>, <101>, <115>, <116>, <68>")); + } + } + + @Test + public void testGoodCoderEncodesEqualValues() throws Exception { + CoderProperties.coderDecodeEncodeEqual(StringUtf8Coder.of(), "TestData"); + } + + /** This coder changes state during encoding/decoding. */ + private static class StateChangingSerializingCoder extends CustomCoder { + private int changedState; + + public StateChangingSerializingCoder() { + changedState = 10; + } + + @Override + public void encode(String value, OutputStream outStream, Context context) + throws CoderException, IOException { + changedState += 1; + StringUtf8Coder.of().encode(value + Strings.repeat("A", changedState), outStream, context); + } + + @Override + public String decode(InputStream inStream, Context context) + throws CoderException, IOException { + String decodedValue = StringUtf8Coder.of().decode(inStream, context); + return decodedValue.substring(0, decodedValue.length() - changedState); + } + } + + @Test + public void testBadCoderThatDependsOnChangingState() throws Exception { + try { + CoderProperties.coderDecodeEncodeEqual(new StateChangingSerializingCoder(), "TestData"); + fail("Expected AssertionError"); + } catch (AssertionError error) { + assertThat(error.getMessage(), CoreMatchers.containsString("TestData")); + } + } + + /** This coder loses information critical to its operation. */ + private static class ForgetfulSerializingCoder extends CustomCoder { + private transient int lostState; + + public ForgetfulSerializingCoder(int lostState) { + this.lostState = lostState; + } + + @Override + public void encode(String value, OutputStream outStream, Context context) + throws CoderException, IOException { + if (lostState == 0) { + throw new RuntimeException("I forgot something..."); + } + StringUtf8Coder.of().encode(value, outStream, context); + } + + @Override + public String decode(InputStream inStream, Context context) + throws CoderException, IOException { + return StringUtf8Coder.of().decode(inStream, context); + } + } + + @Test + public void testBadCoderThatDependsOnStateThatIsLost() throws Exception { + expectedException.expect(RuntimeException.class); + expectedException.expectMessage("I forgot something..."); + CoderProperties.coderDecodeEncodeEqual(new ForgetfulSerializingCoder(1), "TestData"); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializableUtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializableUtilsTest.java index daddde564aa09..38c73ff6263c6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializableUtilsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/SerializableUtilsTest.java @@ -16,26 +16,39 @@ package com.google.cloud.dataflow.sdk.util; -import org.hamcrest.CoreMatchers; -import org.hamcrest.core.IsInstanceOf; -import org.junit.Assert; +import static org.junit.Assert.assertEquals; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.StandardCoder; +import com.google.common.collect.ImmutableList; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.io.Serializable; +import java.util.List; -/** - * Tests for SerializableUtils. - */ +/** Tests for {@link SerializableUtils}. */ @RunWith(JUnit4.class) -@SuppressWarnings("serial") public class SerializableUtilsTest { - static class TestClass implements Serializable { + @Rule public ExpectedException expectedException = ExpectedException.none(); + + /** A class which is serializable by Java. */ + private static class SerializableByJava implements Serializable { final String stringValue; final int intValue; - public TestClass(String stringValue, int intValue) { + public SerializableByJava(String stringValue, int intValue) { this.stringValue = stringValue; this.intValue = intValue; } @@ -46,31 +59,117 @@ public void testTranscode() { String stringValue = "hi bob"; int intValue = 42; - TestClass testObject = new TestClass(stringValue, intValue); + SerializableByJava testObject = new SerializableByJava(stringValue, intValue); + SerializableByJava testCopy = SerializableUtils.ensureSerializable(testObject); - Object copy = - SerializableUtils.deserializeFromByteArray( - SerializableUtils.serializeToByteArray(testObject), - "a TestObject"); + assertEquals(stringValue, testCopy.stringValue); + assertEquals(intValue, testCopy.intValue); + } - Assert.assertThat(copy, new IsInstanceOf(TestClass.class)); - TestClass testCopy = (TestClass) copy; + @Test + public void testDeserializationError() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("unable to deserialize a bogus string"); + SerializableUtils.deserializeFromByteArray( + "this isn't legal".getBytes(), + "a bogus string"); + } - Assert.assertEquals(stringValue, testCopy.stringValue); - Assert.assertEquals(intValue, testCopy.intValue); + /** A class which is not serializable by Java. */ + private static class UnserializableByJava implements Serializable { + @SuppressWarnings("unused") + private Object unserializableField = new Object(); } @Test - public void testDeserializationError() { - try { - SerializableUtils.deserializeFromByteArray( - "this isn't legal".getBytes(), - "a bogus string"); - Assert.fail("should have thrown an exception"); - } catch (Exception exn) { - Assert.assertThat(exn.toString(), - CoreMatchers.containsString( - "unable to deserialize a bogus string")); + public void testSerializationError() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("unable to serialize"); + SerializableUtils.serializeToByteArray(new UnserializableByJava()); + } + + /** A {@link Coder} which is not serializable by Java. */ + private static class UnserializableCoderByJava extends StandardCoder { + private final Object unserializableField = new Object(); + + @Override + public void encode(Object value, OutputStream outStream, Context context) + throws CoderException, IOException { + } + + @Override + public Object decode(InputStream inStream, Context context) + throws CoderException, IOException { + return unserializableField; + } + + @Override + public List> getCoderArguments() { + return ImmutableList.of(); + } + + @Override + public boolean isDeterministic() { + return true; + } + } + + @Test + public void testEnsureSerializableWithUnserializableCoderByJava() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("unable to serialize"); + SerializableUtils.ensureSerializable(new UnserializableCoderByJava()); + } + + /** A {@link Coder} which is not serializable by Jackson. */ + private static class UnserializableCoderByJackson extends StandardCoder { + private final SerializableByJava unserializableField; + + public UnserializableCoderByJackson(SerializableByJava unserializableField) { + this.unserializableField = unserializableField; + } + + @JsonCreator + public static UnserializableCoderByJackson of( + @JsonProperty("unserializableField") SerializableByJava unserializableField) { + return new UnserializableCoderByJackson(unserializableField); + } + + @Override + public CloudObject asCloudObject() { + CloudObject result = super.asCloudObject(); + result.put("unserializableField", unserializableField); + return result; + } + + @Override + public void encode(Object value, OutputStream outStream, Context context) + throws CoderException, IOException { + } + + @Override + public Object decode(InputStream inStream, Context context) + throws CoderException, IOException { + return unserializableField; + } + + @Override + public List> getCoderArguments() { + return ImmutableList.of(); + } + + @Override + public boolean isDeterministic() { + return true; } } + + @Test + public void testEnsureSerializableWithUnserializableCoderByJackson() throws Exception { + expectedException.expect(RuntimeException.class); + expectedException.expectMessage("Unable to deserialize Coder:"); + SerializableUtils.ensureSerializable( + new UnserializableCoderByJackson(new SerializableByJava("TestData", 5))); + } + } From f1e07218359cbdc8ce24a9985bdda696b3ee3f70 Mon Sep 17 00:00:00 2001 From: lcwik Date: Fri, 13 Feb 2015 16:15:21 -0800 Subject: [PATCH 0162/1541] Remove code duplication and standardize on the same exception formatter for job messages and log statements. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86313678 --- .../sdk/runners/worker/DataflowWorker.java | 31 ++----------------- .../DataflowWorkerLoggingFormatter.java | 3 +- .../runners/worker/DataflowWorkerTest.java | 10 +++++- 3 files changed, 13 insertions(+), 31 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java index fcd6ccd618a4e..13003108a022d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java @@ -27,6 +27,7 @@ import com.google.api.services.dataflow.model.WorkItemServiceState; import com.google.api.services.dataflow.model.WorkItemStatus; import com.google.cloud.dataflow.sdk.options.DataflowWorkerHarnessOptions; +import com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingFormatter; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CloudCounterUtils; import com.google.cloud.dataflow.sdk.util.CloudMetricUtils; @@ -43,8 +44,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.io.PrintWriter; -import java.io.StringWriter; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -117,7 +116,7 @@ private boolean doWork(WorkItem workItem) throws IOException { worker = SourceOperationExecutorFactory.create(options, workItem.getSourceOperationTask()); } else { - throw new RuntimeException("unknown kind of work item: " + workItem.toString()); + throw new RuntimeException("Unknown kind of work item: " + workItem.toString()); } DataflowWorkProgressUpdater progressUpdater = @@ -186,37 +185,13 @@ private void handleWorkError(WorkItem workItem, WorkExecutor worker, Throwable e Status error = new Status(); error.setCode(2); // Code.UNKNOWN. TODO: Replace with a generated definition. // TODO: Attach the stack trace as exception details, not to the message. - error.setMessage(buildCloudStackTrace(t)); + error.setMessage(DataflowWorkerLoggingFormatter.formatException(t)); reportStatus(options, "Failure", workItem, worker == null ? null : worker.getOutputCounters(), worker == null ? null : worker.getOutputMetrics(), null/*sourceOperationResponse*/, error == null ? null : Collections.singletonList(error), 0); } - /** - * Recursively goes through an exception, pulling out the stack trace. If the - * exception is a chained exception, it recursively goes through any causes - * and appends them to the stack trace. - */ - private static String buildCloudStackTrace(Throwable t) { - StringWriter result = new StringWriter(); - PrintWriter printResult = new PrintWriter(result); - - printResult.print("Exception: "); - for (;;) { - printResult.println(t.toString()); - for (StackTraceElement frame : t.getStackTrace()) { - printResult.println(frame.toString()); - } - t = t.getCause(); - if (t == null) { - break; - } - printResult.print("Caused by: "); - } - return result.toString(); - } - private void reportStatus(DataflowWorkerHarnessOptions options, String status, WorkItem workItem, @Nullable CounterSet counters, @Nullable Collection> metrics, @Nullable SourceFormat.OperationResponse operationResponse, @Nullable List errors, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java index b6bb0ce960173..9f748eb05228b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java @@ -85,7 +85,6 @@ public static String getWorkId() { return workId.get(); } - @Override public String format(LogRecord record) { String exception = formatException(record.getThrown()); @@ -107,7 +106,7 @@ public String format(LogRecord record) { * @param thrown The throwable to format. * @return A string containing the contents of {@link Throwable#printStackTrace()}. */ - private String formatException(Throwable thrown) { + public static String formatException(Throwable thrown) { if (thrown == null) { return null; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java index 037946c29a917..aee95ef0a28f9 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java @@ -15,6 +15,7 @@ package com.google.cloud.dataflow.sdk.runners.worker; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; import static org.mockito.Matchers.argThat; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -24,6 +25,7 @@ import com.google.cloud.dataflow.sdk.options.DataflowWorkerHarnessOptions; import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper; +import org.hamcrest.CoreMatchers; import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeMatcher; @@ -79,8 +81,14 @@ public void describeTo(Description description) { @Override protected boolean matchesSafely(WorkItemStatus status) { - return status.getCompleted() && !status.getErrors().isEmpty(); + boolean returnValue = status.getCompleted() && !status.getErrors().isEmpty(); + if (returnValue) { + assertThat(status.getErrors().get(0).getMessage(), + CoreMatchers.containsString("java.lang.RuntimeException: Unknown kind of work")); + } + return returnValue; } }; } } + From e02cb7b49b5eb5d41f9512db189b60c559a4d503 Mon Sep 17 00:00:00 2001 From: tudorm Date: Fri, 13 Feb 2015 16:38:59 -0800 Subject: [PATCH 0163/1541] Remove serializations of objects from precondition checks. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86315430 --- .../cloud/dataflow/sdk/util/common/worker/ReadOperation.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java index d7f3f479f0ee3..883107484919d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperation.java @@ -246,8 +246,8 @@ public Reader.ForkResult requestFork(Reader.ForkRequest forkRequest) { private class ReaderObserver implements Observer { @Override public void update(Observable obs, Object obj) { - Preconditions.checkArgument(obs == reader, "unexpected observable" + obs); - Preconditions.checkArgument(obj instanceof Long, "unexpected parameter object: " + obj); + Preconditions.checkArgument(obs == reader, "unexpected observable"); + Preconditions.checkArgument(obj instanceof Long, "unexpected parameter object"); byteCount.addValue((long) obj); } } From ac6787a6665d7922759d9e6970b6c5536a85f004 Mon Sep 17 00:00:00 2001 From: rfernand Date: Tue, 17 Feb 2015 10:20:16 -0800 Subject: [PATCH 0164/1541] Explicitly throwing NoSuchElementException in the iterator. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86505113 --- .../google/cloud/dataflow/sdk/util/common/worker/Reader.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java index 93a398907dd1e..bb0c0b25a9758 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/Reader.java @@ -48,9 +48,10 @@ public interface ReaderIterator extends AutoCloseable { /** * Returns the next element. * + * @throws IOException if attempting to access an element involves IO that fails * @throws NoSuchElementException if there are no more elements */ - public T next() throws IOException; + public T next() throws IOException, NoSuchElementException; /** * Copies the current ReaderIterator. From 37827d7b2fc4a1cf8789b02ee618ccd1ce7f4c79 Mon Sep 17 00:00:00 2001 From: relax Date: Wed, 18 Feb 2015 19:04:00 -0800 Subject: [PATCH 0165/1541] The BigQuery insertion API does not allow more than 500 rows per insertion request. Make sure that BigtableIO honors this limit. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86653226 --- .../cloud/dataflow/sdk/util/BigQueryTableInserter.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableInserter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableInserter.java index c241ee2f25912..0ae41a057b1b2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableInserter.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableInserter.java @@ -45,7 +45,10 @@ public class BigQueryTableInserter { private static final Logger LOG = LoggerFactory.getLogger(BigQueryTableInserter.class); // Approximate amount of table data to upload per InsertAll request. - private static final long UPLOAD_BATCH_SIZE = 64 * 1024; + private static final long UPLOAD_BATCH_SIZE_BYTES = 64 * 1024; + + // The maximum number of rows to upload per InsertAll request. + private static final long MAX_ROWS_PER_BATCH = 500; private final Bigquery client; private final TableReference ref; @@ -92,7 +95,8 @@ public void insertAll(Iterator rowIterator, rows.add(out); dataSize += row.toString().length(); - if (dataSize >= UPLOAD_BATCH_SIZE || !rowIterator.hasNext()) { + if (dataSize >= UPLOAD_BATCH_SIZE_BYTES || rows.size() >= MAX_ROWS_PER_BATCH || + !rowIterator.hasNext()) { TableDataInsertAllRequest content = new TableDataInsertAllRequest(); content.setRows(rows); From 8f800a7a2b013406a2c81df4d2a7d390b899bb84 Mon Sep 17 00:00:00 2001 From: mariand Date: Wed, 18 Feb 2015 19:08:34 -0800 Subject: [PATCH 0166/1541] StateSampler: Atomically set the thread state rather than using locks. Improves S02 in WordCount by ~10%. Also, removed unused stack dumping and cleaned up the member fields. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86653445 --- .../sdk/util/common/worker/StateSampler.java | 151 +++++------------- .../util/common/worker/StateSamplerTest.java | 32 ++-- 2 files changed, 51 insertions(+), 132 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSampler.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSampler.java index 91d90e9d2a05f..59d21b93aa4bd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSampler.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSampler.java @@ -19,51 +19,52 @@ import com.google.cloud.dataflow.sdk.util.common.Counter; import com.google.cloud.dataflow.sdk.util.common.CounterSet; -import java.util.AbstractMap.SimpleEntry; import java.util.ArrayList; import java.util.HashMap; -import java.util.Map; import java.util.Random; import java.util.Timer; import java.util.TimerTask; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import javax.annotation.concurrent.ThreadSafe; + /** * A StateSampler object may be used to obtain an approximate * breakdown of the time spent by an execution context in various * states, as a fraction of the total time. The sampling is taken at * regular intervals, with adjustment for scheduling delay. - * - *

    Thread-safe. */ +@ThreadSafe public class StateSampler extends TimerTask implements AutoCloseable { - private final String prefix; - private CounterSet.AddCounterMutator counterSetMutator; - // Sampling period of internal Timer (thread). - public final long samplingPeriodMs; - public static final int DO_NOT_SAMPLE = -1; public static final long DEFAULT_SAMPLING_PERIOD_MS = 200; - // Array of counters indexed by their state. + + private final String prefix; + private final CounterSet.AddCounterMutator counterSetMutator; + + /** Array of counters indexed by their state. */ private ArrayList> countersByState = new ArrayList<>(); - // Map of state name to state. + + /** Map of state name to state. */ private HashMap statesByName = new HashMap<>(); - // The current state. - private int currentState; - // The timestamp corresponding to the last state change or the last - // time the current state was sampled (and recorded). - private long stateTimestamp = 0; - // When sampling this state, a stack trace is also logged. - private int stateToSampleThreadStacks = DO_NOT_SAMPLE; - // The thread that performed the last state transition. - private Thread sampledThread = null; - // The frequency with which the stack traces are logged, with respect - // to the sampling period. - private static final int SAMPLE_THREAD_STACK_FREQ = 10; - private int sampleThreadStackFreq = 0; + /** The current state. */ + private final AtomicInteger currentState; + + /** Special value of {@code currentState} which we do not sample. */ + private static final int DO_NOT_SAMPLE = -1; + + /** + * The timestamp (in nanoseconds) corresponding to the last time the + * state was sampled (and recorded). + */ + private long stateTimestampNs = 0; - // Using a fixed number of timers for all StateSampler objects. + /** Using a fixed number of timers for all StateSampler objects. */ private static final int NUM_TIMER_THREADS = 16; - // The timers is used for periodically sampling the states. + + /** The timers are used for periodically sampling the states. */ private static Timer[] timers = new Timer[NUM_TIMER_THREADS]; static { for (int i = 0; i < timers.length; ++i) { @@ -86,13 +87,12 @@ public StateSampler(String prefix, long samplingPeriodMs) { this.prefix = prefix; this.counterSetMutator = counterSetMutator; - this.samplingPeriodMs = samplingPeriodMs; - currentState = DO_NOT_SAMPLE; + currentState = new AtomicInteger(DO_NOT_SAMPLE); Random rand = new Random(); int initialDelay = rand.nextInt((int) samplingPeriodMs); timers[rand.nextInt(NUM_TIMER_THREADS)].scheduleAtFixedRate( this, initialDelay, samplingPeriodMs); - stateTimestamp = System.currentTimeMillis(); + stateTimestampNs = System.nanoTime(); } /** @@ -109,37 +109,17 @@ public StateSampler(String prefix, this(prefix, counterSetMutator, DEFAULT_SAMPLING_PERIOD_MS); } - private void printStackTrace(Thread thread) { - System.out.println("Sampled stack trace:"); - StackTraceElement[] stack = thread.getStackTrace(); - for (StackTraceElement elem : stack) { - System.out.println("\t" + elem.toString()); - } - } - - /** - * Selects a state for which the thread stacks will also be logged - * during the sampling. Useful for debugging. - * - * @param state name of the selected state - */ - public synchronized void setStateToSampleThreadStacks(int state) { - stateToSampleThreadStacks = state; - } - @Override - public synchronized void run() { - long now = System.currentTimeMillis(); - if (currentState != DO_NOT_SAMPLE) { - countersByState.get(currentState).addValue(now - stateTimestamp); - if (sampledThread != null - && currentState == stateToSampleThreadStacks - && ++sampleThreadStackFreq >= SAMPLE_THREAD_STACK_FREQ) { - printStackTrace(sampledThread); - sampleThreadStackFreq = 0; + public void run() { + long startTimestampNs = System.nanoTime(); + int state = currentState.get(); + if (state != DO_NOT_SAMPLE) { + synchronized (this) { + countersByState.get(state).addValue( + TimeUnit.NANOSECONDS.toMillis(startTimestampNs - stateTimestampNs)); } } - stateTimestamp = now; + stateTimestampNs = startTimestampNs; } @Override @@ -180,15 +160,8 @@ public int stateForName(String name) { * @param state the new state to transition to * @return the previous state */ - public synchronized int setState(int state) { - // TODO: investigate whether this can be made cheaper, (e.g., - // using atomic operations). - int previousState = currentState; - currentState = state; - if (stateToSampleThreadStacks != DO_NOT_SAMPLE) { - sampledThread = Thread.currentThread(); - } - return previousState; + public int setState(int state) { + return currentState.getAndSet(state); } /** @@ -197,39 +170,10 @@ public synchronized int setState(int state) { * @param name the name of the new state to transition to * @return the previous state */ - public synchronized int setState(String name) { + public int setState(String name) { return setState(stateForName(name)); } - /** - * Returns a tuple consisting of the current state and duration. - * - * @return a {@link Map.Entry} entry with current state and duration - */ - public synchronized Map.Entry getCurrentStateAndDuration() { - if (currentState == DO_NOT_SAMPLE) { - return new SimpleEntry<>("", 0L); - } - - Counter counter = countersByState.get(currentState); - return new SimpleEntry<>(counter.getName(), - counter.getAggregate(false) - + System.currentTimeMillis() - stateTimestamp); - } - - /** - * Get the duration for a given state. - * - * @param state the state whose duration is returned - * @return the duration of a given state - */ - public synchronized long getStateDuration(int state) { - Counter counter = countersByState.get(state); - return counter.getAggregate(false) - + (state == currentState - ? System.currentTimeMillis() - stateTimestamp : 0); - } - /** * Returns an AutoCloseable {@link ScopedState} that will perform a * state transition to the given state, and will automatically reset @@ -239,23 +183,10 @@ public synchronized long getStateDuration(int state) { * @return a {@link ScopedState} that automatically resets the state * to the prior state */ - public synchronized ScopedState scopedState(int state) { + public ScopedState scopedState(int state) { return new ScopedState(this, setState(state)); } - /** - * Returns an AutoCloseable {@link ScopedState} that will perform a - * state transition to the given state, and will automatically reset - * the state to the prior state upon closing. - * - * @param stateName the name of the new state - * @return a {@link ScopedState} that automatically resets the state - * to the prior state - */ - public synchronized ScopedState scopedState(String stateName) { - return new ScopedState(this, setState(stateName)); - } - /** * A nested class that is used to account for states and state * transitions based on lexical scopes. diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java index 291efe14a5546..18e2103840372 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java @@ -16,7 +16,6 @@ package com.google.cloud.dataflow.sdk.util.common.worker; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.util.common.Counter; @@ -26,14 +25,15 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.AbstractMap.SimpleEntry; -import java.util.Map; - /** * Unit tests for the {@link Counter} API. */ @RunWith(JUnit4.class) public class StateSamplerTest { + public static long getCounterLongValue(CounterSet counters, String name) { + Counter counter = (Counter) counters.getExistingCounter(name); + return counter.getAggregate(false); + } @Test public void basicTest() throws InterruptedException { @@ -45,9 +45,6 @@ public void basicTest() throws InterruptedException { int state1 = stateSampler.stateForName("1"); int state2 = stateSampler.stateForName("2"); - assertEquals(new SimpleEntry<>("", 0L), - stateSampler.getCurrentStateAndDuration()); - try (StateSampler.ScopedState s1 = stateSampler.scopedState(state1)) { assert s1 != null; @@ -60,8 +57,8 @@ public void basicTest() throws InterruptedException { Thread.sleep(3 * periodMs); } - long s1 = stateSampler.getStateDuration(state1); - long s2 = stateSampler.getStateDuration(state2); + long s1 = getCounterLongValue(counters, "test-1-msecs"); + long s2 = getCounterLongValue(counters, "test-2-msecs"); System.out.println("basic s1: " + s1); System.out.println("basic s2: " + s2); @@ -82,9 +79,6 @@ public void nestingTest() throws InterruptedException { int state2 = stateSampler.stateForName("2"); int state3 = stateSampler.stateForName("3"); - assertEquals(new SimpleEntry<>("", 0L), - stateSampler.getCurrentStateAndDuration()); - try (StateSampler.ScopedState s1 = stateSampler.scopedState(state1)) { assert s1 != null; @@ -105,9 +99,9 @@ public void nestingTest() throws InterruptedException { Thread.sleep(periodMs); } - long s1 = stateSampler.getStateDuration(state1); - long s2 = stateSampler.getStateDuration(state2); - long s3 = stateSampler.getStateDuration(state3); + long s1 = getCounterLongValue(counters, "test-1-msecs"); + long s2 = getCounterLongValue(counters, "test-2-msecs"); + long s3 = getCounterLongValue(counters, "test-3-msecs"); System.out.println("s1: " + s1); System.out.println("s2: " + s2); @@ -128,17 +122,11 @@ public void nonScopedTest() throws InterruptedException { int state1 = stateSampler.stateForName("1"); int previousState = stateSampler.setState(state1); Thread.sleep(2 * periodMs); - Map.Entry currentStateAndDuration = - stateSampler.getCurrentStateAndDuration(); stateSampler.setState(previousState); - assertEquals("test-1-msecs", currentStateAndDuration.getKey()); long tolerance = periodMs; - long s = currentStateAndDuration.getValue(); + long s = getCounterLongValue(counters, "test-1-msecs"); System.out.println("s: " + s); assertTrue(s >= periodMs - tolerance); assertTrue(s <= 4 * periodMs + tolerance); - - assertTrue(stateSampler.getCurrentStateAndDuration() - .getKey().isEmpty()); } } From 6bfb3e2371c11fa2d0931e08a2e1355cc7cd5c73 Mon Sep 17 00:00:00 2001 From: relax Date: Thu, 19 Feb 2015 01:21:54 -0800 Subject: [PATCH 0167/1541] Cleanup BigQueryIO class. Remove the logic to automatically serialize/deserialize TableRowObjects and replace it with a transient member variable. Replace ThreadLocal HashSet with ConcurrentHashMap. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86670376 --- .../cloud/dataflow/sdk/io/BigQueryIO.java | 132 ++++++------------ 1 file changed, 40 insertions(+), 92 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java index 11ad6f71e82f6..dca65ad9f592e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java @@ -44,12 +44,12 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.io.Serializable; import java.util.ArrayList; -import java.util.HashSet; -import java.util.Iterator; +import java.util.Collections; import java.util.List; +import java.util.Set; import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicLong; import java.util.regex.Matcher; @@ -638,62 +638,6 @@ public boolean getValidate() { */ private static class StreamingWriteFn extends DoFn>, Void> implements DoFn.RequiresKeyedState { - /** - * Class to accumulate BigQuery row data as a list of String. - * DoFn implementation must be Serializable, but BigQuery classes, - * such as TableRow are not. Therefore, convert into JSON String - * for accumulation. - */ - private static class JsonTableRows implements Iterable, Serializable { - /** The list where BigQuery row data is accumulated. */ - private final List jsonRows = new ArrayList<>(); - - /** Iterator of JsonTableRows converts the row in String to TableRow. */ - static class JsonTableRowIterator implements Iterator { - private final Iterator iteratorInternal; - - /** Constructor. */ - JsonTableRowIterator(List jsonRowList) { - iteratorInternal = jsonRowList.iterator(); - } - - @Override - public boolean hasNext() { - return iteratorInternal.hasNext(); - } - - @Override - public TableRow next() { - try { - // Converts the String back into TableRow. - return JSON_FACTORY.fromString(iteratorInternal.next(), TableRow.class); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public void remove() { - iteratorInternal.remove(); - } - } - - /** Returns the iterator. */ - @Override - public Iterator iterator() { - return new JsonTableRowIterator(jsonRows); - } - - /** Adds a BigQuery TableRow. */ - void add(TableRow row) { - try { - // Converts into JSON format. - jsonRows.add(JSON_FACTORY.toString(row)); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - } /** TableReference in JSON. Use String to make the class Serializable. */ private final String jsonTableReference; @@ -701,20 +645,18 @@ void add(TableRow row) { /** TableSchema in JSON. Use String to make the class Serializable. */ private final String jsonTableSchema; + private transient TableReference tableReference; + /** JsonTableRows to accumulate BigQuery rows. */ - private JsonTableRows jsonTableRows; + private transient List tableRows; /** The list of unique ids for each BigQuery table row. */ - private List uniqueIdsForTableRows; + private transient List uniqueIdsForTableRows; /** The list of tables created so far, so we don't try the creation each time. */ - private static ThreadLocal> createdTables = new ThreadLocal>() { - @Override - protected HashSet initialValue() { - return new HashSet<>(); - } - }; + private static Set createdTables = + Collections.newSetFromMap(new ConcurrentHashMap()); /** Constructor. */ StreamingWriteFn(TableReference table, TableSchema schema) { @@ -729,27 +671,25 @@ protected HashSet initialValue() { /** Prepares a target BigQuery table. */ @Override public void startBundle(Context context) { - jsonTableRows = new JsonTableRows(); + tableRows = new ArrayList<>(); uniqueIdsForTableRows = new ArrayList<>(); BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); - Bigquery client = Transport.newBigQueryClient(options).build(); // TODO: Support table sharding and the better place to initialize - // BigQuery table. - HashSet tables = createdTables.get(); - if (!tables.contains(jsonTableSchema)) { - try { - TableSchema tableSchema = JSON_FACTORY.fromString(jsonTableSchema, TableSchema.class); - TableReference tableReference = - JSON_FACTORY.fromString(jsonTableReference, TableReference.class); - + // BigQuery table. + try { + tableReference = + JSON_FACTORY.fromString(jsonTableReference, TableReference.class); + if (!createdTables.contains(jsonTableSchema)) { + TableSchema tableSchema = JSON_FACTORY.fromString(jsonTableSchema, TableSchema.class); + Bigquery client = Transport.newBigQueryClient(options).build(); BigQueryTableInserter inserter = new BigQueryTableInserter(client, tableReference); inserter.tryCreateTable(tableSchema); - tables.add(jsonTableSchema); - } catch (IOException e) { - throw new RuntimeException(e); + createdTables.add(jsonTableSchema); } + } catch (IOException e) { + throw new RuntimeException(e); } } @@ -757,25 +697,33 @@ public void startBundle(Context context) { @Override public void processElement(ProcessContext context) { KV> kv = context.element(); - TableRow tableRow = kv.getValue().getValue(); - uniqueIdsForTableRows.add(kv.getValue().getKey()); - jsonTableRows.add(tableRow); + addRow(kv.getValue().getValue(), kv.getValue().getKey()); } /** Writes the accumulated rows into BigQuery with streaming API. */ @Override public void finishBundle(Context context) { - BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); - Bigquery client = Transport.newBigQueryClient(options).build(); + flushRows(context.getPipelineOptions().as(BigQueryOptions.class)); + } - try { - TableReference tableReference = - JSON_FACTORY.fromString(jsonTableReference, TableReference.class); + /** Accumulate a row to be written to BigQuery. */ + private void addRow(TableRow tableRow, String uniqueId) { + uniqueIdsForTableRows.add(uniqueId); + tableRows.add(tableRow); + } - BigQueryTableInserter inserter = new BigQueryTableInserter(client, tableReference); - inserter.insertAll(jsonTableRows.iterator(), uniqueIdsForTableRows.iterator()); - } catch (IOException e) { - throw new RuntimeException(e); + /** Writes the accumulated rows into BigQuery with streaming API. */ + private void flushRows(BigQueryOptions options) { + if (!tableRows.isEmpty()) { + Bigquery client = Transport.newBigQueryClient(options).build(); + try { + BigQueryTableInserter inserter = new BigQueryTableInserter(client, tableReference); + inserter.insertAll(tableRows.iterator(), uniqueIdsForTableRows.iterator()); + } catch (IOException e) { + throw new RuntimeException(e); + } + tableRows.clear(); + uniqueIdsForTableRows.clear(); } } } From f724a169065e77ec42f1282848714dd7c56a156d Mon Sep 17 00:00:00 2001 From: millsd Date: Thu, 19 Feb 2015 10:30:13 -0800 Subject: [PATCH 0168/1541] Removes notes about streaming requiring whitelisting [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86703478 --- .../google/cloud/dataflow/sdk/io/PubsubIO.java | 12 ++++++------ .../dataflow/sdk/options/StreamingOptions.java | 15 ++------------- 2 files changed, 8 insertions(+), 19 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index 2414feeb34f9f..a405a7547ef19 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -29,14 +29,14 @@ import java.util.regex.Pattern; /** - * [Whitelisting Required] Read and Write transforms for Pub/Sub streams. These transforms create + * Read and Write transforms for Pub/Sub streams. These transforms create * and consume unbounded {@link com.google.cloud.dataflow.sdk.values.PCollection}s. * - *

    Important: PubsubIO is experimental. It is not supported by the - * {@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner} and is only supported in the - * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner} for users whitelisted in a - * streaming early access program and who enable - * {@link com.google.cloud.dataflow.sdk.options.StreamingOptions#setStreaming(boolean)}. + *

    {@code PubsubIO} is experimental. It is only usable + * with the {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner} + * and requires + * {@link com.google.cloud.dataflow.sdk.options.StreamingOptions#setStreaming(boolean)} + * to be enabled. * *

    You should expect this class to change significantly in future versions of the SDK * or be removed entirely. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java index 6deca04b4888f..56db83da2a548 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/StreamingOptions.java @@ -17,22 +17,11 @@ package com.google.cloud.dataflow.sdk.options; /** - * [Whitelisting Required] Options used to configure the streaming backend. - * - *

    Important: Streaming support is experimental. It is only supported in the - * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner} for users whitelisted in a - * streaming early access program. - * - *

    You should expect this class to change significantly in future - * versions of the SDK or be removed entirely. + * Options used to configure the streaming backend. */ public interface StreamingOptions extends ApplicationNameOptions, GcpOptions, PipelineOptions { - /** - * Note that this feature is currently experimental and only available to users whitelisted in - * a streaming early access program. - */ - @Description("True if running in streaming mode (experimental)") + @Description("True if running in streaming mode") boolean isStreaming(); void setStreaming(boolean value); } From 25c3a7e548d188dc4beeb29d69bfacaf0707482a Mon Sep 17 00:00:00 2001 From: robertwb Date: Thu, 19 Feb 2015 11:10:53 -0800 Subject: [PATCH 0169/1541] Slightly stronger coder inference. Now a TemplateType> can infer either argument's Coder from the other. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86707653 --- .../cloud/dataflow/sdk/coders/CoderRegistry.java | 6 +++--- .../cloud/dataflow/sdk/coders/StandardCoder.java | 2 +- .../dataflow/sdk/coders/CoderRegistryTest.java | 15 +++++++++++---- 3 files changed, 15 insertions(+), 8 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java index 37128918b5d71..ec2ff2579cdc1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java @@ -413,7 +413,7 @@ public Coder[] getDefaultCoders( "Cannot encode elements of type " + typeArgs[i] + " with " + knownCoders[i]); } - context.put(typeArgs[i], knownCoders[i]); + fillTypeBindings(typeArgs[i], knownCoders[i], context); } } Coder[] result = new Coder[typeArgs.length]; @@ -574,7 +574,7 @@ Coder getDefaultCoder(Type type, Map> typeCoderBindings) { return getDefaultCoder((Class) type); } else if (type instanceof ParameterizedType) { return this.getDefaultCoder((ParameterizedType) type, - typeCoderBindings); + typeCoderBindings); } else if (type instanceof TypeVariable || type instanceof WildcardType) { // No default coder for an unknown generic type. @@ -666,7 +666,7 @@ Map> createTypeBindings(TypeToken typeToken, void fillTypeBindings(Type type, Coder coder, Map> typeCoderBindings) { - if (type instanceof TypeVariable) { + if (type instanceof TypeVariable || type instanceof Class) { LOG.debug("Binding type {} to Coder {}", type, coder); typeCoderBindings.put(type, coder); } else if (type instanceof ParameterizedType) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java index 7a35fdcafbf27..9f475a3d9a786 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java @@ -53,7 +53,7 @@ public List> getComponents() { @Override public boolean equals(Object o) { - if (this.getClass() != o.getClass()) { + if (o == null || this.getClass() != o.getClass()) { return false; } StandardCoder that = (StandardCoder) o; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java index a05816c776886..2c411fae42316 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java @@ -92,28 +92,35 @@ public void testTemplateInference() { CoderRegistry registry = getStandardRegistry(); MyTemplateClass> instance = new MyTemplateClass>() {}; - Coder> expected = ListCoder.of(MyValueCoder.of()); + Coder> listCoder = ListCoder.of(MyValueCoder.of()); // The map method operates on parameter names. Map> coderMap = registry.getDefaultCoders( instance.getClass(), MyTemplateClass.class, Collections.singletonMap("A", MyValueCoder.of())); - assertEquals(expected, coderMap.get("B")); + assertEquals(listCoder, coderMap.get("B")); + + // Check we can infer the other direction as well. + Map> coderMap2 = registry.getDefaultCoders( + instance.getClass(), + MyTemplateClass.class, + Collections.singletonMap("B", listCoder)); + assertEquals(MyValueCoder.of(), coderMap2.get("A")); // The array interface operates on position. Coder[] coders = registry.getDefaultCoders( instance.getClass(), MyTemplateClass.class, new Coder[] { MyValueCoder.of(), null }); - assertEquals(expected, coders[1]); + assertEquals(listCoder, coders[1]); // The "last argument" coder handles a common case. Coder> actual = registry.getDefaultCoder( instance.getClass(), MyTemplateClass.class, MyValueCoder.of()); - assertEquals(expected, actual); + assertEquals(listCoder, actual); try { registry.getDefaultCoder( From 1ba2a06a3da3fd1992d49250fed880375b81d883 Mon Sep 17 00:00:00 2001 From: millsd Date: Fri, 20 Feb 2015 11:36:07 -0800 Subject: [PATCH 0170/1541] Add javadoc to GroupByKey explaining what will happen in the presence of late data [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86806759 --- .../com/google/cloud/dataflow/sdk/io/PubsubIO.java | 10 +++++++++- .../cloud/dataflow/sdk/transforms/GroupByKey.java | 5 +++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index a405a7547ef19..667653deea115 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -195,7 +195,10 @@ public static Bound subscription(String subscription) { * the first time it sees each record. All windowing will be done relative to these timestamps. * Windows are closed based on an estimate of when this source has finished producing data for * a timestamp range, which means that late data can arrive after a window has been closed. The - * {#dropLateData} field allows you to control what to do with late data. + * {#dropLateData} field allows you to control what to do with late data. The relaxes the + * semantics of {@code GroupByKey}; see + * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} for additional information on + * late data and windowing. */ public static Bound timestampLabel(String timestampLabel) { return new Bound().timestampLabel(timestampLabel); @@ -203,6 +206,11 @@ public static Bound timestampLabel(String timestampLabel) { /** * If true, then late-arriving data from this source will be dropped. + * + *

    If late data is not dropped, data for a window can arrive after that window has already + * been closed. The relaxes the semantics of {@code GroupByKey}; see + * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} + * for additional information on late data and windowing. */ public static Bound dropLateData(boolean dropLateData) { return new Bound().dropLateData(dropLateData); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index 4d11723c99638..471229f308e9a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -107,6 +107,11 @@ * in the window that it corresponds to. The output {@code PCollection} will * have the same {@link WindowFn} as the input. * + *

    If the input {@code PCollection} contains late data (see + * {@link com.google.cloud.dataflow.sdk.PubsubIO.Read.Bound#timestampLabel} + * for an example of how this can occur), then there may be multiple elements + * output by a {@code GroupByKey} that correspond to the same key and window. + * *

    If the {@link WindowFn} of the input requires merging, it is not * valid to apply another {@code GroupByKey} without first applying a new * {@link WindowFn}. From 81a4c3cc0629bec78358170e409ece21239c15ba Mon Sep 17 00:00:00 2001 From: bchambers Date: Fri, 20 Feb 2015 13:21:05 -0800 Subject: [PATCH 0171/1541] Add Coder#verifyDeterministic() to document the reasons for non-determinism. Implementations should overload to explain why they are non-determinstic. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86815676 --- .../cloud/dataflow/sdk/coders/Coder.java | 70 +++++++++++++++++++ .../dataflow/sdk/coders/CustomCoder.java | 1 + .../dataflow/sdk/coders/DelegateCoder.java | 6 ++ .../dataflow/sdk/coders/DoubleCoder.java | 7 ++ .../dataflow/sdk/coders/EntityCoder.java | 7 ++ .../dataflow/sdk/coders/InstantCoder.java | 5 ++ .../sdk/coders/IterableLikeCoder.java | 7 ++ .../cloud/dataflow/sdk/coders/KvCoder.java | 7 ++ .../cloud/dataflow/sdk/coders/ListCoder.java | 8 +++ .../cloud/dataflow/sdk/coders/MapCoder.java | 7 ++ .../sdk/coders/SerializableCoder.java | 7 ++ .../cloud/dataflow/sdk/coders/SetCoder.java | 7 ++ .../dataflow/sdk/coders/StandardCoder.java | 37 ++++++++++ .../dataflow/sdk/coders/StringUtf8Coder.java | 4 ++ .../sdk/coders/TableRowJsonCoder.java | 7 ++ .../sdk/coders/TextualIntegerCoder.java | 4 ++ .../dataflow/sdk/coders/VarIntCoder.java | 4 ++ .../dataflow/sdk/coders/VarLongCoder.java | 4 ++ .../cloud/dataflow/sdk/coders/VoidCoder.java | 4 ++ .../sdk/transforms/ApproximateQuantiles.java | 14 +++- .../cloud/dataflow/sdk/transforms/Count.java | 2 +- .../dataflow/sdk/transforms/GroupByKey.java | 9 ++- .../cloud/dataflow/sdk/transforms/Top.java | 7 ++ .../sdk/transforms/join/CoGbkResult.java | 7 ++ .../sdk/transforms/join/UnionCoder.java | 8 +++ .../dataflow/sdk/util/TimerOrElement.java | 7 ++ .../dataflow/sdk/util/WindowedValue.java | 20 ++++++ .../dataflow/sdk/values/TimestampedValue.java | 8 +++ .../dataflow/sdk/coders/CoderProperties.java | 12 ++-- .../sdk/coders/CoderRegistryTest.java | 4 ++ .../sdk/coders/StringDelegateCoderTest.java | 10 +-- .../runners/worker/CombineValuesFnTest.java | 3 + .../sdk/transforms/GroupByKeyTest.java | 3 +- 33 files changed, 301 insertions(+), 16 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java index 3760cb82003bb..c009e6ff7ac11 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java @@ -18,13 +18,18 @@ import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; +import java.util.Arrays; import java.util.List; +import javax.annotation.Nullable; + /** * A {@code Coder} defines how to encode and decode values of type {@code T} into byte streams. * @@ -125,9 +130,31 @@ public T decode(InputStream inStream, Context context) * same for an instance of an object even if produced on different * computers at different times. * + * + * @deprecated {@link #verifyDeterministic()} should be used instead to + * produce explanations of why a given Coder is non-deterministic. */ + @Deprecated public boolean isDeterministic(); + /** + * Throw {@link NonDeterministicException} if the coding is not deterministic. + * + *

    In order for a {@code Coder} to be considered deterministic, + * the following must be true: + *

      + *
    • two values which compare as equal (via {@code Object.equals()} + * or {@code Comparable.compareTo()}, if supported), have the same + * encoding. + *
    • the {@code Coder} always produces a canonical encoding, which is the + * same for an instance of an object even if produced on different + * computers at different times. + *
    + * + * @throws Coder.NonDeterministicException if this coder is not deterministic. + */ + public void verifyDeterministic() throws Coder.NonDeterministicException; + /** * Returns whether {@link #registerByteSizeObserver} cheap enough to * call for every element, that is, if this {@code Coder} can @@ -151,4 +178,47 @@ public T decode(InputStream inStream, Context context) public void registerByteSizeObserver( T value, ElementByteSizeObserver observer, Context context) throws Exception; + + /** + * Exception thrown by {@link Coder#verifyDeterministic()} if the encoding is + * not deterministic. + */ + public static class NonDeterministicException extends Throwable { + private Coder coder; + private List reasons; + + public NonDeterministicException( + Coder coder, String reason, @Nullable NonDeterministicException e) { + this(coder, Arrays.asList(reason), e); + } + + public NonDeterministicException(Coder coder, String reason) { + this(coder, Arrays.asList(reason), null); + } + + public NonDeterministicException(Coder coder, List reasons) { + this(coder, reasons, null); + } + + public NonDeterministicException( + Coder coder, + List reasons, + @Nullable NonDeterministicException cause) { + super(cause); + Preconditions.checkArgument(reasons.size() > 0, + "Reasons must not be empty."); + this.reasons = reasons; + this.coder = coder; + } + + public Iterable getReasons() { + return reasons; + } + + @Override + public String getMessage() { + return String.format("%s is not deterministic because:\n %s", + coder, Joiner.on("\n ").join(reasons)); + } + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java index 6b31297a1071c..f7ce7003217a4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java @@ -73,6 +73,7 @@ public CloudObject asCloudObject() { } @Override + @Deprecated public boolean isDeterministic() { return false; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java index 9cc75167872fd..6d3f926145e02 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java @@ -66,10 +66,16 @@ public T decode(InputStream inStream, Context context) throws CoderException, IO } @Override + @Deprecated public boolean isDeterministic() { return coder.isDeterministic(); } + @Override + public void verifyDeterministic() throws NonDeterministicException { + coder.verifyDeterministic(); + } + @Override public String toString() { return "DelegateCoder(" + coder + ")"; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java index 1726ea9ed127d..ee647d4fbcc1a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DoubleCoder.java @@ -70,10 +70,17 @@ public Double decode(InputStream inStream, Context context) * recommended for use in operations which require deterministic inputs. */ @Override + @Deprecated public boolean isDeterministic() { return false; } + @Override + public void verifyDeterministic() throws NonDeterministicException { + throw new NonDeterministicException(this, + "Floating point encodings are not guaranteed to be deterministic."); + } + /** * Returns true since registerByteSizeObserver() runs in constant time. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java index da9769303b702..fe24a55ce7257 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/EntityCoder.java @@ -77,7 +77,14 @@ protected long getEncodedElementByteSize(Entity value, Context context) * which makes the encoding non-deterministic. */ @Override + @Deprecated public boolean isDeterministic() { return false; } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + throw new NonDeterministicException(this, + "Datastore encodings can hold arbitrary Object instances"); + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java index 5918eb220aef6..3ec04e61ba2d1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/InstantCoder.java @@ -19,6 +19,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import org.joda.time.Instant; + import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -55,7 +56,11 @@ public Instant decode(InputStream inStream, Context context) } @Override + @Deprecated public boolean isDeterministic() { return true; } + + @Override + public void verifyDeterministic() { } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java index c9332853e267f..e8cfb29a9fc60 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java @@ -137,10 +137,17 @@ public List> getCoderArguments() { * while the encoding differs. */ @Override + @Deprecated public boolean isDeterministic() { return false; } + @Override + public void verifyDeterministic() throws NonDeterministicException { + throw new NonDeterministicException(this, + "IterableLikeCoder can not guarantee deterministic ordering."); + } + /** * Returns whether iterable can use lazy counting, since that * requires minimal extra computation. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java index bf6e7c9d78892..f2276d9ea2120 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoder.java @@ -102,10 +102,17 @@ public List> getCoderArguments() { } @Override + @Deprecated public boolean isDeterministic() { return getKeyCoder().isDeterministic() && getValueCoder().isDeterministic(); } + @Override + public void verifyDeterministic() throws NonDeterministicException { + verifyDeterministic("Key coder must be deterministic", getKeyCoder()); + verifyDeterministic("Value coder must be deterministic", getValueCoder()); + } + @Override public CloudObject asCloudObject() { CloudObject result = super.asCloudObject(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java index f6f04b60d34ff..90685b8072cca 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ListCoder.java @@ -70,7 +70,15 @@ public static List getInstanceComponents(List exampleValue) { * the general IterableLikeCoder is not. */ @Override + @Deprecated public boolean isDeterministic() { return getElemCoder().isDeterministic(); } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + verifyDeterministic( + "ListCoder.elemCoder must be deterministic", getElemCoder()); + } + } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java index 1e12ef347e70f..25e4a99d61168 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java @@ -126,10 +126,17 @@ public List> getCoderArguments() { * two HashMap instances may be equal but produce different encodings. */ @Override + @Deprecated public boolean isDeterministic() { return false; } + @Override + public void verifyDeterministic() throws NonDeterministicException { + throw new NonDeterministicException(this, + "Ordering of entries in a Map may be non-deterministic."); + } + /** * Notifies ElementByteSizeObserver about the byte size of the * encoded value using this coder. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java index 303370aac0e51..b24b27e558a17 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java @@ -109,10 +109,17 @@ public CloudObject asCloudObject() { } @Override + @Deprecated public boolean isDeterministic() { return false; } + @Override + public void verifyDeterministic() throws NonDeterministicException { + throw new NonDeterministicException(this, + "Java Serialization may be non-deterministic."); + } + @Override public boolean equals(Object other) { if (getClass() != other.getClass()) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java index 9a65d39a0e403..88cae0a64c93a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SetCoder.java @@ -60,10 +60,17 @@ public static SetCoder of( * two {@code HashSet} instances may be equal but produce different encodings. */ @Override + @Deprecated public boolean isDeterministic() { return false; } + @Override + public void verifyDeterministic() throws NonDeterministicException { + throw new NonDeterministicException(this, + "Ordering of elements in a set may be non-deterministic."); + } + /** * Returns the first element in this set if it is non-empty, * otherwise returns {@code null}. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java index 9f475a3d9a786..b5838908497e5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java @@ -24,9 +24,12 @@ import java.io.ByteArrayOutputStream; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; +import javax.annotation.Nullable; + /** * A StandardCoder is one that defines equality, hashing, and printing * via the class name and recursively using {@link #getComponents}. @@ -140,4 +143,38 @@ public void registerByteSizeObserver( throws Exception { observer.update(getEncodedElementByteSize(value, context)); } + + @SuppressWarnings("deprecation") + @Override + public void verifyDeterministic() throws NonDeterministicException { + if (!isDeterministic()) { + throw new NonDeterministicException(this, + getClass().getSimpleName() + " reported it was not determinsitic."); + } + } + + protected void verifyDeterministic(String message, Iterable> coders) + throws NonDeterministicException { + for (Coder coder : coders) { + try { + coder.verifyDeterministic(); + } catch (NonDeterministicException e) { + throw new NonDeterministicException(this, message, e); + } + } + } + + protected void verifyDeterministic(String message, Coder... coders) + throws NonDeterministicException { + verifyDeterministic(message, Arrays.asList(coders)); + } + + protected void addReasons(String prefix, List accumulator, + @Nullable List newReasons) { + if (newReasons != null) { + for (String reason : newReasons) { + accumulator.add(prefix + reason); + } + } + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java index 48b807b6fcf8c..0c903501c0c73 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java @@ -105,10 +105,14 @@ public String decode(InputStream inStream, Context context) } @Override + @Deprecated public boolean isDeterministic() { return true; } + @Override + public void verifyDeterministic() { } + protected long getEncodedElementByteSize(String value, Context context) throws Exception { if (value == null) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java index 349ff323c1519..0ce713c29c256 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java @@ -74,7 +74,14 @@ private TableRowJsonCoder() { } * non-deterministic. */ @Override + @Deprecated public boolean isDeterministic() { return false; } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + throw new NonDeterministicException(this, + "TableCell can hold arbitrary instances which may be non-deterministic."); + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java index 92f343bd4d7e9..a78ae3778e1d7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java @@ -59,10 +59,14 @@ public Integer decode(InputStream inStream, Context context) } @Override + @Deprecated public boolean isDeterministic() { return true; } + @Override + public void verifyDeterministic() { } + protected long getEncodedElementByteSize(Integer value, Context context) throws Exception { if (value == null) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java index f357fce890284..e30a094273f19 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java @@ -68,10 +68,14 @@ public Integer decode(InputStream inStream, Context context) } @Override + @Deprecated public boolean isDeterministic() { return true; } + @Override + public void verifyDeterministic() { } + /** * Returns true since registerByteSizeObserver() runs in constant time. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java index 50866383bca2a..56a20cf4093e7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java @@ -68,10 +68,14 @@ public Long decode(InputStream inStream, Context context) } @Override + @Deprecated public boolean isDeterministic() { return true; } + @Override + public void verifyDeterministic() { } + /** * Returns true since registerByteSizeObserver() runs in constant time. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java index e949fdfd03484..a3bed700eb209 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java @@ -50,10 +50,14 @@ public Void decode(InputStream inStream, Context context) { } @Override + @Deprecated public boolean isDeterministic() { return true; } + @Override + public void verifyDeterministic() { } + /** * Returns true since registerByteSizeObserver() runs in constant time. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java index 5702583ebe65c..3a583f639756d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java @@ -654,8 +654,20 @@ public void registerByteSizeObserver( } @Override + @Deprecated public boolean isDeterministic() { - return elementListCoder.isDeterministic(); + return elementCoder.isDeterministic() + && elementListCoder.isDeterministic(); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException{ + verifyDeterministic( + "QuantileState.ElementCoder must be deterministic", + elementCoder); + verifyDeterministic( + "QuantileState.ElementListCoder must be deterministic", + elementListCoder); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java index b6e4561f2dde4..f5fceb29c5096 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java @@ -120,7 +120,7 @@ public void processElement(ProcessContext c) { * encoding each element using the input {@code PCollection}'s * {@code Coder}, then comparing the encoded bytes. Because of this, * the input coder must be deterministic. (See - * {@link com.google.cloud.dataflow.sdk.coders.Coder#isDeterministic()} for more detail). + * {@link com.google.cloud.dataflow.sdk.coders.Coder#verifyDeterministic()} for more detail). * Performing the comparison in this manner admits efficient parallel evaluation. * *

    By default, the {@code Coder} of the keys of the output diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index 471229f308e9a..f89b224fb9829 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -19,6 +19,7 @@ import static com.google.cloud.dataflow.sdk.util.CoderUtils.encodeToByteArray; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException; import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.coders.KvCoder; @@ -65,7 +66,7 @@ * keys of the input {@code PCollection}, and then comparing the * encoded bytes. This admits efficient parallel evaluation. Note that * this requires that the {@code Coder} of the keys be deterministic (see - * {@link Coder#isDeterministic()}). If the key {@code Coder} is not + * {@link Coder#verifyDeterministic()}). If the key {@code Coder} is not * deterministic, an exception is thrown at runtime. * *

    By default, the {@code Coder} of the keys of the output @@ -295,9 +296,11 @@ public void finishSpecifying() { // Verify that the input Coder> is a KvCoder, and that // the key coder is deterministic. Coder keyCoder = getKeyCoder(); - if (!keyCoder.isDeterministic()) { + try { + keyCoder.verifyDeterministic(); + } catch (NonDeterministicException e) { throw new IllegalStateException( - "the key Coder must be deterministic for grouping"); + "the keyCoder of a GroupByKey must be deterministic", e); } if (getOutput().isOrdered()) { throw new IllegalStateException( diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java index c4fb15bd23232..bc0270f01bce7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java @@ -441,10 +441,17 @@ public Heap decode(InputStream inStream, Coder.Context context) } @Override + @Deprecated public boolean isDeterministic() { return listCoder.isDeterministic(); } + @Override + public void verifyDeterministic() throws NonDeterministicException { + verifyDeterministic( + "HeapCoder requires a deterministic list coder", listCoder); + } + @Override public boolean isRegisterByteSizeObserverCheap( Heap value, Context context) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java index 56fc3fc84bc0a..81b12d1c09792 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java @@ -234,9 +234,16 @@ public boolean equals(Object other) { } @Override + @Deprecated public boolean isDeterministic() { return mapCoder.isDeterministic(); } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + verifyDeterministic( + "CoGbkResult requires the mapCoder to be deterministic", mapCoder); + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java index f3d128c67b9e1..a2b56c2f1cabb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/UnionCoder.java @@ -140,6 +140,7 @@ private UnionCoder(List> elementCoders) { } @Override + @Deprecated public boolean isDeterministic() { for (Coder elementCoder : elementCoders) { if (!elementCoder.isDeterministic()) { @@ -149,4 +150,11 @@ public boolean isDeterministic() { return true; } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + verifyDeterministic( + "UnionCoder is only deterministic if all element coders are", + elementCoders); + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerOrElement.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerOrElement.java index d0216d3075782..5303ca3ec844f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerOrElement.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerOrElement.java @@ -155,11 +155,18 @@ public void registerByteSizeObserver( } } + @Deprecated @Override public boolean isDeterministic() { return elemCoder.isDeterministic(); } + @Override + public void verifyDeterministic() throws NonDeterministicException { + verifyDeterministic( + "TimerOrElementCoder requires a deterministic elemCoder", elemCoder); + } + @Override public List> getCoderArguments() { return Arrays.asList(elemCoder); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java index 4897a3e10c229..f724e440261d7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java @@ -275,10 +275,21 @@ public WindowedValue decode(InputStream inStream, Context context) } @Override + @Deprecated public boolean isDeterministic() { return valueCoder.isDeterministic() && windowCoder.isDeterministic(); } + @Override + public void verifyDeterministic() throws NonDeterministicException { + verifyDeterministic( + "FullWindowedValueCoder requires a deterministic valueCoder", + valueCoder); + verifyDeterministic( + "FullWindowedValueCoder requires a deterministic windowCoder", + windowCoder); + } + @Override public void registerByteSizeObserver(WindowedValue value, ElementByteSizeObserver observer, @@ -351,10 +362,19 @@ public WindowedValue decode(InputStream inStream, Context context) } @Override + @Deprecated public boolean isDeterministic() { return valueCoder.isDeterministic(); } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + verifyDeterministic( + "ValueOnlyWindowedValueCoder requires a deterministic valueCoder", + valueCoder); + } + @Override public void registerByteSizeObserver( WindowedValue value, ElementByteSizeObserver observer, Context context) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java index ffa0202e15b2c..1886b7f0a9490 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java @@ -108,10 +108,18 @@ public TimestampedValue decode(InputStream inStream, Context context) } @Override + @Deprecated public boolean isDeterministic() { return valueCoder.isDeterministic(); } + @Override + public void verifyDeterministic() throws NonDeterministicException { + verifyDeterministic( + "TimestampedValueCoder requires a deterministic valueCoder", + valueCoder); + } + @Override public List> getCoderArguments() { return Arrays.>asList(valueCoder); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java index 1b9200a9164de..f846418284173 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderProperties.java @@ -21,13 +21,13 @@ import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; +import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException; import com.google.cloud.dataflow.sdk.util.SerializableUtils; import com.google.cloud.dataflow.sdk.util.Serializer; import com.google.common.collect.Iterables; -import org.hamcrest.CoreMatchers; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -71,8 +71,12 @@ public static void coderDeterministic( public static void coderDeterministicInContext( Coder coder, Coder.Context context, T value1, T value2) throws Exception { - assertThat("Expected that the coder is deterministic", - coder.isDeterministic(), CoreMatchers.is(true)); + + try { + coder.verifyDeterministic(); + } catch (NonDeterministicException e) { + fail("Expected that the coder is deterministic"); + } assertThat("Expected that the passed in values are equal()", value1, equalTo(value2)); assertThat( encode(coder, context, value1), diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java index 2c411fae42316..c2a1e6c9f023f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java @@ -218,9 +218,13 @@ public CloudObject asCloudObject() { return null; } + @Deprecated @Override public boolean isDeterministic() { return true; } + @Override + public void verifyDeterministic() { } + @Override public boolean isRegisterByteSizeObserverCheap(MyValue value, Context context) { return true; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoderTest.java index f930dfafbda68..d74a74f7ac57d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoderTest.java @@ -19,6 +19,8 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException; + import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -44,12 +46,10 @@ public class StringDelegateCoderTest { // Tests - private static final List TEST_CONTEXTS = Arrays.asList( - Coder.Context.NESTED, - Coder.Context.OUTER); - + @SuppressWarnings("deprecation") @Test - public void testDeterministic() throws Exception { + public void testDeterministic() throws Exception, NonDeterministicException { + uriCoder.verifyDeterministic(); assertThat(uriCoder.isDeterministic(), equalTo(true)); for (String uriString : TEST_URI_STRINGS) { CoderProperties.coderDeterministic(uriCoder, new URI(uriString), new URI(uriString)); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java index abddf51ad5bc9..a8040b6347df6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFnTest.java @@ -150,6 +150,9 @@ public MeanInts.CountSum decode(InputStream inStream, Context context) @Override public boolean isDeterministic() { return true; } + @Override + public void verifyDeterministic() { } + public CloudObject asCloudObject() { return makeCloudEncoding(this.getClass().getName()); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java index 5d97a7a657b93..5cc50dc1e7aa1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java @@ -35,7 +35,6 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; -import org.hamcrest.Matchers; import org.joda.time.Duration; import org.junit.Assert; import org.junit.Rule; @@ -168,7 +167,7 @@ public void testGroupByKeyEmpty() { @Test public void testGroupByKeyNonDeterministic() throws Exception { expectedEx.expect(IllegalStateException.class); - expectedEx.expectMessage(Matchers.containsString("must be deterministic")); + expectedEx.expectMessage("must be deterministic"); List, Integer>> ungroupedPairs = Arrays.asList(); From bf6b39189aa0aafd590dc831e18267aba302443f Mon Sep 17 00:00:00 2001 From: davor Date: Fri, 20 Feb 2015 15:41:59 -0800 Subject: [PATCH 0172/1541] Fix cross-platform line endings issues by specifying repository-wide settings. This fixes #7 on GitHub. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86828537 --- .gitattributes | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) create mode 100644 .gitattributes diff --git a/.gitattributes b/.gitattributes new file mode 100644 index 0000000000000..cce74a2d901d5 --- /dev/null +++ b/.gitattributes @@ -0,0 +1,26 @@ +# The default behavior, which overrides 'core.autocrlf', is to use Git's +# built-in heuristics to determine whether a particular file is text or binary. +# Text files are automatically normalized to the user's platforms. +* text=auto + +# Explicitly declare text files that should always be normalized and converted +# to native line endings. +.gitattributes text +.gitignore text +LICENSE text +*.avsc text +*.html text +*.java text +*.md text +*.properties text +*.proto text +*.py text +*.sh text +*.xml text +*.yml text + +# Declare files that will always have CRLF line endings on checkout. +# *.sln text eol=crlf + +# Explicitly denote all files that are truly binary and should not be modified. +# *.jpg binary From 021243892bf3b2457ae0e72bfb186183b0bb86f7 Mon Sep 17 00:00:00 2001 From: chamikara Date: Sat, 21 Feb 2015 00:15:10 -0800 Subject: [PATCH 0173/1541] Fix following bugs in direct runner GCS file pattern expansion. * Add support for range patterns (e.g., "gs:[][1-5]"). * GcsUtil.expand() should give zero results for directories. Updated storage field of GcsUtil to be non-final and added a protected setter so that Storage can be mocked for testing purposes. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86853046 --- .../cloud/dataflow/sdk/util/GcsUtil.java | 31 +++-- .../cloud/dataflow/sdk/util/GcsUtilTest.java | 122 ++++++++++++++++++ 2 files changed, 142 insertions(+), 11 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java index dd51b669e2820..6f7ae37cb2c91 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java @@ -33,7 +33,6 @@ import java.io.IOException; import java.nio.channels.SeekableByteChannel; import java.nio.channels.WritableByteChannel; -import java.util.Arrays; import java.util.LinkedList; import java.util.List; import java.util.concurrent.ExecutorService; @@ -71,7 +70,7 @@ public GcsUtil create(PipelineOptions options) { private static final long MAX_LIST_ITEMS_PER_CALL = 1024; /** Matches a glob containing a wildcard, capturing the portion before the first wildcard. */ - private static final Pattern GLOB_PREFIX = Pattern.compile("(?[^*?]*)[*?].*"); + private static final Pattern GLOB_PREFIX = Pattern.compile("(?[^\\[*?]*)[\\[*?].*"); private static final String RECURSIVE_WILDCARD = "[*]{2}"; @@ -84,7 +83,7 @@ public GcsUtil create(PipelineOptions options) { ///////////////////////////////////////////////////////////////////////////// /** Client for the GCS API. */ - private final Storage storage; + private Storage storage; // Helper delegate for turning IOExceptions from API calls into higher-level semantics. private final ApiErrorExtractor errorExtractor = new ApiErrorExtractor(); @@ -110,22 +109,32 @@ private GcsUtil(Storage storageClient, ExecutorService executorService) { this.executorService = executorService; } + // Use this only for testing purposes. + protected void setStorageClient(Storage storage) { + this.storage = storage; + } + /** - * Expands a pattern into matched paths. The pattern path may contain - * globs, which are expanded in the result. + * Expands a pattern into matched paths. The pattern path may contain globs, which are expanded in + * the result. This function validates the existence of each matched file in GCS. */ public List expand(GcsPath gcsPattern) throws IOException { Preconditions.checkArgument(isGcsPatternSupported(gcsPattern.getObject())); Matcher m = GLOB_PREFIX.matcher(gcsPattern.getObject()); + Pattern p = null; + String prefix = null; if (!m.matches()) { - return Arrays.asList(gcsPattern); + // Not a glob. But we should verify that the file exists in GCS. + prefix = gcsPattern.getObject(); + p = Pattern.compile(gcsPattern.getObject()); + } else { + // Part before the first wildcard character. + prefix = m.group("PREFIX"); + p = Pattern.compile(globToRegexp(gcsPattern.getObject())); } - // Part before the first wildcard character. - String prefix = m.group("PREFIX"); - Pattern p = Pattern.compile(globToRegexp(gcsPattern.getObject())); - LOG.info("matching files in bucket {}, prefix {} against pattern {}", - gcsPattern.getBucket(), prefix, p.toString()); + LOG.info("matching files in bucket {}, prefix {} against pattern {}", gcsPattern.getBucket(), + prefix, p.toString()); // List all objects that start with the prefix (including objects in sub-directories). Storage.Objects.List listObject = storage.objects().list(gcsPattern.getBucket()); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GcsUtilTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GcsUtilTest.java index c7a7c55f5b585..e6ec624f45373 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GcsUtilTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GcsUtilTest.java @@ -16,21 +16,34 @@ package com.google.cloud.dataflow.sdk.util; +import static org.hamcrest.Matchers.contains; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.when; import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.util.Throwables; +import com.google.api.services.storage.Storage; +import com.google.api.services.storage.model.Objects; +import com.google.api.services.storage.model.StorageObject; import com.google.cloud.dataflow.sdk.options.GcsOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.common.collect.ImmutableList; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.Mockito; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -39,6 +52,8 @@ /** Test case for {@link GcsUtil}. */ @RunWith(JUnit4.class) public class GcsUtilTest { + @Rule public ExpectedException exception = ExpectedException.none(); + @Test public void testGlobTranslation() { assertEquals("foo", GcsUtil.globToRegexp("foo")); @@ -102,4 +117,111 @@ public void run() { assertTrue("Expected tasks to complete", executorService.awaitTermination(10, TimeUnit.SECONDS)); } + + @Test + public void testGlobExpansion() throws IOException { + GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class); + pipelineOptions.setGcpCredential(Mockito.mock(Credential.class)); + GcsUtil gcsUtil = pipelineOptions.getGcsUtil(); + + Storage mockStorage = Mockito.mock(Storage.class); + gcsUtil.setStorageClient(mockStorage); + + Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class); + Storage.Objects.List mockStorageList = Mockito.mock(Storage.Objects.List.class); + + Objects modelObjects = new Objects(); + List items = new ArrayList<>(); + // A directory + items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/")); + + // Files within the directory + items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/file1name")); + items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/file2name")); + items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/file3name")); + items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/otherfile")); + items.add(new StorageObject().setBucket("testbucket").setName("testdirectory/anotherfile")); + + modelObjects.setItems(items); + + when(mockStorage.objects()).thenReturn(mockStorageObjects); + when(mockStorageObjects.list("testbucket")).thenReturn(mockStorageList); + when(mockStorageList.execute()).thenReturn(modelObjects); + + // Test a single file. + { + GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/otherfile"); + List expectedFiles = + ImmutableList.of(GcsPath.fromUri("gs://testbucket/testdirectory/otherfile")); + + assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray())); + } + + // Directories should not match. + { + GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/"); + List pathList = gcsUtil.expand(pattern); + assertEquals(pathList.size(), 0); + } + + { + GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory"); + List pathList = gcsUtil.expand(pattern); + assertEquals(pathList.size(), 0); + } + + // Test patterns. + { + GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/file*"); + List expectedFiles = ImmutableList.of( + GcsPath.fromUri("gs://testbucket/testdirectory/file1name"), + GcsPath.fromUri("gs://testbucket/testdirectory/file2name"), + GcsPath.fromUri("gs://testbucket/testdirectory/file3name")); + + assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray())); + } + + { + GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/file[1-3]*"); + List expectedFiles = ImmutableList.of( + GcsPath.fromUri("gs://testbucket/testdirectory/file1name"), + GcsPath.fromUri("gs://testbucket/testdirectory/file2name"), + GcsPath.fromUri("gs://testbucket/testdirectory/file3name")); + + assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray())); + } + + { + GcsPath pattern = GcsPath.fromUri("gs://testbucket/testdirectory/file?name"); + List expectedFiles = ImmutableList.of( + GcsPath.fromUri("gs://testbucket/testdirectory/file1name"), + GcsPath.fromUri("gs://testbucket/testdirectory/file2name"), + GcsPath.fromUri("gs://testbucket/testdirectory/file3name")); + + assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray())); + } + + { + GcsPath pattern = GcsPath.fromUri("gs://testbucket/test*ectory/fi*name"); + List expectedFiles = ImmutableList.of( + GcsPath.fromUri("gs://testbucket/testdirectory/file1name"), + GcsPath.fromUri("gs://testbucket/testdirectory/file2name"), + GcsPath.fromUri("gs://testbucket/testdirectory/file3name")); + + assertThat(expectedFiles, contains(gcsUtil.expand(pattern).toArray())); + } + } + + // Patterns that contain recursive wildcards ('**') are not supported. + @Test + public void testRecursiveGlobExpansionFails() throws IOException { + GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class); + pipelineOptions.setGcpCredential(Mockito.mock(Credential.class)); + GcsUtil gcsUtil = pipelineOptions.getGcsUtil(); + GcsPath pattern = GcsPath.fromUri("gs://testbucket/test**"); + + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Unsupported wildcard usage"); + gcsUtil.expand(pattern); + } } From 207007ba498662e8e6953e3eff978789d31ab2d9 Mon Sep 17 00:00:00 2001 From: davor Date: Sat, 21 Feb 2015 10:53:36 -0800 Subject: [PATCH 0174/1541] Travis CI: add retries to decrease the false-positive rate. Ideally, we should be using 'travis_retry' on 'mvn dependency:go-offline' only. However, due to http://jira.codehaus.org/browse/MDEP-82, that doesn't work. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86870674 --- .travis.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index dc499659491a4..a9e4b84d71244 100644 --- a/.travis.yml +++ b/.travis.yml @@ -36,8 +36,8 @@ before_install: - if [ "$TRAVIS_OS_NAME" == "linux" ]; then jdk_switcher use "$CUSTOM_JDK"; fi install: - - mvn install clean -U -DskipTests=true + - travis_retry mvn install clean -U -DskipTests=true script: - - mvn verify -U - - ./test_wordcount.sh + - travis_retry mvn verify -U + - travis_retry ./test_wordcount.sh From 1f8d4c4bd821e1b9cb545e6bfb3eaccf58af5108 Mon Sep 17 00:00:00 2001 From: malo Date: Sat, 21 Feb 2015 13:19:49 -0800 Subject: [PATCH 0175/1541] Add GenomicsVariantSimilarity integration test. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86874467 --- checkstyle.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/checkstyle.xml b/checkstyle.xml index c6c07ef9f1c98..062f1346d9583 100644 --- a/checkstyle.xml +++ b/checkstyle.xml @@ -103,7 +103,7 @@ page at http://checkstyle.sourceforge.net/config.html --> - + From 63a89683f15475e069dd0911db25b8e2d99dbfd7 Mon Sep 17 00:00:00 2001 From: lcwik Date: Mon, 23 Feb 2015 11:07:16 -0800 Subject: [PATCH 0176/1541] Add support for toString() to print out serialized pipeline options in addition to the ones programmatically set. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86973245 --- .../sdk/options/ProxyInvocationHandler.java | 15 ++++++++-- .../options/ProxyInvocationHandlerTest.java | 28 +++++++++++++++++++ 2 files changed, 40 insertions(+), 3 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java index e74e5a8f6e820..0212bfa463c3e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java @@ -54,6 +54,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.SortedMap; import java.util.TreeMap; /** @@ -182,15 +183,23 @@ public int hashCode() { } /** - * This will output all the currently set values. + * This will output all the currently set values. This is a relatively costly function + * as it will call {@code toString()} on each object that has been set and format + * the results in a readable format. * - * @return A string representation of this. + * @return A pretty printed string representation of this. */ @Override public synchronized String toString() { + SortedMap sortedOptions = new TreeMap<>(); + // Add the options that we received from deserialization + sortedOptions.putAll(jsonOptions); + // Override with any programmatically set options. + sortedOptions.putAll(options); + StringBuilder b = new StringBuilder(); b.append("Current Settings:\n"); - for (Map.Entry entry : new TreeMap<>(options).entrySet()) { + for (Map.Entry entry : sortedOptions.entrySet()) { b.append(" " + entry.getKey() + ": " + entry.getValue() + "\n"); } return b.toString(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java index d893ba4d08d30..8d2f859f83d82 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java @@ -235,6 +235,34 @@ public void testToString() throws Exception { proxy.toString()); } + @Test + public void testToStringAfterDeserializationContainsJsonEntries() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + Simple proxy = handler.as(Simple.class); + proxy.setString("stringValue"); + DefaultAnnotations proxy2 = proxy.as(DefaultAnnotations.class); + proxy2.setLong(57L); + assertEquals("Current Settings:\n" + + " long: 57\n" + + " string: \"stringValue\"\n", + serializeDeserialize(PipelineOptions.class, proxy2).toString()); + } + + @Test + public void testToStringAfterDeserializationContainsOverriddenEntries() throws Exception { + ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.newHashMap()); + Simple proxy = handler.as(Simple.class); + proxy.setString("stringValue"); + DefaultAnnotations proxy2 = proxy.as(DefaultAnnotations.class); + proxy2.setLong(57L); + Simple deserializedOptions = serializeDeserialize(Simple.class, proxy2); + deserializedOptions.setString("overriddenValue"); + assertEquals("Current Settings:\n" + + " long: 57\n" + + " string: overriddenValue\n", + deserializedOptions.toString()); + } + /** A test interface containing an unknown method. */ public static interface UnknownMethod { void unknownMethod(); From 170376d652b3d0fc894d7dfc87c760f255ea236e Mon Sep 17 00:00:00 2001 From: relax Date: Mon, 23 Feb 2015 11:35:42 -0800 Subject: [PATCH 0177/1541] Fix retry logic in BigQueryIO. Previously if an insert failed, we threw an exception. However this caused us to retry all of the rows processed in the bundle, even if the failure was only for a single row. Now we detect which rows failed and attempt to retry only those rows. If insert fails five times in a row, we throw an exception to make sure the failure is exposed to the user. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=86976384 --- .../cloud/dataflow/sdk/io/BigQueryIO.java | 4 +- .../sdk/util/BigQueryTableInserter.java | 150 ++++++++++++------ .../dataflow/sdk/util/BigQueryUtilTest.java | 68 ++++++++ 3 files changed, 175 insertions(+), 47 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java index dca65ad9f592e..63e29aaf07162 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java @@ -718,7 +718,7 @@ private void flushRows(BigQueryOptions options) { Bigquery client = Transport.newBigQueryClient(options).build(); try { BigQueryTableInserter inserter = new BigQueryTableInserter(client, tableReference); - inserter.insertAll(tableRows.iterator(), uniqueIdsForTableRows.iterator()); + inserter.insertAll(tableRows, uniqueIdsForTableRows); } catch (IOException e) { throw new RuntimeException(e); } @@ -854,7 +854,7 @@ private static void evaluateWriteHelper( transform.writeDisposition, transform.createDisposition, transform.schema); List tableRows = context.getPCollection(transform.getInput()); - inserter.insertAll(tableRows.iterator()); + inserter.insertAll(tableRows); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableInserter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableInserter.java index 0ae41a057b1b2..dbe8a26a7d672 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableInserter.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BigQueryTableInserter.java @@ -32,7 +32,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.Iterator; +import java.util.ArrayList; import java.util.LinkedList; import java.util.List; @@ -50,8 +50,15 @@ public class BigQueryTableInserter { // The maximum number of rows to upload per InsertAll request. private static final long MAX_ROWS_PER_BATCH = 500; + // The maximum number of times to retry inserting rows into BigQuery. + private static final int MAX_INSERT_ATTEMPTS = 5; + + // The initial backoff after a failure inserting rows into BigQuery. + private static final long INITIAL_INSERT_BACKOFF_INTERVAL_MS = 200L; + private final Bigquery client; private final TableReference ref; + private final long maxRowsPerBatch; /** * Constructs a new row inserter. @@ -62,63 +69,116 @@ public class BigQueryTableInserter { public BigQueryTableInserter(Bigquery client, TableReference ref) { this.client = client; this.ref = ref; + this.maxRowsPerBatch = MAX_ROWS_PER_BATCH; } /** - * Insert all rows from the given iterator. + * Constructs a new row inserter. + * + * @param client a BigQuery client + * @param ref identifies the table to insert into */ - public void insertAll(Iterator rowIterator) throws IOException { - insertAll(rowIterator, null); + public BigQueryTableInserter(Bigquery client, TableReference ref, int maxRowsPerBatch) { + this.client = client; + this.ref = ref; + this.maxRowsPerBatch = maxRowsPerBatch; } /** - * Insert all rows from the given iterator using specified insertIds if not null. + * Insert all rows from the given list. */ - public void insertAll(Iterator rowIterator, - @Nullable Iterator insertIdIterator) throws IOException { - // Upload in batches. - List rows = new LinkedList<>(); - int numInserted = 0; - int dataSize = 0; - while (rowIterator.hasNext()) { - TableRow row = rowIterator.next(); - TableDataInsertAllRequest.Rows out = new TableDataInsertAllRequest.Rows(); - if (insertIdIterator != null) { - if (insertIdIterator.hasNext()) { - out.setInsertId(insertIdIterator.next()); - } else { - throw new AssertionError("If insertIdIterator is not null it needs to have at least " - + "as many elements as rowIterator"); - } + public void insertAll(List rowList) throws IOException { + insertAll(rowList, null); + } + + /** + * Insert all rows from the given list using specified insertIds if not null. + */ + public void insertAll(List rowList, + @Nullable List insertIdList) throws IOException { + if (insertIdList != null && rowList.size() != insertIdList.size()) { + throw new AssertionError("If insertIdList is not null it needs to have at least " + + "as many elements as rowList"); + } + + + AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff( + MAX_INSERT_ATTEMPTS, + INITIAL_INSERT_BACKOFF_INTERVAL_MS); + + List allErrors = new ArrayList<>(); + // These lists contain the rows to publish. Initially the contain the entire list. If there are + // failures, they will contain only the failed rows to be retried. + List rowsToPublish = rowList; + List idsToPublish = insertIdList; + while (true) { + List retryRows = new ArrayList<>(); + List retryIds = null; + if (idsToPublish != null) { + retryIds = new ArrayList<>(); } - out.setJson(row.getUnknownKeys()); - rows.add(out); - - dataSize += row.toString().length(); - if (dataSize >= UPLOAD_BATCH_SIZE_BYTES || rows.size() >= MAX_ROWS_PER_BATCH || - !rowIterator.hasNext()) { - TableDataInsertAllRequest content = new TableDataInsertAllRequest(); - content.setRows(rows); - - LOG.info("Number of rows in BigQuery insert: {}", rows.size()); - numInserted += rows.size(); - - Bigquery.Tabledata.InsertAll insert = client.tabledata() - .insertAll(ref.getProjectId(), ref.getDatasetId(), ref.getTableId(), - content); - TableDataInsertAllResponse response = insert.execute(); - List errors = response - .getInsertErrors(); - if (errors != null && !errors.isEmpty()) { - throw new IOException("Insert failed: " + errors); + int strideIndex = 0; + // Upload in batches. + List rows = new LinkedList<>(); + int dataSize = 0; + for (int i = 0; i < rowsToPublish.size(); ++i) { + TableRow row = rowsToPublish.get(i); + TableDataInsertAllRequest.Rows out = new TableDataInsertAllRequest.Rows(); + if (idsToPublish != null) { + out.setInsertId(idsToPublish.get(i)); } + out.setJson(row.getUnknownKeys()); + rows.add(out); + + dataSize += row.toString().length(); + if (dataSize >= UPLOAD_BATCH_SIZE_BYTES || rows.size() >= maxRowsPerBatch || + i == rowsToPublish.size() - 1) { + TableDataInsertAllRequest content = new TableDataInsertAllRequest(); + content.setRows(rows); + + Bigquery.Tabledata.InsertAll insert = client.tabledata() + .insertAll(ref.getProjectId(), ref.getDatasetId(), ref.getTableId(), + content); + TableDataInsertAllResponse response = insert.execute(); + List errors = response.getInsertErrors(); + if (errors != null) { + allErrors.addAll(errors); + for (TableDataInsertAllResponse.InsertErrors error : errors) { + if (error.getIndex() == null) { + throw new IOException("Insert failed: " + allErrors); + } + + int errorIndex = error.getIndex().intValue() + strideIndex; + retryRows.add(rowsToPublish.get(errorIndex)); + if (retryIds != null) { + retryIds.add(idsToPublish.get(errorIndex)); + } + } + } - dataSize = 0; - rows.clear(); + dataSize = 0; + strideIndex = i + 1; + rows.clear(); + } } - } - LOG.info("Number of rows written to BigQuery: {}", numInserted); + if (!allErrors.isEmpty() && !backoff.atMaxAttempts()) { + try { + Thread.sleep(backoff.nextBackOffMillis()); + } catch (InterruptedException e) { + // ignore. + } + LOG.info("Retrying failed inserts to BigQuery"); + rowsToPublish = retryRows; + idsToPublish = retryIds; + allErrors.clear(); + } else { + break; + } + } + if (!allErrors.isEmpty()) { + throw new IOException("Insert failed: " + allErrors); + } } /** diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BigQueryUtilTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BigQueryUtilTest.java index ca75e6f94ca7b..783c44847a6cf 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BigQueryUtilTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BigQueryUtilTest.java @@ -16,10 +16,12 @@ package com.google.cloud.dataflow.sdk.util; +import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; @@ -27,6 +29,8 @@ import com.google.api.services.bigquery.Bigquery; import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableCell; +import com.google.api.services.bigquery.model.TableDataInsertAllRequest; +import com.google.api.services.bigquery.model.TableDataInsertAllResponse; import com.google.api.services.bigquery.model.TableDataList; import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableReference; @@ -49,6 +53,7 @@ import org.mockito.MockitoAnnotations; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.LinkedList; import java.util.List; @@ -66,6 +71,7 @@ public class BigQueryUtilTest { @Mock private Bigquery.Tables mockTables; @Mock private Bigquery.Tables.Get mockTablesGet; @Mock private Bigquery.Tabledata mockTabledata; + @Mock private Bigquery.Tabledata.InsertAll mockInsertAll; @Mock private Bigquery.Tabledata.List mockTabledataList; @Before @@ -82,6 +88,39 @@ public void tearDown() { verifyNoMoreInteractions(mockTabledataList); } + private void onInsertAll(List> errorIndicesSequence) throws Exception { + when(mockClient.tabledata()) + .thenReturn(mockTabledata); + + List responses = new ArrayList<>(); + for (List errorIndices : errorIndicesSequence) { + List errors = new ArrayList<>(); + for (long i : errorIndices) { + TableDataInsertAllResponse.InsertErrors error = + new TableDataInsertAllResponse.InsertErrors(); + error.setIndex(i); + } + TableDataInsertAllResponse response = new TableDataInsertAllResponse(); + response.setInsertErrors(errors); + responses.add(response); + } + + + when(mockTabledata.insertAll( + anyString(), anyString(), anyString(), any(TableDataInsertAllRequest.class))) + .thenReturn(mockInsertAll); + when(mockInsertAll.execute()) + .thenReturn(responses.get(0), + responses.subList(1, responses.size()).toArray( + new TableDataInsertAllResponse[responses.size() - 1])); + } + + private void verifyInsertAll(int expectedRetries) throws IOException { + verify(mockClient, times(expectedRetries)).tabledata(); + verify(mockTabledata, times(expectedRetries)) + .insertAll(anyString(), anyString(), anyString(), any(TableDataInsertAllRequest.class)); + } + private void onTableGet(Table table) throws IOException { when(mockClient.tables()) .thenReturn(mockTables); @@ -303,4 +342,33 @@ public void testWriteEmptyFail() throws IOException { verifyTabledataList(); } } + + @Test + public void testInsertAll() throws Exception, IOException { + // Build up a list of indices to fail on each invocation. This should result in + // 5 calls to insertAll. + List> errorsIndices = new ArrayList<>(); + errorsIndices.add(Arrays.asList(0L, 5L, 10L, 15L, 20L)); + errorsIndices.add(Arrays.asList(0L, 2L, 4L)); + errorsIndices.add(Arrays.asList(0L, 2L)); + errorsIndices.add(new ArrayList()); + onInsertAll(errorsIndices); + + TableReference ref = BigQueryIO + .parseTableSpec("project:dataset.table"); + BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, ref, 5); + + List rows = new ArrayList<>(); + List ids = new ArrayList<>(); + for (int i = 0; i < 25; ++i) { + rows.add(new TableRow()); + ids.add(new String()); + } + + try { + inserter.insertAll(rows, ids); + } finally { + verifyInsertAll(5); + } + } } From 09361a516c329a66082b1f6a03699888df4b7b25 Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 20 Feb 2015 14:47:56 +0100 Subject: [PATCH 0178/1541] increase scope of a few classes to write APIs against them --- .../sdk/transforms/join/CoGbkResultSchema.java | 2 +- .../transforms/join/KeyedPCollectionTuple.java | 17 ++++++++++++++++- .../sdk/transforms/join/RawUnionValue.java | 2 +- 3 files changed, 18 insertions(+), 3 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultSchema.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultSchema.java index 6ab1042e3b186..6bb9d055c5018 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultSchema.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultSchema.java @@ -38,7 +38,7 @@ * CoGroupByKey). */ @SuppressWarnings("serial") -class CoGbkResultSchema implements Serializable { +public class CoGbkResultSchema implements Serializable { private final TupleTagList tupleTagList; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/KeyedPCollectionTuple.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/KeyedPCollectionTuple.java index a9fd4b684f85b..1dfbf9efbf72c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/KeyedPCollectionTuple.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/KeyedPCollectionTuple.java @@ -151,7 +151,8 @@ public void finishSpecifying() { * A utility class to help ensure coherence of tag and input PCollection * types. */ - static class TaggedKeyedPCollection { + public static class TaggedKeyedPCollection { + final TupleTag tupleTag; final PCollection> pCollection; @@ -161,6 +162,20 @@ public TaggedKeyedPCollection( this.tupleTag = tupleTag; this.pCollection = pCollection; } + + /** + * Returns the underlying PCollection of this TaggedKeyedPCollection. + */ + public PCollection> getCollection() { + return pCollection; + } + + /** + * Returns the TupleTag of this TaggedKeyedPCollection. + */ + public TupleTag getTupleTag() { + return tupleTag; + } } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/RawUnionValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/RawUnionValue.java index b52f8b3e49c2f..3b6fa73868732 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/RawUnionValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/RawUnionValue.java @@ -23,7 +23,7 @@ * This corresponds to an integer union tag and value. The mapping of * union tag to type must come from elsewhere. */ -class RawUnionValue { +public class RawUnionValue { private final int unionTag; private final Object value; From 3604b24906a1c28f33d7a988fcfdecdf1c592c1f Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 20 Feb 2015 11:04:45 +0100 Subject: [PATCH 0179/1541] Add Getter for fn in Combine.PerKey This enables PipelineRunners to generate a more efficient operation that does the groping and aggregation of values in one operation instead of first grouping and then aggregating in two steps. --- .../com/google/cloud/dataflow/sdk/transforms/Combine.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java index 429d10aa9eb5c..1c6841fca04f6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -910,6 +910,13 @@ private PerKey( this.fn = fn; } + /** + * Returns the KeyedCombineFn used by this Combine operation. + */ + public KeyedCombineFn getFn() { + return fn; + } + @Override public PCollection> apply(PCollection> input) { return input From a806df86c59840f433da871ee4d96dcfc648ecb0 Mon Sep 17 00:00:00 2001 From: bchambers Date: Tue, 24 Feb 2015 11:26:52 -0800 Subject: [PATCH 0180/1541] Update the AvroCoder so that it can return true for isDeterministic. This assumes that AvroCoder uses directBinaryEncoder so Arrays and Maps will be encoded as (number of elements) followed by each elements. They are therefore deterministic if the underlying Array/Collecion/Map is ordered. This deals with several AvroAnnotations: @Stringable: Not deterministic because we know nothing about #toString() @AvroSchema: We aren't sure enough about this to be confident it will lead to deterministic encodings. @AvroEncode: Not deterministic because we know nothing about the encoder. @AvroName: Recognizes that this causes a different field name to be used. This doesn't address the case of GenericRecord or SpecificRecord from a given schema. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87075588 --- .../cloud/dataflow/sdk/coders/AvroCoder.java | 294 ++++++++++++- .../dataflow/sdk/coders/StandardCoder.java | 14 +- .../dataflow/sdk/coders/AvroCoderTest.java | 387 +++++++++++++++++- 3 files changed, 679 insertions(+), 16 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java index 75216120ac0fb..3200f54b8bd4c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java @@ -18,7 +18,9 @@ import static com.google.cloud.dataflow.sdk.util.Structs.addString; +import com.google.cloud.dataflow.sdk.transforms.GroupByKey; import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.common.reflect.TypeToken; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -27,21 +29,35 @@ import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.BinaryEncoder; import org.apache.avro.io.DatumReader; import org.apache.avro.io.DatumWriter; import org.apache.avro.io.DecoderFactory; import org.apache.avro.io.EncoderFactory; +import org.apache.avro.reflect.AvroEncode; +import org.apache.avro.reflect.AvroName; +import org.apache.avro.reflect.AvroSchema; import org.apache.avro.reflect.ReflectData; import org.apache.avro.reflect.ReflectDatumReader; import org.apache.avro.reflect.ReflectDatumWriter; +import org.apache.avro.reflect.Union; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; +import java.util.SortedSet; /** * An encoder using Avro binary format. @@ -78,6 +94,11 @@ * ... * } * + *

    + * The implementation attempts to determine if the Avro encoding of the given type will satisfy + * the criteria of {@link Coder#verifyDeterministic} by inspecting both the type and the + * Schema provided or generated by Avro. Only coders that are deterministic can be used in + * {@link GroupByKey} operations. * * @param the type of elements handled by this coder */ @@ -124,6 +145,9 @@ public static AvroCoder of( private final Class type; private final Schema schema; + + private final List nonDeterministicReasons; + private final DatumWriter writer; private final DatumReader reader; private final EncoderFactory encoderFactory = new EncoderFactory(); @@ -132,6 +156,13 @@ public static AvroCoder of( protected AvroCoder(Class type, Schema schema) { this.type = type; this.schema = schema; + + if (GenericRecord.class.isAssignableFrom(type)) { + nonDeterministicReasons = Arrays.asList( + "GenericRecord may have non-deterministic fields."); + } else { + nonDeterministicReasons = new AvroDeterminismChecker().check(TypeToken.of(type), schema); + } this.reader = createDatumReader(); this.writer = createDatumWriter(); } @@ -170,11 +201,26 @@ public CloudObject asCloudObject() { } /** - * Depends upon the structure being serialized. + * Returns true if the given type should be deterministically encoded using + * the given Schema, the directBinaryEncoder, and the ReflectDatumWriter or + * GenericDatumWriter. */ @Override + @Deprecated public boolean isDeterministic() { - return false; + return nonDeterministicReasons.isEmpty(); + } + + /** + * Raises an exception describing reasons why the type may not be deterministically + * encoded using the given Schema, the directBinaryEncoder, and the ReflectDatumWriter + * or GenericDatumWriter. + */ + @Override + public void verifyDeterministic() throws NonDeterministicException { + if (!nonDeterministicReasons.isEmpty()) { + throw new NonDeterministicException(this, nonDeterministicReasons); + } } /** @@ -228,4 +274,248 @@ private Object readResolve() { return new AvroCoder(type, parser.parse(schemaStr)); } } + + /** + * Helper class encapsulating the various pieces of state maintained by the + * recursive walk used for checking if the encoding will be deterministic. + */ + protected static class AvroDeterminismChecker { + + // Reasons that the original type are not deterministic. This accumulates + // the actual output. + private List reasons = new ArrayList<>(); + + // Types that are currently "open". Used to make sure we don't have any + // recursive types. Note that we assume that all occurrences of a given type + // are equal, rather than tracking pairs of type + schema. + private Set> activeTypes = new HashSet<>(); + + /** + * Report an error in the current context. + */ + private void reportError(String context, String fmt, Object... args) { + String message = String.format(fmt, args); + reasons.add(context + ": " + message); + } + + /** + * Classes that are serialized by Avro using their toString() are only deterministic + * if their associated #toString() method is deterministic. Rather than determine all + * of them, we conservatively list some classes that definitely are, and treat any + * others an non-deterministic. + */ + private static final Set> DETERMINISTIC_STRINGABLE_CLASSES = new HashSet<>(); + static { + DETERMINISTIC_STRINGABLE_CLASSES.add(java.math.BigDecimal.class); + DETERMINISTIC_STRINGABLE_CLASSES.add(java.math.BigInteger.class); + DETERMINISTIC_STRINGABLE_CLASSES.add(java.net.URI.class); + DETERMINISTIC_STRINGABLE_CLASSES.add(java.net.URL.class); + DETERMINISTIC_STRINGABLE_CLASSES.add(String.class); + } + + /** + * Return true if the given type token is a subtype of *any* of the listed parents. + */ + private static boolean isSubtypeOf(TypeToken type, Class... parents) { + for (Class parent : parents) { + if (TypeToken.of(parent).isAssignableFrom(type)) { + return true; + } + } + return false; + } + + protected AvroDeterminismChecker() {} + + // The entry point for the check. Should not be recursively called. + public List check(TypeToken type, Schema schema) { + recurse(type.getRawType().getName(), type, schema); + return reasons; + } + + // This is the method that should be recursively called. It sets up the path + // and visited types correctly. + private void recurse(String context, TypeToken type, Schema schema) { + if (type.getRawType().isAnnotationPresent(AvroSchema.class)) { + reportError(context, "Custom schemas are not supported -- remove @AvroSchema."); + return; + } + + if (!activeTypes.add(type)) { + reportError(context, "%s appears recursively", type); + return; + } + + doCheck(context, type, schema); + activeTypes.remove(type); + } + + private void doCheck(String context, TypeToken type, Schema schema) { + switch (schema.getType()) { + case ARRAY: + checkArray(context, type, schema); + break; + case ENUM: + // Enums should be deterministic, since they depend only on the ordinal. + break; + case FIXED: + // Depending on the implementation of GenericFixed, we don't know how + // the given field will be encoded. So, we assume that it isn't + // deterministic. + reportError(context, "FIXED encodings are not guaranteed to be deterministic"); + break; + case MAP: + checkMap(context, type, schema); + break; + case RECORD: + checkRecord(context, type, schema); + break; + case UNION: + checkUnion(context, type, schema); + break; + case STRING: + checkString(context, type); + break; + case BOOLEAN: + case BYTES: + case DOUBLE: + case INT: + case FLOAT: + case LONG: + case NULL: + // For types that Avro encodes using one of the above primitives, we assume they are + // deterministic. + break; + default: + // In any other case (eg., new types added to Avro) we cautiously return + // false. + reportError(context, "Unknown Avro Schema Type: %s", schema.getType()); + break; + } + } + + private void checkString(String context, TypeToken type) { + // For types that are encoded as strings, we need to make sure they're in an approved + // whitelist. For other types that are annotated @Stringable, Avro will just use the + // #toString() methods, which has no guarantees of determinism. + if (!DETERMINISTIC_STRINGABLE_CLASSES.contains(type.getRawType())) { + reportError(context, "%s may not have deterministic #toString()", type); + } + } + + private void checkUnion(String context, TypeToken type, Schema schema) { + if (!type.getRawType().isAnnotationPresent(Union.class)) { + reportError(context, "Expected type %s to have @Union annotation", type); + return; + } + + // Errors associated with this union will use the base class as their context. + String baseClassContext = type.getRawType().getName(); + + // For a union, we need to make sure that each possible instantiation is deterministic. + for (Schema concrete : schema.getTypes()) { + @SuppressWarnings("unchecked") + TypeToken unionType = TypeToken.of(ReflectData.get().getClass(concrete)); + + recurse(baseClassContext, unionType, concrete); + } + } + + private void checkRecord(String context, TypeToken type, Schema schema) { + // If the the record isn't a true class, but rather a GenericRecord, SpecificRecord, etc. + // with a specificified schema, then we need to make the decision based on the generated + // implementations. + if (isSubtypeOf(type, IndexedRecord.class)) { + // TODO: Update this once we support deterministic GenericRecord/SpecificRecords. + reportError(context, "IndexedRecords may be non-deterministic"); + return; + } + + // For a record, we want to make sure that all the fields are deterministic. + Class clazz = type.getRawType(); + for (org.apache.avro.Schema.Field fieldSchema : schema.getFields()) { + Field field = getField(clazz, fieldSchema.name()); + String fieldContext = field.getDeclaringClass().getName() + "#" + field.getName(); + + if (field.isAnnotationPresent(AvroEncode.class)) { + reportError(fieldContext, + "Custom encoders may be non-deterministic -- remove @AvroEncode"); + continue; + } + + if (field.isAnnotationPresent(AvroSchema.class)) { + reportError(fieldContext, "Custom schemas are not supported -- remove @AvroSchema"); + continue; + } + + TypeToken fieldType = type.resolveType(field.getGenericType()); + recurse(fieldContext, fieldType, fieldSchema.schema()); + } + } + + private void checkMap(String context, TypeToken type, Schema schema) { + if (!isSubtypeOf(type, SortedMap.class)) { + reportError(context, "%s may not be deterministically ordered", type); + } + + // Avro (currently) asserts that all keys are strings. + // In case that changes, we double check that the key was a string: + Class keyType = type.resolveType(Map.class.getTypeParameters()[0]).getRawType(); + if (!String.class.equals(keyType)) { + reportError(context, "map keys should be Strings, but was %s", keyType); + } + + recurse(context, + type.resolveType(Map.class.getTypeParameters()[1]), + schema.getValueType()); + } + + private void checkArray(String context, TypeToken type, Schema schema) { + TypeToken elementType = null; + if (type.isArray()) { + // The type is an array (with ordering)-> deterministic iff the element is deterministic. + elementType = type.getComponentType(); + } else if (isSubtypeOf(type, Collection.class)) { + if (isSubtypeOf(type, List.class, SortedSet.class)) { + // Ordered collection -> deterministic iff the element is deterministic + elementType = type.resolveType(Collection.class.getTypeParameters()[0]); + } else { + // Not an ordered collection -> not deterministic + reportError(context, "%s may not be deterministically ordered", type); + return; + } + } else { + // If it was an unknown type encoded as an array, be conservative and assume + // that we don't know anything about the order. + reportError(context, "encoding %s as an ARRAY was unexpected"); + return; + } + + // If we get here, it's either a deterministically-ordered Collection, or + // an array. Either way, the type is deterministic iff the element type is + // deterministic. + recurse(context, elementType, schema.getElementType()); + } + + /** + * Extract a field from a class. We need to look at the declared fields so that we can + * see private fields. We may need to walk up to the parent to get classes from the parent. + */ + private static Field getField(Class clazz, String name) { + while (clazz != null) { + for (Field field : clazz.getDeclaredFields()) { + AvroName avroName = field.getAnnotation(AvroName.class); + if (avroName != null && name.equals(avroName.value())) { + return field; + } else if (avroName == null && name.equals(field.getName())) { + return field; + } + } + clazz = clazz.getSuperclass(); + } + + throw new IllegalArgumentException( + "Unable to get field " + name + " from class " + clazz); + } + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java index b5838908497e5..2df352757a142 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java @@ -28,8 +28,6 @@ import java.util.Collections; import java.util.List; -import javax.annotation.Nullable; - /** * A StandardCoder is one that defines equality, hashing, and printing * via the class name and recursively using {@link #getComponents}. @@ -148,8 +146,7 @@ public void registerByteSizeObserver( @Override public void verifyDeterministic() throws NonDeterministicException { if (!isDeterministic()) { - throw new NonDeterministicException(this, - getClass().getSimpleName() + " reported it was not determinsitic."); + throw new NonDeterministicException(this, "Coder reported it was not determinsitic."); } } @@ -168,13 +165,4 @@ protected void verifyDeterministic(String message, Coder... coders) throws NonDeterministicException { verifyDeterministic(message, Arrays.asList(coders)); } - - protected void addReasons(String prefix, List accumulator, - @Nullable List newReasons) { - if (newReasons != null) { - for (String reason : newReasons) { - accumulator.add(prefix + reason); - } - } - } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java index 43023a1f63dd9..781c8a85766db 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java @@ -16,8 +16,14 @@ package com.google.cloud.dataflow.sdk.coders; +import static org.hamcrest.Matchers.containsString; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder.Context; +import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.Create; @@ -27,10 +33,19 @@ import com.google.cloud.dataflow.sdk.util.SerializableUtils; import com.google.cloud.dataflow.sdk.values.PCollection; +import org.apache.avro.AvroTypeException; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; +import org.apache.avro.reflect.AvroName; +import org.apache.avro.reflect.AvroSchema; +import org.apache.avro.reflect.ReflectData; +import org.apache.avro.reflect.Stringable; +import org.apache.avro.reflect.Union; +import org.hamcrest.Description; +import org.hamcrest.Matcher; import org.hamcrest.Matchers; +import org.hamcrest.TypeSafeMatcher; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -38,6 +53,17 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; /** Tests for {@link AvroCoder}. */ @RunWith(JUnit4.class) @@ -92,7 +118,7 @@ public String toString() { } } - static class GetTextFn extends DoFn { + private static class GetTextFn extends DoFn { @Override public void processElement(ProcessContext c) { c.output(c.element().text); @@ -188,4 +214,363 @@ public void testAvroCoderIsSerializable() throws Exception { // Check that the coder is serializable using the regular JSON approach. SerializableUtils.ensureSerializable(coder); } + + private final void assertDeterministic(Class clazz) { + try { + AvroCoder.of(clazz).verifyDeterministic(); + } catch (NonDeterministicException e) { + fail("Expected AvroCoder<" + clazz + "> to be deterministic."); + } + } + + private final void assertNonDeterministic(Class clazz, + Matcher reason1) { + try { + AvroCoder.of(clazz).verifyDeterministic(); + fail("Expected AvroCoder<" + clazz + "> to be non-deterministic."); + } catch (NonDeterministicException e) { + assertThat(e.getReasons(), Matchers.iterableWithSize(1)); + assertThat(e.getReasons(), Matchers.contains(reason1)); + } + } + + @Test + public void testDeterministicInteger() { + assertDeterministic(Integer.class); + } + + @Test + public void testDeterministicInt() { + assertDeterministic(int.class); + } + + private static class SimpleDeterministicClass { + @SuppressWarnings("unused") + private Integer intField; + @SuppressWarnings("unused") + private char charField; + @SuppressWarnings("unused") + private Integer[] intArray; + } + + @Test + public void testDeterministicSimple() { + assertDeterministic(SimpleDeterministicClass.class); + } + + private static class UnorderedMapClass { + @SuppressWarnings("unused") + private Map mapField; + } + + private Matcher reasonMatcher(final String prefix, final String messagePart) { + return new TypeSafeMatcher(String.class) { + @Override + public void describeTo(Description description) { + description.appendText(String.format("Reason starting with '%s' containing '%s'", + prefix, messagePart)); + } + + @Override + protected boolean matchesSafely(String item) { + return item.startsWith(prefix) && item.contains(messagePart); + } + }; + } + + private Matcher reasonClass(Class clazz, String message) { + return reasonMatcher(clazz.getName(), message); + } + + private Matcher reasonField( + Class clazz, String field, String message) { + return reasonMatcher(clazz.getName() + "#" + field, message); + } + + @Test + public void testDeterministicUnorderedMap() { + assertNonDeterministic(UnorderedMapClass.class, + reasonField(UnorderedMapClass.class, "mapField", + "java.util.Map " + + "may not be deterministically ordered")); + } + + private static class NonDeterministicArray { + @SuppressWarnings("unused") + private UnorderedMapClass[] arrayField; + } + @Test + public void testDeterministicNonDeterministicArray() { + assertNonDeterministic(NonDeterministicArray.class, + reasonField(UnorderedMapClass.class, "mapField", + "java.util.Map" + + " may not be deterministically ordered")); + } + + private static class SubclassOfUnorderedMapClass extends UnorderedMapClass {} + + + @Test + public void testDeterministicNonDeterministicChild() { + // Super class has non deterministic fields. + assertNonDeterministic( + SubclassOfUnorderedMapClass.class, + reasonField(UnorderedMapClass.class, "mapField", + "may not be deterministically ordered")); + } + + private static class SubclassHidingParent extends UnorderedMapClass { + @SuppressWarnings("unused") + @AvroName("mapField2") // AvroName is not enough + private int mapField; + } + + @Test + public void testAvroProhibitsShadowing() { + // This test verifies that Avro won't serialize a class with two fields of + // the same name. This is important for our error reporting, and also how + // we lookup a field. + try { + ReflectData.get().getSchema(SubclassHidingParent.class); + fail("Expected AvroTypeException"); + } catch (AvroTypeException e) { + assertThat(e.getMessage(), containsString("mapField")); + assertThat(e.getMessage(), containsString("two fields named")); + } + } + + private static class FieldWithAvroName { + @AvroName("name") + @SuppressWarnings("unused") + private int someField; + } + + @Test + public void testDeterministicWithAvroName() { + assertDeterministic(FieldWithAvroName.class); + } + + @Test + public void testDeterminismSortedMap() { + assertDeterministic(StringSortedMapField.class); + } + + private static class StringSortedMapField { + @SuppressWarnings("unused") + SortedMap sortedMapField; + } + + @Test + public void testDeterminismTreeMapValue() { + // The value is non-deterministic, so we should fail. + assertNonDeterministic(TreeMapNonDetValue.class, + reasonField(UnorderedMapClass.class, "mapField", + "java.util.Map " + + "may not be deterministically ordered")); + } + + private static class TreeMapNonDetValue { + @SuppressWarnings("unused") + TreeMap nonDeterministicField; + } + + @Test + public void testDeterminismUnorderedMap() { + // LinkedHashMap is not deterministically ordered, so we should fail. + assertNonDeterministic( + LinkedHashMapField.class, + reasonField(LinkedHashMapField.class, "nonDeterministicMap", + "java.util.LinkedHashMap " + + "may not be deterministically ordered")); + } + + private static class LinkedHashMapField { + @SuppressWarnings("unused") + LinkedHashMap nonDeterministicMap; + } + + @Test + public void testDeterminismCollection() { + assertNonDeterministic(StringCollection.class, + reasonField(StringCollection.class, "stringCollection", + "java.util.Collection may not be deterministically ordered")); + } + + private static class StringCollection { + @SuppressWarnings("unused") + Collection stringCollection; + } + + @Test + public void testDeterminismList() { + assertDeterministic(StringList.class); + assertDeterministic(StringArrayList.class); + } + + private static class StringList { + @SuppressWarnings("unused") + List stringCollection; + } + + private static class StringArrayList { + @SuppressWarnings("unused") + ArrayList stringCollection; + } + + @Test + public void testDeterminismSet() { + assertDeterministic(StringSortedSet.class); + assertDeterministic(StringTreeSet.class); + assertNonDeterministic(StringHashSet.class, + reasonField(StringHashSet.class, "stringCollection", + "java.util.HashSet may not be deterministically ordered")); + } + + private static class StringSortedSet{ + @SuppressWarnings("unused") + SortedSet stringCollection; + } + + private static class StringTreeSet { + @SuppressWarnings("unused") + TreeSet stringCollection; + } + + private static class StringHashSet { + @SuppressWarnings("unused") + HashSet stringCollection; + } + + @Test + public void testDeterminismCollectionValue() { + assertNonDeterministic(OrderedSetOfNonDetValues.class, + reasonField(UnorderedMapClass.class, "mapField", + "may not be deterministically ordered")); + assertNonDeterministic(ListOfNonDetValues.class, + reasonField(UnorderedMapClass.class, "mapField", + "may not be deterministically ordered")); + } + + private static class OrderedSetOfNonDetValues { + @SuppressWarnings("unused") + SortedSet set; + } + + private static class ListOfNonDetValues { + @SuppressWarnings("unused") + List set; + } + + @Test + public void testDeterminismUnion() { + assertDeterministic(DeterministicUnionBase.class); + assertNonDeterministic( + NonDeterministicUnionBase.class, + reasonField(UnionCase3.class, "mapField", "may not be deterministically ordered")); + } + + @Test + public void testDeterminismStringable() { + assertDeterministic(String.class); + assertNonDeterministic(StringableClass.class, + reasonClass(StringableClass.class, "may not have deterministic #toString()")); + } + + @Stringable + private static class StringableClass { + } + + @Test + public void testDeterminismCyclicClass() { + assertNonDeterministic(Cyclic.class, + reasonClass(Cyclic.class, "appears recursively")); + assertNonDeterministic(CyclicField.class, + reasonField(Cyclic.class, "cyclicField", + Cyclic.class.getName() + " appears recursively")); + assertNonDeterministic(IndirectCycle1.class, + reasonField(IndirectCycle2.class, "field2", + IndirectCycle1.class.getName() + " appears recursively")); + } + + private static class Cyclic { + @SuppressWarnings("unused") + int intField; + @SuppressWarnings("unused") + Cyclic cyclicField; + } + + private static class CyclicField { + @SuppressWarnings("unused") + Cyclic cyclicField2; + } + + private static class IndirectCycle1 { + @SuppressWarnings("unused") + IndirectCycle2 field1; + } + + private static class IndirectCycle2 { + @SuppressWarnings("unused") + IndirectCycle1 field2; + } + + @Test + public void testDeterminismHasCustomSchema() { + assertNonDeterministic(HasCustomSchema.class, + reasonClass(HasCustomSchema.class, "Custom schemas are not supported")); + } + + private static class HasCustomSchema { + @AvroSchema("{\"name\": \"bar\", \"type\": \"record\", \"fields\": [" + + "{\"name\": \"foo\", \"type\": \"int\"}]}") + @SuppressWarnings("unused") + GenericRecord genericRecord; + } + + @Test + public void testAvroCoderTreeMapDeterminism() + throws Exception, NonDeterministicException { + TreeMapField size1 = new TreeMapField(); + TreeMapField size2 = new TreeMapField(); + + // Different order for entries + size1.field.put("hello", "world"); + size1.field.put("another", "entry"); + + size2.field.put("another", "entry"); + size2.field.put("hello", "world"); + + AvroCoder coder = AvroCoder.of(TreeMapField.class); + coder.verifyDeterministic(); + + ByteArrayOutputStream outStream1 = new ByteArrayOutputStream(); + ByteArrayOutputStream outStream2 = new ByteArrayOutputStream(); + + Context context = Context.NESTED; + coder.encode(size1, outStream1, context); + coder.encode(size2, outStream2, context); + + assertTrue(Arrays.equals( + outStream1.toByteArray(), outStream2.toByteArray())); + } + + private static class TreeMapField { + private TreeMap field = new TreeMap<>(); + } + + @Union({ UnionCase1.class, UnionCase2.class }) + private abstract static class DeterministicUnionBase {} + + @Union({ UnionCase1.class, UnionCase2.class, UnionCase3.class }) + private abstract static class NonDeterministicUnionBase {} + + private static class UnionCase1 extends DeterministicUnionBase {} + private static class UnionCase2 extends DeterministicUnionBase { + @SuppressWarnings("unused") + String field; + } + private static class UnionCase3 extends NonDeterministicUnionBase { + @SuppressWarnings("unused") + private Map mapField; + } } From f247a0de03250af8c1d36c8e972b04f422c0dfa0 Mon Sep 17 00:00:00 2001 From: amyu Date: Tue, 24 Feb 2015 13:00:26 -0800 Subject: [PATCH 0181/1541] Streaming examples that use PubSub, from a 'traffic sensor' domain. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87083984 --- .../examples/TrafficStreamingMaxLaneFlow.java | 397 ++++++++++++++++++ .../examples/TrafficStreamingRoutes.java | 369 ++++++++++++++++ 2 files changed, 766 insertions(+) create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/TrafficStreamingMaxLaneFlow.java create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/TrafficStreamingRoutes.java diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/TrafficStreamingMaxLaneFlow.java b/examples/src/main/java/com/google/cloud/dataflow/examples/TrafficStreamingMaxLaneFlow.java new file mode 100644 index 0000000000000..77a392bf6e3f3 --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/TrafficStreamingMaxLaneFlow.java @@ -0,0 +1,397 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.DefaultCoder; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.PubsubIO; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.apache.avro.reflect.Nullable; +import org.joda.time.Duration; + +import java.util.ArrayList; +import java.util.List; + +/** + * A streaming Dataflow Example using BigQuery output, in the 'traffic sensor' domain. + * + *

    Concepts: The streaming runner, sliding windows, PubSub topic ingestion, use of the AvroCoder + * to encode a custom class, and custom Combine transforms. + * + *

    This pipeline takes as input traffic sensor data from a PubSub topic, and analyzes it using + * SlidingWindows. For each window, it finds the lane that had the highest flow recorded, for each + * sensor station. It writes those max values along with auxiliary info to a BigQuery table. + * + *

    This pipeline expects input from + * + * this script, + * which publishes traffic sensor data to a PubSub topic. After you've started this pipeline, start + * up the input generation script as per its instructions. The default SlidingWindow parameters + * assume that you're running this script with the {@literal --replay} flag, which simulates pauses + * in the sensor data publication. + * + *

    To run this example using the Dataflow service, you must provide an input + * PubSub topic and an output BigQuery table, using the {@literal --inputTopic}, + * {@literal --dataset}, and {@literal --table} options. Since this is a streaming + * pipeline that never completes, select the non-blocking pipeline runner by specifying + * {@literal --runner=DataflowPipelineRunner}. + * + *

    When you are done running the example, cancel your pipeline so that you do not continue to + * be charged for its instances. You can do this by visiting + * https://console.developers.google.com/project/your-project-name/dataflow/job-id + * in the Developers Console. You should also terminate the generator script so that you do not + * use unnecessary PubSub quota. + */ +public class TrafficStreamingMaxLaneFlow { + + static final int WINDOW_DURATION = 60; // Default sliding window duration in minutes + static final int WINDOW_SLIDE_EVERY = 5; // Default window 'slide every' setting in minutes + + /** + * This class holds information about each lane in a station reading, along with some general + * information from the reading. + */ + @DefaultCoder(AvroCoder.class) + static class LaneInfo { + @Nullable String stationId; + @Nullable String lane; + @Nullable String direction; + @Nullable String freeway; + @Nullable String recordedTimestamp; + @Nullable Integer laneFlow; + @Nullable Integer totalFlow; + @Nullable Double laneAO; + @Nullable Double laneAS; + + public LaneInfo() {} + + public LaneInfo(String stationId, String lane, String direction, String freeway, + String timestamp, Integer laneFlow, Double laneAO, + Double laneAS, Integer totalFlow) { + this.stationId = stationId; + this.lane = lane; + this.direction = direction; + this.freeway = freeway; + this.recordedTimestamp = timestamp; + this.laneFlow = laneFlow; + this.laneAO = laneAO; + this.laneAS = laneAS; + this.totalFlow = totalFlow; + } + + public String getStationId() { + return this.stationId; + } + public String getLane() { + return this.lane; + } + public String getDirection() { + return this.direction; + } + public String getFreeway() { + return this.freeway; + } + public String getRecordedTimestamp() { + return this.recordedTimestamp; + } + public Integer getLaneFlow() { + return this.laneFlow; + } + public Double getLaneAO() { + return this.laneAO; + } + public Double getLaneAS() { + return this.laneAS; + } + public Integer getTotalFlow() { + return this.totalFlow; + } + } + + /** + * Extract flow information for each of the 8 lanes in a reading, and output as separate tuples. + * This will let us determine which lane has the max flow for that station over the span of the + * window, and output not only the max flow from that calculcation, but other associated + * information. The number of lanes for which data is present depends upon which freeway the data + * point comes from. + */ + static class ExtractFlowInfoFn extends DoFn> { + @Override + public void processElement(ProcessContext c) { + String[] items = c.element().split(","); + // extract the sensor information for the lanes from the input string fields. + String timestamp = items[0]; + String stationId = items[1]; + String freeway = items[2]; + String direction = items[3]; + Integer totalFlow = tryIntParse(items[7]); + // lane 1 + Integer lane1Flow = tryIntParse(items[11]); + Double lane1AO = tryDoubleParse(items[12]); + Double lane1AS = tryDoubleParse(items[13]); + // lane2 + Integer lane2Flow = tryIntParse(items[16]); + Double lane2AO = tryDoubleParse(items[17]); + Double lane2AS = tryDoubleParse(items[18]); + // lane3 + Integer lane3Flow = tryIntParse(items[21]); + Double lane3AO = tryDoubleParse(items[22]); + Double lane3AS = tryDoubleParse(items[23]); + // lane4 + Integer lane4Flow = tryIntParse(items[26]); + Double lane4AO = tryDoubleParse(items[27]); + Double lane4AS = tryDoubleParse(items[28]); + // lane5 + Integer lane5Flow = tryIntParse(items[31]); + Double lane5AO = tryDoubleParse(items[32]); + Double lane5AS = tryDoubleParse(items[33]); + // lane6 + Integer lane6Flow = tryIntParse(items[36]); + Double lane6AO = tryDoubleParse(items[37]); + Double lane6AS = tryDoubleParse(items[38]); + // lane7 + Integer lane7Flow = tryIntParse(items[41]); + Double lane7AO = tryDoubleParse(items[42]); + Double lane7AS = tryDoubleParse(items[43]); + // lane8 + Integer lane8Flow = tryIntParse(items[46]); + Double lane8AO = tryDoubleParse(items[47]); + Double lane8AS = tryDoubleParse(items[48]); + + // For each lane in the reading, output LaneInfo keyed to its station. + LaneInfo laneInfo1 = new LaneInfo(stationId, "lane1", direction, freeway, timestamp, + lane1Flow, lane1AO, lane1AS, totalFlow); + c.output(KV.of(stationId, laneInfo1)); + LaneInfo laneInfo2 = new LaneInfo(stationId, "lane2", direction, freeway, timestamp, + lane2Flow, lane2AO, lane2AS, totalFlow); + c.output(KV.of(stationId, laneInfo2)); + LaneInfo laneInfo3 = new LaneInfo(stationId, "lane3", direction, freeway, timestamp, + lane3Flow, lane3AO, lane3AS, totalFlow); + c.output(KV.of(stationId, laneInfo3)); + LaneInfo laneInfo4 = new LaneInfo(stationId, "lane4", direction, freeway, timestamp, + lane4Flow, lane4AO, lane4AS, totalFlow); + c.output(KV.of(stationId, laneInfo4)); + LaneInfo laneInfo5 = new LaneInfo(stationId, "lane5", direction, freeway, timestamp, + lane5Flow, lane5AO, lane5AS, totalFlow); + c.output(KV.of(stationId, laneInfo5)); + LaneInfo laneInfo6 = new LaneInfo(stationId, "lane6", direction, freeway, timestamp, + lane6Flow, lane6AO, lane6AS, totalFlow); + c.output(KV.of(stationId, laneInfo6)); + LaneInfo laneInfo7 = new LaneInfo(stationId, "lane7", direction, freeway, timestamp, + lane7Flow, lane7AO, lane7AS, totalFlow); + c.output(KV.of(stationId, laneInfo7)); + LaneInfo laneInfo8 = new LaneInfo(stationId, "lane8", direction, freeway, timestamp, + lane8Flow, lane8AO, lane8AS, totalFlow); + c.output(KV.of(stationId, laneInfo8)); + } + } + + /** + * A custom 'combine function' used with the Combine.perKey transform. Used to find the max lane + * flow over all the data points in the Window. Extracts the lane flow from the input string and + * determines whether it's the max seen so far. We're using a custom combiner instead of the Max + * transform because we want to retain the additional information we've associated with the flow + * value. + */ + public static class MaxFlow implements SerializableFunction, LaneInfo> { + @Override + public LaneInfo apply(Iterable input) { + Integer max = 0; + LaneInfo maxInfo = new LaneInfo(); + for (LaneInfo item : input) { + Integer flow = item.getLaneFlow(); + if (flow != null && (flow >= max)) { + max = flow; + maxInfo = item; + } + } + return maxInfo; + } + } + + /** + * Format the results of the Max Lane flow calculation to a TableRow, to save to BigQuery. + * Add the timestamp from the window context. + */ + static class FormatMaxesFn extends DoFn, TableRow> { + @Override + public void processElement(ProcessContext c) { + + LaneInfo laneInfo = (LaneInfo) c.element().getValue(); + TableRow row = new TableRow() + .set("station_id", c.element().getKey()) + .set("direction", laneInfo.getDirection()) + .set("freeway", laneInfo.getFreeway()) + .set("lane_max_flow", laneInfo.getLaneFlow()) + .set("lane", laneInfo.getLane()) + .set("avg_occ", laneInfo.getLaneAO()) + .set("avg_speed", laneInfo.getLaneAS()) + .set("total_flow", laneInfo.getTotalFlow()) + .set("recorded_timestamp", laneInfo.getRecordedTimestamp()) + .set("window_timestamp", c.timestamp().toString()); + c.output(row); + } + + /** Defines the BigQuery schema used for the output. */ + static TableSchema getSchema() { + List fields = new ArrayList<>(); + fields.add(new TableFieldSchema().setName("station_id").setType("STRING")); + fields.add(new TableFieldSchema().setName("direction").setType("STRING")); + fields.add(new TableFieldSchema().setName("freeway").setType("STRING")); + fields.add(new TableFieldSchema().setName("lane_max_flow").setType("INTEGER")); + fields.add(new TableFieldSchema().setName("lane").setType("STRING")); + fields.add(new TableFieldSchema().setName("avg_occ").setType("FLOAT")); + fields.add(new TableFieldSchema().setName("avg_speed").setType("FLOAT")); + fields.add(new TableFieldSchema().setName("total_flow").setType("INTEGER")); + fields.add(new TableFieldSchema().setName("window_timestamp").setType("TIMESTAMP")); + fields.add(new TableFieldSchema().setName("recorded_timestamp").setType("STRING")); + TableSchema schema = new TableSchema().setFields(fields); + return schema; + } + } + + /** + * This PTransform extracts lane info, calculates the max lane flow found for a given station (for + * the current Window) using a custom 'combiner', and formats the results for BigQuery. + */ + static class MaxLaneFlow + extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection rows) { + // row... => ... + PCollection> flowInfo = rows.apply( + ParDo.of(new ExtractFlowInfoFn())); + + // stationId, LaneInfo => stationId + max lane flow info + PCollection> flowMaxes = + flowInfo.apply(Combine.perKey( + new MaxFlow())); + + // ... => row... + PCollection results = flowMaxes.apply( + ParDo.of(new FormatMaxesFn())); + + return results; + } + } + + /** + * Options supported by {@link TrafficStreamingMaxLaneFlow}. + *

    + * Inherits standard configuration options. + */ + private interface TrafficStreamingMaxLaneFlowOptions extends PipelineOptions { + @Description("Input PubSub topic") + @Validation.Required + String getInputTopic(); + void setInputTopic(String value); + + @Description("BigQuery dataset name") + @Validation.Required + String getDataset(); + void setDataset(String value); + + @Description("BigQuery table name") + @Validation.Required + String getTable(); + void setTable(String value); + + @Description("Numeric value of sliding window duration, in minutes") + @Default.Integer(WINDOW_DURATION) + Integer getWindowDuration(); + void setWindowDuration(Integer value); + + @Description("Numeric value of window 'slide every' setting, in minutes") + @Default.Integer(WINDOW_SLIDE_EVERY) + Integer getWindowSlideEvery(); + void setWindowSlideEvery(Integer value); + } + + /** + * Sets up and starts streaming pipeline. + */ + public static void main(String[] args) { + TrafficStreamingMaxLaneFlowOptions options = PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(TrafficStreamingMaxLaneFlowOptions.class); + DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); + dataflowOptions.setStreaming(true); + + Pipeline pipeline = Pipeline.create(options); + TableReference tableRef = new TableReference(); + tableRef.setProjectId(dataflowOptions.getProject()); + tableRef.setDatasetId(options.getDataset()); + tableRef.setTableId(options.getTable()); + pipeline + .apply(PubsubIO.Read.topic(options.getInputTopic())) + /* map the incoming data stream into sliding windows. The default window duration values + work well if you're running the accompanying PubSub generator script with the + --replay flag, which simulates pauses in the sensor data publication. You may want to + adjust them otherwise. */ + .apply(Window.into(SlidingWindows.of( + Duration.standardMinutes(options.getWindowDuration())). + every(Duration.standardMinutes(options.getWindowSlideEvery())))) + .apply(new MaxLaneFlow()) + .apply(BigQueryIO.Write.to(tableRef) + .withSchema(FormatMaxesFn.getSchema())); + + /* When you are done running the example, cancel your pipeline so that you do not continue to + be charged for its instances. You can do this by visiting + https://console.developers.google.com/project/your-project-name/dataflow/job-id + in the Developers Console. You should also terminate the generator script so that you do not + use unnecessary PubSub quota. */ + pipeline.run(); + } + + private static Integer tryIntParse(String number) { + try { + return Integer.parseInt(number); + } catch (NumberFormatException e) { + return null; + } + } + + private static Double tryDoubleParse(String number) { + try { + return Double.parseDouble(number); + } catch (NumberFormatException e) { + return null; + } + } +} + diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/TrafficStreamingRoutes.java b/examples/src/main/java/com/google/cloud/dataflow/examples/TrafficStreamingRoutes.java new file mode 100644 index 0000000000000..df065167b6bf4 --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/TrafficStreamingRoutes.java @@ -0,0 +1,369 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; +import com.google.cloud.dataflow.sdk.coders.DefaultCoder; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.PubsubIO; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.GroupByKey; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.base.MoreObjects; + +import org.apache.avro.reflect.Nullable; +import org.joda.time.Duration; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Hashtable; +import java.util.List; +import java.util.Map; + + +/** + * A streaming Dataflow Example using BigQuery output, in the 'traffic sensor' domain. + * + *

    Concepts: The streaming runner, GroupByKey, keyed state, sliding windows, and + * PubSub topic ingestion. + * + *

    This pipeline takes as input traffic sensor data from a PubSub topic, and analyzes it using + * SlidingWindows. For each window, it calculates the average speed over the window for some small + * set of predefined 'routes', and looks for 'slowdowns' in those routes. It uses keyed state to + * track slowdown information across successive sliding windows. It writes its results to a + * BigQuery table. + * + *

    This pipeline expects input from + * + * this script, + * which publishes traffic sensor data to a PubSub topic. After you've started this pipeline, start + * up the input generation script as per its instructions. The default SlidingWindow parameters + * assume that you're running this script without the {@literal --replay} flag, so that there are + * no simulated pauses in the sensor data publication. + * + *

    To run this example using the Dataflow service, you must provide an input + * PubSub topic and an output BigQuery table, using the {@literal --inputTopic}, + * {@literal --dataset}, and {@literal --table} options. Since this is a streaming + * pipeline that never completes, select the non-blocking pipeline runner by specifying + * {@literal --runner=DataflowPipelineRunner}. + * + *

    When you are done running the example, cancel your pipeline so that you do not continue to + * be charged for its instances. You can do this by visiting + * https://console.developers.google.com/project/your-project-name/dataflow/job-id + * in the Developers Console. You should also terminate the generator script so that you do not + * use unnecessary PubSub quota. + */ +public class TrafficStreamingRoutes { + // Instantiate some small predefined San Diego routes to analyze + static Map sdStations = buildStationInfo(); + static final int WINDOW_DURATION = 3; // Default sliding window duration in minutes + static final int WINDOW_SLIDE_EVERY = 1; // Default window 'slide every' setting in minutes + + /** + * This class holds information about a station reading's average speed. + */ + @DefaultCoder(AvroCoder.class) + static class StationSpeed { + @Nullable String stationId; + @Nullable Double avgSpeed; + + public StationSpeed() {} + + public StationSpeed(String stationId, Double avgSpeed) { + this.stationId = stationId; + this.avgSpeed = avgSpeed; + } + + public String getStationId() { + return this.stationId; + } + public Double getAvgSpeed() { + return this.avgSpeed; + } + } + + /** + * This class holds information about a route's speed/slowdown. + */ + @DefaultCoder(AvroCoder.class) + static class RouteInfo { + @Nullable String route; + @Nullable Double avgSpeed; + @Nullable Boolean slowdownEvent; + + + public RouteInfo() {} + + public RouteInfo(String route, Double avgSpeed, Boolean slowdownEvent) { + this.route = route; + this.avgSpeed = avgSpeed; + this.slowdownEvent = slowdownEvent; + } + + public String getRoute() { + return this.route; + } + public Double getAvgSpeed() { + return this.avgSpeed; + } + public Boolean getSlowdownEvent() { + return this.slowdownEvent; + } + } + + /** + * Filter out readings for the stations along predefined 'routes', and output + * (station, speed info) keyed on route. + */ + static class ExtractStationSpeedFn extends DoFn> { + @Override + public void processElement(ProcessContext c) { + String[] items = c.element().split(","); + String stationId = items[1]; + String stationType = items[4]; + Double avgSpeed = tryDoubleParse(items[9]); + // For this analysis, use only 'main line' station types + if (stationType.equals("ML")) { + // For this simple example, filter out everything but some hardwired routes. + if (sdStations.containsKey(stationId)) { + StationSpeed stationSpeed = new StationSpeed(stationId, avgSpeed); + // The tuple key is the 'route' name stored in the 'sdStations' hash. + c.output(KV.of(sdStations.get(stationId), stationSpeed)); + } + } + } + } + + /* + * For a given route, track average speed for the window. Calculate whether traffic is currently + * slowing down, via a predefined threshold. Use keyed state to keep a count of the speed drops, + * with at least 3 in a row constituting a 'slowdown'. + * Note: these calculations are for example purposes only, and are unrealistic and oversimplified. + */ + static class GatherStats extends DoFn>, KV> + implements DoFn.RequiresKeyedState { + + static final int SLOWDOWN_THRESH = 67; + static final int SLOWDOWN_COUNT_CAP = 3; + + @Override + public void processElement(ProcessContext c) throws IOException { + String route = c.element().getKey(); + CodedTupleTag tag = CodedTupleTag.of(route, BigEndianIntegerCoder.of()); + // For the given key (a route), get the keyed state. + Integer slowdownCount = MoreObjects.firstNonNull(c.keyedState().lookup(tag), 0); + Double speedSum = 0.0; + Integer scount = 0; + Iterable infoList = c.element().getValue(); + // For all stations in the route, sum (non-null) speeds. Keep a count of the non-null speeds. + for (StationSpeed item : infoList) { + Double speed = item.getAvgSpeed(); + if (speed != null) { + speedSum += speed; + scount++; + } + } + // calculate average speed. + if (scount == 0) { + return; + } + Double speedAvg = speedSum / scount; + Boolean slowdownEvent = false; + if (speedAvg != null) { + // see if the speed falls below defined threshold. If it does, increment the count of + // slow readings, as retrieved from the keyed state, up to the defined cap. + if (speedAvg < SLOWDOWN_THRESH) { + if (slowdownCount < SLOWDOWN_COUNT_CAP) { + slowdownCount++; + } + } else if (slowdownCount > 0) { + // if speed is not below threshold, then decrement the count of slow readings. + slowdownCount--; + } + // if our count of slowdowns has reached its cap, we consider this a 'slowdown event' + if (slowdownCount >= SLOWDOWN_COUNT_CAP) { + slowdownEvent = true; + } + } + // store the new slowdownCount in the keyed state for the route key. + c.keyedState().store(tag, slowdownCount); + RouteInfo routeInfo = new RouteInfo(route, speedAvg, slowdownEvent); + c.output(KV.of(route, routeInfo)); + } + } + + /** + * Format the results of the slowdown calculations to a TableRow, to save to BigQuery. + */ + static class FormatStatsFn extends DoFn, TableRow> { + @Override + public void processElement(ProcessContext c) { + RouteInfo routeInfo = c.element().getValue(); + TableRow row = new TableRow() + .set("avg_speed", routeInfo.getAvgSpeed()) + .set("slowdown_event", routeInfo.getSlowdownEvent()) + .set("route", c.element().getKey()) + .set("window_timestamp", c.timestamp().toString()); + c.output(row); + } + + /** Defines the BigQuery schema used for the output. */ + static TableSchema getSchema() { + List fields = new ArrayList<>(); + fields.add(new TableFieldSchema().setName("route").setType("STRING")); + fields.add(new TableFieldSchema().setName("avg_speed").setType("FLOAT")); + fields.add(new TableFieldSchema().setName("slowdown_event").setType("BOOLEAN")); + fields.add(new TableFieldSchema().setName("window_timestamp").setType("TIMESTAMP")); + TableSchema schema = new TableSchema().setFields(fields); + return schema; + } + } + + /** + * This PTransform extracts speed info from traffic station readings. + * It groups the readings by 'route' and analyzes traffic slowdown for that route, using keyed + * state to retain previous slowdown information. Then, it formats the results for BigQuery. + */ + static class TrackSpeed extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection rows) { + // row... => ... + PCollection> flowInfo = rows.apply( + ParDo.of(new ExtractStationSpeedFn())); + + // Apply a GroupByKey transform to collect a list of all station + // readings for a given route. + PCollection>> timeGroup = flowInfo.apply( + GroupByKey.create()); + + // Analyze 'slowdown' over the route readings. + PCollection> stats = timeGroup.apply(ParDo.of(new GatherStats())); + + // Format the results for writing to BigQuery + PCollection results = stats.apply( + ParDo.of(new FormatStatsFn())); + + return results; + } + } + + + /** + * Options supported by {@link TrafficStreamingRoutes}. + *

    + * Inherits standard configuration options. + */ + private interface TrafficStreamingRoutesOptions extends PipelineOptions { + @Description("Input PubSub topic") + @Validation.Required + String getInputTopic(); + void setInputTopic(String value); + + @Description("BigQuery dataset name") + @Validation.Required + String getDataset(); + void setDataset(String value); + + @Description("BigQuery table name") + @Validation.Required + String getTable(); + void setTable(String value); + + @Description("Numeric value of sliding window duration, in minutes") + @Default.Integer(WINDOW_DURATION) + Integer getWindowDuration(); + void setWindowDuration(Integer value); + + @Description("Numeric value of window 'slide every' setting, in minutes") + @Default.Integer(WINDOW_SLIDE_EVERY) + Integer getWindowSlideEvery(); + void setWindowSlideEvery(Integer value); + } + + /** + * Sets up and starts streaming pipeline. + */ + public static void main(String[] args) { + TrafficStreamingRoutesOptions options = PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(TrafficStreamingRoutesOptions.class); + DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); + dataflowOptions.setStreaming(true); + + Pipeline pipeline = Pipeline.create(options); + TableReference tableRef = new TableReference(); + tableRef.setProjectId(dataflowOptions.getProject()); + tableRef.setDatasetId(options.getDataset()); + tableRef.setTableId(options.getTable()); + pipeline + .apply(PubsubIO.Read.topic(options.getInputTopic())) + /* map the incoming data stream into sliding windows. + The default window duration values work well if you're running the accompanying PubSub + generator script without the --replay flag, so that there are no simulated pauses in + the sensor data publication. You may want to adjust the values otherwise. */ + .apply(Window.into(SlidingWindows.of( + Duration.standardMinutes(options.getWindowDuration())). + every(Duration.standardMinutes(options.getWindowSlideEvery())))) + .apply(new TrackSpeed()) + .apply(BigQueryIO.Write.to(tableRef) + .withSchema(FormatStatsFn.getSchema())); + + /* When you are done running the example, cancel your pipeline so that you do not continue to + be charged for its instances. You can do this by visiting + https://console.developers.google.com/project/your-project-name/dataflow/job-id + in the Developers Console. You should also terminate the generator script so that you do not + use unnecessary PubSub quota. */ + pipeline.run(); + } + + private static Double tryDoubleParse(String number) { + try { + return Double.parseDouble(number); + } catch (NumberFormatException e) { + return null; + } + } + + /** Define some small hard-wired San Diego 'routes' to track based on sensor station ID. */ + private static Map buildStationInfo() { + Map stations = new Hashtable(); + stations.put("1108413", "SDRoute1"); // from freeway 805 S + stations.put("1108699", "SDRoute2"); // from freeway 78 E + stations.put("1108702", "SDRoute2"); + return stations; + } + +} + From dd3985db3bb763d1b44350a49689292d73958ab4 Mon Sep 17 00:00:00 2001 From: robertwb Date: Tue, 24 Feb 2015 15:21:01 -0800 Subject: [PATCH 0182/1541] Hashtag auto-completion streaming Dataflow example. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87098231 --- .../cloud/dataflow/examples/AutoComplete.java | 465 ++++++++++++++++++ .../dataflow/examples/AutoCompleteTest.java | 181 +++++++ .../dataflow/sdk/transforms/Combine.java | 187 +++++++ .../cloud/dataflow/sdk/transforms/Top.java | 3 +- .../dataflow/sdk/transforms/CombineTest.java | 42 ++ 5 files changed, 877 insertions(+), 1 deletion(-) create mode 100644 examples/src/main/java/com/google/cloud/dataflow/examples/AutoComplete.java create mode 100644 examples/src/test/java/com/google/cloud/dataflow/examples/AutoCompleteTest.java diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/AutoComplete.java b/examples/src/main/java/com/google/cloud/dataflow/examples/AutoComplete.java new file mode 100644 index 0000000000000..70d638c0ca1ab --- /dev/null +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/AutoComplete.java @@ -0,0 +1,465 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.api.services.datastore.DatastoreV1.Entity; +import com.google.api.services.datastore.DatastoreV1.Key; +import com.google.api.services.datastore.DatastoreV1.Value; +import com.google.api.services.datastore.client.DatastoreHelper; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.DefaultCoder; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.DatastoreIO; +import com.google.cloud.dataflow.sdk.io.PubsubIO; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Filter; +import com.google.cloud.dataflow.sdk.transforms.Flatten; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Partition; +import com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.Top; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PBegin; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; + +import org.joda.time.Duration; + +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * An example that computes the most popular hash tags for a for every prefix, + * which can be used for auto-completion. + * + *

    Concepts: Using the same pipeline in both streaming and batch, combiners, + * composite transforms. + * + *

    To execute this pipeline using the Dataflow service in batch mode, + * specify pipeline configuration: + * --project= + * --stagingLocation=gs:// + * --runner=[Blocking]DataflowPipelineRunner + * --inputFile=gs://path/to/input*.txt + * [--outputDataset=] + * + *

    To execute this pipeline using the Dataflow service in streaming mode, + * specify pipeline configuration: + * --project= + * --stagingLocation=gs:// + * --runner=DataflowPipelineRunner + * --inputTopic=/topics/someproject/sometopic + * [--outputDataset=] + * + *

    Which will update the datastore every 10 seconds based on the last 30 minutes + * of data received. + */ +public class AutoComplete { + + /** + * A PTransform that takes as input a list of tokens and returns + * the most common tokens per prefix. + */ + public static class ComputeTopCompletions + extends PTransform, PCollection>>> { + private final int candidatesPerPrefix; + private final boolean recursive; + + protected ComputeTopCompletions(int candidatesPerPrefix, boolean recursive) { + this.candidatesPerPrefix = candidatesPerPrefix; + this.recursive = recursive; + } + + public static ComputeTopCompletions top(int candidatesPerPrefix, boolean recursive) { + return new ComputeTopCompletions(candidatesPerPrefix, recursive); + } + + @Override + public PCollection>> apply(PCollection input) { + PCollection candidates = input + // First count how often each token appears. + .apply(new Count.PerElement()) + + // Map the KV outputs of Count into our own CompletionCandiate class. + .apply(ParDo.of( + new DoFn, CompletionCandidate>() { + @Override + public void processElement(ProcessContext c) { + c.output(new CompletionCandidate(c.element().getKey(), c.element().getValue())); + } + })); + + // Compute the top via either a flat or recursive algorithm. + if (recursive) { + return candidates + .apply(new ComputeTopRecursive(candidatesPerPrefix, 1)) + .apply(Flatten.>>pCollections()); + } else { + return candidates + .apply(new ComputeTopFlat(candidatesPerPrefix, 1)); + } + } + } + + /** + * Lower latency, but more expensive. + */ + private static class ComputeTopFlat + extends PTransform, + PCollection>>> { + + private final int candidatesPerPrefix; + private final int minPrefix; + + public ComputeTopFlat(int candidatesPerPrefix, int minPrefix) { + this.candidatesPerPrefix = candidatesPerPrefix; + this.minPrefix = minPrefix; + } + + @Override + public PCollection>> apply( + PCollection input) { + return input + // For each completion candidate, map it to all prefixes. + .apply(ParDo.of(new AllPrefixes(minPrefix))) + + // Find and return the top candiates for each prefix. + .apply(Top.largestPerKey(candidatesPerPrefix) + .withHotKeys(new HotKeySpread())); + } + + private static class HotKeySpread implements SerializableFunction { + @Override + public Integer apply(String input) { + return (int) Math.pow(4, 5 - input.length()); + } + } + } + + /** + * Cheaper but higher latency. + * + *

    Returns two PCollections, the first is top prefixes of size greater + * than minPrefix, and the second is top prefixes of size exactly + * minPrefix. + */ + private static class ComputeTopRecursive + extends PTransform, + PCollectionList>>> { + + private final int candidatesPerPrefix; + private final int minPrefix; + + public ComputeTopRecursive(int candidatesPerPrefix, int minPrefix) { + this.candidatesPerPrefix = candidatesPerPrefix; + this.minPrefix = minPrefix; + } + + private class KeySizePartitionFn implements PartitionFn>> { + public int partitionFor(KV> elem, int numPartitions) { + return elem.getKey().length() > minPrefix ? 0 : 1; + } + } + + private static class FlattenTops + extends DoFn>, CompletionCandidate> { + public void processElement(ProcessContext c) { + for (CompletionCandidate cc : c.element().getValue()) { + c.output(cc); + } + } + } + + public PCollectionList>> apply( + PCollection input) { + if (minPrefix > 10) { + // Base case, partitioning to return the output in the expected format. + return input + .apply(new ComputeTopFlat(candidatesPerPrefix, minPrefix)) + .apply(Partition.of(2, new KeySizePartitionFn())); + } else { + // If a candidate is in the top N for prefix a...b, it must also be in the top + // N for a...bX for every X, which is typlically a much smaller set to consider. + // First, compute the top candidate for prefixes of size at least minPrefix + 1. + PCollectionList>> larger = input + .apply(new ComputeTopRecursive(candidatesPerPrefix, minPrefix + 1)); + // Consider the top candidates for each prefix of length minPrefix + 1... + PCollection>> small = + PCollectionList + .of(larger.get(1).apply(ParDo.of(new FlattenTops()))) + // ...together with those (previously excluded) candidates of length + // exactly minPrefix... + .and(input.apply(Filter.by(new SerializableFunction() { + public Boolean apply(CompletionCandidate c) { + return c.getValue().length() == minPrefix; + } + }))) + .apply(Flatten.pCollections()) + // ...set the key to be the minPrefix-length prefix... + .apply(ParDo.of(new AllPrefixes(minPrefix, minPrefix))) + // ...and (re)apply the Top operator to all of them together. + .apply(Top.largestPerKey(candidatesPerPrefix)); + return PCollectionList + .of(larger.apply(Flatten.>>pCollections())) + .and(small); + } + } + } + + /** + * A DoFn that keys each candidate by all its prefixes. + */ + private static class AllPrefixes + extends DoFn> { + private final int minPrefix; + private final int maxPrefix; + public AllPrefixes(int minPrefix) { + this(minPrefix, Integer.MAX_VALUE); + } + public AllPrefixes(int minPrefix, int maxPrefix) { + this.minPrefix = minPrefix; + this.maxPrefix = maxPrefix; + } + @Override + public void processElement(ProcessContext c) { + String word = c.element().value; + for (int i = minPrefix; i <= Math.min(word.length(), maxPrefix); i++) { + c.output(KV.of(word.substring(0, i), c.element())); + } + } + } + + /** + * Class used to store tag-count pairs. + */ + @DefaultCoder(AvroCoder.class) + static class CompletionCandidate implements Comparable { + private long count; + private String value; + + public CompletionCandidate(String value, long count) { + this.value = value; + this.count = count; + } + + public long getCount() { + return count; + } + + public String getValue() { + return value; + } + + // Empty constructor required for Avro decoding. + @SuppressWarnings("unused") + public CompletionCandidate() {} + + @Override + public int compareTo(CompletionCandidate o) { + if (this.count < o.count) { + return -1; + } else if (this.count == o.count) { + return this.value.compareTo(o.value); + } else { + return 1; + } + } + + @Override + public boolean equals(Object other) { + if (other instanceof CompletionCandidate) { + CompletionCandidate that = (CompletionCandidate) other; + return this.count == that.count && this.value.equals(that.value); + } else { + return false; + } + } + + @Override + public String toString() { + return "CompletionCandidate[" + value + ", " + count + "]"; + } + } + + /** + * Takes as input a set of strings, and emits each #hashtag found therein. + */ + static class ExtractHashtags extends DoFn { + public void processElement(ProcessContext c) { + Matcher m = Pattern.compile("#\\S+").matcher(c.element()); + while (m.find()) { + c.output(m.group().substring(1)); + } + } + } + + static class FormatForBigquery extends DoFn>, TableRow> { + public void processElement(ProcessContext c) { + List completions = new ArrayList<>(); + for (CompletionCandidate cc : c.element().getValue()) { + completions.add(new TableRow() + .set("count", cc.getCount()) + .set("tag", cc.getValue())); + } + TableRow row = new TableRow() + .set("prefix", c.element().getKey()) + .set("tags", completions); + c.output(row); + } + } + + /** + * Takes as input a the top candidates per prefix, and emits an entity + * suitable for writing to Datastore. + */ + static class FormatForDatastore extends DoFn>, Entity> { + private String kind; + + public FormatForDatastore(String kind) { + this.kind = kind; + } + + public void processElement(ProcessContext c) { + Entity.Builder entityBuilder = Entity.newBuilder(); + // Create entities with same ancestor Key.??? + Key ancestorKey = DatastoreHelper.makeKey(kind, "root").build(); + Key key = DatastoreHelper.makeKey(ancestorKey, c.element().getKey()).build(); + + entityBuilder.setKey(key); + List candidates = new ArrayList<>(); + for (CompletionCandidate tag : c.element().getValue()) { + Entity.Builder tagEntity = Entity.newBuilder(); + tagEntity.addProperty( + DatastoreHelper.makeProperty("tag", DatastoreHelper.makeValue(tag.value))); + tagEntity.addProperty( + DatastoreHelper.makeProperty("count", DatastoreHelper.makeValue(tag.count))); + candidates.add(DatastoreHelper.makeValue(tagEntity).build()); + } + entityBuilder.addProperty( + DatastoreHelper.makeProperty("candidates", DatastoreHelper.makeValue(candidates))); + c.output(entityBuilder.build()); + } + } + + /** + * Options supported by this class. + * + *

    Inherits standard Dataflow configuration options. + */ + private static interface Options extends PipelineOptions { + @Description("Input text file") + String getInputFile(); + void setInputFile(String value); + + @Description("Input Pubsub topic") + String getInputTopic(); + void setInputTopic(String value); + + @Description("Whether to use the recursive algorithm") + @Default.Boolean(true) + Boolean getRecursive(); + void setRecursive(Boolean value); + + @Description("BigQuery table to write to, specified as " + + ":.. The dataset must already exist.") + String getOutputBigqueryTable(); + void setOutputBigqueryTable(String value); + + @Description("Dataset entity kind") + @Default.String("autocomplete-demo") + String getKind(); + void setKind(String value); + + @Description("Dataset ID to write to in datastore") + String getOutputDataset(); + void setOutputDataset(String value); + } + + public static void main(String[] args) { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + + // We support running the same pipeline in either + // batch or windowed streaming mode. + PTransform> readSource; + WindowFn windowFn; + if (options.getInputFile() != null) { + readSource = TextIO.Read.from(options.getInputFile()); + windowFn = new GlobalWindows(); + } else { + DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); + dataflowOptions.setStreaming(true); + readSource = PubsubIO.Read.topic(options.getInputTopic()); + windowFn = SlidingWindows.of(Duration.standardMinutes(30)).every(Duration.standardSeconds(5)); + } + + // Create the pipeline. + Pipeline p = Pipeline.create(options); + PCollection>> toWrite = p + .apply(readSource) + .apply(ParDo.of(new ExtractHashtags())) + .apply(Window.into(windowFn)) + .apply(ComputeTopCompletions.top(10, options.getRecursive())); + + // Optionally write the result out to bigquery... + if (options.getOutputBigqueryTable() != null) { + List tagFields = new ArrayList<>(); + tagFields.add(new TableFieldSchema().setName("count").setType("INTEGER")); + tagFields.add(new TableFieldSchema().setName("tag").setType("STRING")); + List fields = new ArrayList<>(); + fields.add(new TableFieldSchema().setName("prefix").setType("STRING")); + fields.add(new TableFieldSchema().setName("tags").setType("RECORD").setFields(tagFields)); + TableSchema schema = new TableSchema().setFields(fields); + + toWrite + .apply(ParDo.of(new FormatForBigquery())) + .apply(BigQueryIO.Write + .to(options.getOutputBigqueryTable()) + .withSchema(schema) + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE)); + } + + // ...and to Datastore. + if (options.getOutputDataset() != null) { + toWrite + .apply(ParDo.of(new FormatForDatastore(options.getKind()))) + .apply(DatastoreIO.write().to(options.getOutputDataset())); + } + + // Run the pipeline. + p.run(); + } +} diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/AutoCompleteTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/AutoCompleteTest.java new file mode 100644 index 0000000000000..7f72095ed911c --- /dev/null +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/AutoCompleteTest.java @@ -0,0 +1,181 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples; + +import com.google.cloud.dataflow.examples.AutoComplete.CompletionCandidate; +import com.google.cloud.dataflow.examples.AutoComplete.ComputeTopCompletions; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.DoFn.ProcessContext; +import com.google.cloud.dataflow.sdk.transforms.Filter; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TimestampedValue; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +/** + * Tests of AutoComplete. + */ +@RunWith(Parameterized.class) +public class AutoCompleteTest implements Serializable { + + private boolean recursive; + + public AutoCompleteTest(Boolean recursive) { + this.recursive = recursive; + } + + @Parameterized.Parameters + public static Collection primeNumbers() { + return Arrays.asList(new Object[][] { + { true }, + { false } + }); + } + + @Test + public void testAutoComplete() { + List words = Arrays.asList( + "apple", + "apple", + "apricot", + "banana", + "blackberry", + "blackberry", + "blackberry", + "blueberry", + "blueberry", + "cherry"); + + Pipeline p = TestPipeline.create(); + + PCollection input = p.apply(Create.of(words)); + + PCollection>> output = + input.apply(new ComputeTopCompletions(2, recursive)) + .apply(Filter.by( + new SerializableFunction>, Boolean>() { + public Boolean apply(KV> element) { + return element.getKey().length() <= 2; + } + })); + + DataflowAssert.that(output).containsInAnyOrder( + KV.of("a", parseList("apple:2", "apricot:1")), + KV.of("ap", parseList("apple:2", "apricot:1")), + KV.of("b", parseList("blackberry:3", "blueberry:2")), + KV.of("ba", parseList("banana:1")), + KV.of("bl", parseList("blackberry:3", "blueberry:2")), + KV.of("c", parseList("cherry:1")), + KV.of("ch", parseList("cherry:1"))); + p.run(); + } + + @Test + public void testTinyAutoComplete() { + List words = Arrays.asList("x", "x", "x", "xy", "xy", "xyz"); + + Pipeline p = TestPipeline.create(); + + PCollection input = p.apply(Create.of(words)); + + PCollection>> output = + input.apply(new ComputeTopCompletions(2, recursive)); + + DataflowAssert.that(output).containsInAnyOrder( + KV.of("x", parseList("x:3", "xy:2")), + KV.of("xy", parseList("xy:2", "xyz:1")), + KV.of("xyz", parseList("xyz:1"))); + p.run(); + } + + @Test + public void testWindowedAutoComplete() { + List> words = Arrays.asList( + TimestampedValue.of("xA", new Instant(1)), + TimestampedValue.of("xA", new Instant(1)), + TimestampedValue.of("xB", new Instant(1)), + TimestampedValue.of("xB", new Instant(2)), + TimestampedValue.of("xB", new Instant(2))); + + Pipeline p = TestPipeline.create(); + + PCollection input = p + .apply(Create.of(words)) + .apply(new ReifyTimestamps()); + + PCollection>> output = + input.apply(Window.into(SlidingWindows.of(new Duration(2)))) + .apply(new ComputeTopCompletions(2, recursive)); + + DataflowAssert.that(output).containsInAnyOrder( + // Window [0, 2) + KV.of("x", parseList("xA:2", "xB:1")), + KV.of("xA", parseList("xA:2")), + KV.of("xB", parseList("xB:1")), + + // Window [1, 3) + KV.of("x", parseList("xB:3", "xA:2")), + KV.of("xA", parseList("xA:2")), + KV.of("xB", parseList("xB:3")), + + // Window [2, 3) + KV.of("x", parseList("xB:2")), + KV.of("xB", parseList("xB:2"))); + p.run(); + } + + private static List parseList(String... entries) { + List all = new ArrayList<>(); + for (String s : entries) { + String[] countValue = s.split(":"); + all.add(new CompletionCandidate(countValue[0], Integer.valueOf(countValue[1]))); + } + return all; + } + + private static class ReifyTimestamps + extends PTransform>, PCollection> { + public PCollection apply(PCollection> input) { + return input.apply(ParDo.of(new DoFn, T>() { + @Override + public void processElement(ProcessContext c) { + c.outputWithTimestamp(c.element().getValue(), c.element().getTimestamp()); + } + })); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java index 1c6841fca04f6..e078bd7ad0e8d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -20,11 +20,17 @@ import com.google.cloud.dataflow.sdk.coders.CoderRegistry; import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.VarIntCoder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; +import com.google.cloud.dataflow.sdk.values.PCollectionTuple; import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TupleTagList; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -910,6 +916,39 @@ private PerKey( this.fn = fn; } + /** + * If a single key has disproportionately many values, it may become a + * bottleneck, especially in streaming mode. This returns a new per-key + * combining transform that inserts an intermediate node to combine "hot" + * keys partially before performing the full combine. + * + * @param hotKeySpread a function from keys to an integer N, where the key + * will be spread among N intermediate nodes for partial combining. + * If N is less than or equal to 1, this key will not be sent through an + * intermediate node. + */ + public PerKeyWithHotKeys withHotKeys( + SerializableFunction hotKeySpread) { + return new PerKeyWithHotKeys(fn, hotKeySpread).withName(name); + } + + /** + * Like {@link #withHotKeys(SerializableFunction)}, but returning the given + * constant value for every key. + */ + public PerKeyWithHotKeys withHotKeys(final int hotKeySpread) { + return withHotKeys( + new SerializableFunction(){ + @Override public Integer apply(K unused) { return hotKeySpread; } + }); + } + + @Override + @SuppressWarnings("unchecked") + public PerKey withName(String name) { + return (PerKey) super.withName(name); + } + /** * Returns the KeyedCombineFn used by this Combine operation. */ @@ -930,6 +969,154 @@ protected String getKindString() { } } + /** + * Like {@link PerKey}, but sharding the combining of hot keys. + */ + public static class PerKeyWithHotKeys + extends PTransform>, PCollection>> { + + private final transient KeyedCombineFn fn; + private final SerializableFunction hotKeySpread; + + private PerKeyWithHotKeys( + KeyedCombineFn fn, + SerializableFunction hotKeySpread) { + this.fn = fn; + this.hotKeySpread = hotKeySpread; + } + + @Override + @SuppressWarnings("unchecked") + public PerKeyWithHotKeys withName(String name) { + return (PerKeyWithHotKeys) super.withName(name); + } + + @Override + public PCollection> apply(PCollection> input) { + return applyHelper(input); + } + + private PCollection> applyHelper(PCollection> input) { + // Name the accumulator type. + @SuppressWarnings("unchecked") + final KeyedCombineFn fn = (KeyedCombineFn) this.fn; + + // A CombineFn's mergeAccumulator can be applied in a tree-like fashon. + // Here we shard the key using an integer nonce, combine on that partial + // set of values, then drop the nonce and do a final combine of the + // aggregates. We do this by splitting the original CombineFn into two, + // on that does addInput + merge and another that does merge + extract. + KeyedCombineFn, VI, VA, VA> hotPreCombine = + new KeyedCombineFn, VI, VA, VA>() { + @Override + public VA createAccumulator(KV key) { + return fn.createAccumulator(key.getKey()); + } + @Override + public void addInput(KV key, VA accumulator, VI value) { + fn.addInput(key.getKey(), accumulator, value); + } + @Override + public VA mergeAccumulators(KV key, Iterable accumulators) { + return fn.mergeAccumulators(key.getKey(), accumulators); + } + @Override + public VA extractOutput(KV key, VA accumulator) { + return accumulator; + } + @Override + @SuppressWarnings("unchecked") + public Coder getAccumulatorCoder( + CoderRegistry registry, Coder> keyCoder, Coder inputCoder) { + return fn.getAccumulatorCoder( + registry, ((KvCoder) keyCoder).getKeyCoder(), inputCoder); + } + }; + + @SuppressWarnings("unchecked") + final KvCoder inputCoder = ((KvCoder) input.getCoder()); + // List required because the accumulator must be mutable. + KeyedCombineFn, VO> hotPostCombine = + new KeyedCombineFn, VO>() { + @Override + public List createAccumulator(K key) { + return new ArrayList<>(); + } + @Override + public void addInput(K key, List accumulator, VA value) { + VA merged = fn.mergeAccumulators( + key, Iterables.concat(accumulator, ImmutableList.of(value))); + accumulator.clear(); + accumulator.add(merged); + } + @Override + public List mergeAccumulators(K key, Iterable> accumulators) { + List singleton = new ArrayList<>(); + singleton.add(fn.mergeAccumulators(key, Iterables.concat(accumulators))); + return singleton; + } + @Override + public VO extractOutput(K key, List accumulator) { + return fn.extractOutput(key, fn.mergeAccumulators(key, accumulator)); + } + @Override + public Coder getDefaultOutputCoder( + CoderRegistry registry, Coder keyCoder, Coder accumulatorCoder) { + return fn.getDefaultOutputCoder(registry, keyCoder, inputCoder.getValueCoder()); + } + }; + + // Use the provided hotKeySpread fn to split into "hot" and "cold" keys, + // augmenting the hot keys with a nonce. + final TupleTag, VI>> hot = new TupleTag<>(); + final TupleTag> cold = new TupleTag<>(); + PCollectionTuple split = input.apply( + ParDo.of(new DoFn, KV>(){ + int counter = 0; + @Override + public void processElement(ProcessContext c) { + KV kv = c.element(); + int spread = hotKeySpread.apply(kv.getKey()); + if (spread <= 1) { + c.output(kv); + } else { + int nonce = counter++ % spread; + c.sideOutput(hot, KV.of(KV.of(kv.getKey(), nonce), kv.getValue())); + } + } + }) + .withOutputTags(cold, TupleTagList.of(hot))); + + // Combine the hot and cold keys separately. + PCollection> combinedHot = split + .get(hot) + .setCoder(KvCoder.of(KvCoder.of(inputCoder.getKeyCoder(), VarIntCoder.of()), + inputCoder.getValueCoder())) + .apply(Combine.perKey(hotPreCombine)) + .apply(ParDo.of( + new DoFn, VA>, KV>() { + @Override + public void processElement(ProcessContext c) { + c.output(KV.of(c.element().getKey().getKey(), c.element().getValue())); + } + })) + .apply(Combine.perKey(hotPostCombine)); + PCollection> combinedCold = split + .get(cold) + .setCoder(inputCoder) + .apply(Combine.perKey(fn)); + + // Return the union of the hot and cold key results. + return PCollectionList.of(combinedHot).and(combinedCold) + .apply(Flatten.>pCollections()); + } + + @Override + protected String getKindString() { + return "Combine.PerKey"; + } + } + ///////////////////////////////////////////////////////////////////////////// diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java index bc0270f01bce7..cc6ee9d7b889b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.coders.CustomCoder; import com.google.cloud.dataflow.sdk.coders.ListCoder; import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn; +import com.google.cloud.dataflow.sdk.transforms.Combine.PerKey; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -303,7 +304,7 @@ PTransform, PCollection>> largest(int count) { * which take a {@code PCollection} and return the top elements. */ public static > - PTransform>, PCollection>>> + PerKey> largestPerKey(int count) { return Combine.perKey( new TopCombineFn<>(count, new Largest()).asKeyedFn()) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java index 284ae327eabcf..2dec13aa10afb 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java @@ -36,6 +36,7 @@ import com.google.cloud.dataflow.sdk.runners.RecordingPipelineVisitor; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; @@ -312,6 +313,36 @@ counter.new Counter(8, 2, 0, 0), counter.new Counter(1, 1, 0, 0))); } + private static final SerializableFunction hotKeySpread = + new SerializableFunction() { + @Override + public Integer apply(String input) { + return input.equals("a") ? 3 : 0; + } + }; + + @Test + public void testHotKeyCombining() { + Pipeline p = TestPipeline.create(); + PCollection> input = copy(createInput(p, TABLE), 10); + + KeyedCombineFn mean = + new MeanInts().asKeyedFn(); + PCollection> coldMean = input.apply( + Combine.perKey(mean).withHotKeys(0)); + PCollection> warmMean = input.apply( + Combine.perKey(mean).withHotKeys(hotKeySpread)); + PCollection> hotMean = input.apply( + Combine.perKey(mean).withHotKeys(5)); + + List> expected = Arrays.asList(KV.of("a", 2.0), KV.of("b", 7.0)); + DataflowAssert.that(coldMean).containsInAnyOrder(expected); + DataflowAssert.that(warmMean).containsInAnyOrder(expected); + DataflowAssert.that(hotMean).containsInAnyOrder(expected); + + p.run(); + } + //////////////////////////////////////////////////////////////////////////// // Test classes, for different kinds of combining fns. @@ -588,4 +619,15 @@ public Coder getAccumulatorCoder( return SerializableCoder.of(Counter.class); } } + + private static PCollection copy(PCollection pc, final int n) { + return pc.apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception { + for (int i = 0; i < n; i++) { + c.output(c.element()); + } + } + })); + } } From 7c83f0dfd40e24fb2b8739b3f874b8fc2c44b649 Mon Sep 17 00:00:00 2001 From: vanya Date: Tue, 24 Feb 2015 15:30:36 -0800 Subject: [PATCH 0183/1541] Throw an exception when we get an unexpected End of Stream marker from HttpClient. This fixes potential data loss when reading from GCS and the connection gets prematurely terminated. Unfortunately it's impossible to make it work for compressed streams at this level, since we can't get a proper count of transferred bytes. Root cause of the problem: JDK ignores incomplete HTTP responses when Content-Length is specified. Chunked encoded streams work fine and unexpected termination is being detected. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87099191 --- .../gcsio/GoogleCloudStorageReadChannel.java | 25 ++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java index 61b931561c370..13c3dc427eaab 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java @@ -78,6 +78,7 @@ public class GoogleCloudStorageReadChannel implements SeekableByteChannel { // Size of the object being read. private long size = -1; + private boolean isCompressedStream; // Maximum number of automatic retries when reading from the underlying channel without making // progress; each time at least one byte is successfully read, the counter of attempted retries @@ -256,6 +257,12 @@ public int read(ByteBuffer buffer) int numBytesRead = readChannel.read(buffer); Preconditions.checkState(numBytesRead != 0, "Read 0 bytes without blocking!"); if (numBytesRead < 0) { + // Check that we didn't get a premature End of Stream signal by checking the number of + // bytes read against the stream size. Unfortunately we don't have information about the + // actual size of the data stream when stream compression is used, so we can only ignore + // this case here. + Preconditions.checkState(isCompressedStream || currentPosition == size, + "Received end of stream result before all the file data has been received"); break; } totalBytesRead += numBytesRead; @@ -347,7 +354,17 @@ public int read(ByteBuffer buffer) // If this method was called when the stream was already at EOF // (indicated by totalBytesRead == 0) then return EOF else, // return the number of bytes read. - return (totalBytesRead == 0) ? -1 : totalBytesRead; + boolean isEndOfStream = (totalBytesRead == 0); + if (isEndOfStream) { + // Check that we didn't get a premature End of Stream signal by checking the number of bytes + // read against the stream size. Unfortunately we don't have information about the actual size + // of the data stream when stream compression is used, so we can only ignore this case here. + Preconditions.checkState(isCompressedStream || currentPosition == size, + "Failed to read any data before all the file data has been received"); + return -1; + } else { + return totalBytesRead; + } } @Override @@ -531,6 +548,12 @@ protected InputStream openStreamAndSetSize(long newPosition) } } + // If the content is compressed, content length reported in the header is counting the number of + // compressed bytes. That means that we cannot rely on the reported content length to check that + // we have received all the data from the data stream. + String contentEncoding = response.getContentEncoding(); + isCompressedStream = (contentEncoding != null && contentEncoding.contains("gzip")); + String contentRange = response.getHeaders().getContentRange(); if (response.getHeaders().getContentLength() != null) { size = response.getHeaders().getContentLength() + newPosition; From 2203745152c6c7387abbb1897fa3f4f42f4738f1 Mon Sep 17 00:00:00 2001 From: sgmc Date: Tue, 24 Feb 2015 15:55:20 -0800 Subject: [PATCH 0184/1541] Add AvroIO validation to SDK. This enables the service to check permissions on the inputs and outputs for AvroIO when a Dataflow is submitted. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87101426 --- .../google/cloud/dataflow/sdk/io/AvroIO.java | 97 ++++++++++++++++--- .../runners/dataflow/AvroIOTranslator.java | 2 + .../cloud/dataflow/sdk/io/AvroIOTest.java | 16 +++ 3 files changed, 99 insertions(+), 16 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java index 2eaf60343e5f6..d7c2f1ef47da3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java @@ -171,6 +171,18 @@ public static Bound withSchema(String schema) { return withSchema((new Schema.Parser()).parse(schema)); } + /** + * Returns a AvroIO.Read PTransform that has GCS path validation on + * pipeline creation disabled. + * + *

    This can be useful in the case where the GCS input location does + * not exist at the pipeline creation time, but is expected to be available + * at execution time. + */ + public static Bound withoutValidation() { + return new Bound<>(GenericRecord.class).withoutValidation(); + } + /** * A PTransform that reads from an Avro file (or multiple Avro * files matching a pattern) and returns a bounded PCollection containing @@ -190,16 +202,19 @@ public static class Bound extends PTransform> { /** The schema of the input file. */ @Nullable final Schema schema; + /** An option to indicate if input validation is desired. Default is true. */ + final boolean validate; Bound(Class type) { - this(null, null, type, null); + this(null, null, type, null, true); } - Bound(String name, String filepattern, Class type, Schema schema) { + Bound(String name, String filepattern, Class type, Schema schema, boolean validate) { super(name); this.filepattern = filepattern; this.type = type; this.schema = schema; + this.validate = validate; } /** @@ -207,7 +222,7 @@ public static class Bound extends PTransform> { * with the given step name. Does not modify this object. */ public Bound named(String name) { - return new Bound<>(name, filepattern, type, schema); + return new Bound<>(name, filepattern, type, schema, validate); } /** @@ -217,7 +232,7 @@ public Bound named(String name) { * filepatterns.) Does not modify this object. */ public Bound from(String filepattern) { - return new Bound<>(name, filepattern, type, schema); + return new Bound<>(name, filepattern, type, schema, validate); } /** @@ -229,7 +244,7 @@ public Bound from(String filepattern) { * the resulting PCollection */ public Bound withSchema(Class type) { - return new Bound<>(name, filepattern, type, ReflectData.get().getSchema(type)); + return new Bound<>(name, filepattern, type, ReflectData.get().getSchema(type), validate); } /** @@ -238,7 +253,7 @@ public Bound withSchema(Class type) { * Does not modify this object. */ public Bound withSchema(Schema schema) { - return new Bound<>(name, filepattern, GenericRecord.class, schema); + return new Bound<>(name, filepattern, GenericRecord.class, schema, validate); } /** @@ -250,6 +265,19 @@ public Bound withSchema(String schema) { return withSchema((new Schema.Parser()).parse(schema)); } + /** + * Returns a new TextIO.Read PTransform that's like this one but + * that has GCS input path validation on pipeline creation disabled. + * Does not modify this object. + * + *

    This can be useful in the case where the GCS input location does + * not exist at the pipeline creation time, but is expected to be + * available at execution time. + */ + public Bound withoutValidation() { + return new Bound<>(name, filepattern, type, schema, false); + } + @Override public PCollection apply(PInput input) { if (filepattern == null) { @@ -285,6 +313,10 @@ public Schema getSchema() { return schema; } + public boolean needsValidation() { + return validate; + } + static { DirectPipelineRunner.registerDefaultTransformEvaluator( Bound.class, new DirectPipelineRunner.TransformEvaluator() { @@ -398,6 +430,18 @@ public static Bound withSchema(String schema) { return withSchema((new Schema.Parser()).parse(schema)); } + /** + * Returns a AvroIO.Write PTransform that has GCS path validation on + * pipeline creation disabled. + * + *

    This can be useful in the case where the GCS output location does + * not exist at the pipeline creation time, but is expected to be available + * at execution time. + */ + public static Bound withoutValidation() { + return new Bound<>(GenericRecord.class).withoutValidation(); + } + /** * A PTransform that writes a bounded PCollection to an Avro file (or * multiple Avro files matching a sharding pattern). @@ -421,13 +465,15 @@ public static class Bound extends PTransform, PDone> { /** The schema of the output file. */ @Nullable final Schema schema; + /** An option to indicate if output validation is desired. Default is true. */ + final boolean validate; Bound(Class type) { - this(null, null, "", 0, ShardNameTemplate.INDEX_OF_MAX, type, null); + this(null, null, "", 0, ShardNameTemplate.INDEX_OF_MAX, type, null, true); } Bound(String name, String filenamePrefix, String filenameSuffix, int numShards, - String shardTemplate, Class type, Schema schema) { + String shardTemplate, Class type, Schema schema, boolean validate) { super(name); this.filenamePrefix = filenamePrefix; this.filenameSuffix = filenameSuffix; @@ -435,6 +481,7 @@ public static class Bound extends PTransform, PDone> { this.shardTemplate = shardTemplate; this.type = type; this.schema = schema; + this.validate = validate; } /** @@ -443,7 +490,7 @@ public static class Bound extends PTransform, PDone> { */ public Bound named(String name) { return new Bound<>( - name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema); + name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema, validate); } /** @@ -457,7 +504,7 @@ public Bound named(String name) { public Bound to(String filenamePrefix) { validateOutputComponent(filenamePrefix); return new Bound<>( - name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema); + name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema, validate); } /** @@ -471,7 +518,7 @@ public Bound to(String filenamePrefix) { public Bound withSuffix(String filenameSuffix) { validateOutputComponent(filenameSuffix); return new Bound<>( - name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema); + name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema, validate); } /** @@ -491,7 +538,7 @@ public Bound withSuffix(String filenameSuffix) { public Bound withNumShards(int numShards) { Preconditions.checkArgument(numShards >= 0); return new Bound<>( - name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema); + name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema, validate); } /** @@ -504,7 +551,7 @@ public Bound withNumShards(int numShards) { */ public Bound withShardNameTemplate(String shardTemplate) { return new Bound<>( - name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema); + name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema, validate); } /** @@ -517,7 +564,7 @@ public Bound withShardNameTemplate(String shardTemplate) { *

    Does not modify this object. */ public Bound withoutSharding() { - return new Bound<>(name, filenamePrefix, filenameSuffix, 1, "", type, schema); + return new Bound<>(name, filenamePrefix, filenameSuffix, 1, "", type, schema, validate); } /** @@ -529,7 +576,7 @@ public Bound withoutSharding() { */ public Bound withSchema(Class type) { return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, - ReflectData.get().getSchema(type)); + ReflectData.get().getSchema(type), validate); } /** @@ -539,7 +586,7 @@ public Bound withSchema(Class type) { */ public Bound withSchema(Schema schema) { return new Bound<>(name, filenamePrefix, filenameSuffix, numShards, shardTemplate, - GenericRecord.class, schema); + GenericRecord.class, schema, validate); } /** @@ -551,6 +598,20 @@ public Bound withSchema(String schema) { return withSchema((new Schema.Parser()).parse(schema)); } + /** + * Returns a new TextIO.Write PTransform that's like this one but + * that has GCS output path validation on pipeline creation disabled. + * Does not modify this object. + * + *

    This can be useful in the case where the GCS output location does + * not exist at the pipeline creation time, but is expected to be + * available at execution time. + */ + public Bound withoutValidation() { + return new Bound<>( + name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema, false); + } + @Override public PDone apply(PCollection input) { if (filenamePrefix == null) { @@ -605,6 +666,10 @@ public Schema getSchema() { return schema; } + public boolean needsValidation() { + return validate; + } + static { DirectPipelineRunner.registerDefaultTransformEvaluator( Bound.class, new DirectPipelineRunner.TransformEvaluator() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java index e23afad8357b9..d4c2355d4e402 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java @@ -57,6 +57,7 @@ private void translateReadHelper( context.addInput(PropertyNames.FORMAT, "avro"); context.addInput(PropertyNames.FILEPATTERN, filepattern); context.addValueOnlyOutput(PropertyNames.OUTPUT, transform.getOutput()); + context.addInput(PropertyNames.VALIDATE_SOURCE, transform.needsValidation()); // TODO: Orderedness? } } @@ -102,6 +103,7 @@ private void translateWriteHelper( context.addInput(PropertyNames.FILENAME_PREFIX, filenamePrefix); context.addInput(PropertyNames.SHARD_NAME_TEMPLATE, transform.getShardTemplate()); context.addInput(PropertyNames.FILENAME_SUFFIX, transform.getFilenameSuffix()); + context.addInput(PropertyNames.VALIDATE_SINK, transform.needsValidation()); long numShards = transform.getNumShards(); if (numShards > 0) { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java index 1d45f4ab9386c..e211eb5941f83 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java @@ -18,7 +18,9 @@ import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.coders.Coder; @@ -361,6 +363,20 @@ public void testWriteFromSchemaString() throws Exception { "HerWrite"); } + @Test + public void testReadWithoutValidationFlag() throws Exception { + AvroIO.Read.Bound read = AvroIO.Read.from("gs://bucket/foo*/baz"); + assertTrue(read.needsValidation()); + assertFalse(read.withoutValidation().needsValidation()); + } + + @Test + public void testWriteWithoutValidationFlag() throws Exception { + AvroIO.Write.Bound write = AvroIO.Write.to("gs://bucket/foo/baz"); + assertTrue(write.needsValidation()); + assertFalse(write.withoutValidation().needsValidation()); + } + // TODO: for Write only, test withSuffix, withNumShards, // withShardNameTemplate and withoutSharding. } From a3b6e1d8b9595325e64f991206ca2ef88c277b74 Mon Sep 17 00:00:00 2001 From: lcwik Date: Wed, 25 Feb 2015 09:26:35 -0800 Subject: [PATCH 0185/1541] Make the full set of PipelineOptions specified by the user during construction of their Dataflow available on the worker. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87162112 --- sdk/pom.xml | 2 +- .../options/DataflowPipelineDebugOptions.java | 4 + .../sdk/options/PipelineOptionsFactory.java | 32 +++-- .../runners/DataflowPipelineTranslator.java | 10 ++ .../worker/StreamingDataflowWorker.java | 113 ++++++++---------- .../options/PipelineOptionsFactoryTest.java | 6 +- .../DataflowPipelineTranslatorTest.java | 28 +++++ .../DataflowWorkProgressUpdaterTest.java | 2 +- 8 files changed, 119 insertions(+), 78 deletions(-) diff --git a/sdk/pom.xml b/sdk/pom.xml index b3537af67fd96..abe71557a5c96 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -189,7 +189,7 @@ com.google.apis google-api-services-dataflow - v1beta3-rev5-1.19.1 + v1beta3-rev7-1.19.1 diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java index ce536e69a93d9..0ec0bbf044375 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java @@ -22,6 +22,8 @@ import com.google.cloud.dataflow.sdk.util.PathValidator; import com.google.cloud.dataflow.sdk.util.Stager; +import com.fasterxml.jackson.annotation.JsonIgnore; + import java.util.List; /** @@ -82,6 +84,7 @@ public interface DataflowPipelineDebugOptions extends PipelineOptions { /** * The validator class used to validate path names. */ + @JsonIgnore @Description("The validator class used to validate path names.") @Default.InstanceFactory(PathValidatorFactory.class) PathValidator getPathValidator(); @@ -98,6 +101,7 @@ public interface DataflowPipelineDebugOptions extends PipelineOptions { /** * The stager instance used to stage files. */ + @JsonIgnore @Description("The class use to stage packages.") @Default.InstanceFactory(StagerFactory.class) Stager getStager(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java index 24f8e15069108..3885b61de09a0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java @@ -50,6 +50,7 @@ import java.beans.IntrospectionException; import java.beans.Introspector; import java.beans.PropertyDescriptor; +import java.io.IOException; import java.lang.reflect.Method; import java.lang.reflect.Modifier; import java.lang.reflect.Proxy; @@ -495,9 +496,28 @@ static List getPropertyDescriptors( * {@link DataflowWorkerHarness}. */ @Deprecated - public static DataflowWorkerHarnessOptions createFromSystemProperties() { - DataflowWorkerHarnessOptions options = as(DataflowWorkerHarnessOptions.class); - options.setRunner(null); + public static DataflowWorkerHarnessOptions createFromSystemProperties() throws IOException { + ObjectMapper objectMapper = new ObjectMapper(); + DataflowWorkerHarnessOptions options; + if (System.getProperties().containsKey("sdk_pipeline_options")) { + String serializedOptions = System.getProperty("sdk_pipeline_options"); + LOG.info("Worker harness starting with: " + serializedOptions); + options = objectMapper.readValue(serializedOptions, PipelineOptions.class) + .as(DataflowWorkerHarnessOptions.class); + } else { + options = PipelineOptionsFactory.as(DataflowWorkerHarnessOptions.class); + } + + // These values will not be known at job submission time and must be provided. + if (System.getProperties().containsKey("worker_id")) { + options.setWorkerId(System.getProperty("worker_id")); + } + if (System.getProperties().containsKey("job_id")) { + options.setJobId(System.getProperty("job_id")); + } + + // TODO: Remove setting these options once we have migrated to passing + // through the pipeline options. if (System.getProperties().containsKey("root_url")) { options.setApiRootUrl(System.getProperty("root_url")); } @@ -513,15 +533,9 @@ public static DataflowWorkerHarnessOptions createFromSystemProperties() { if (System.getProperties().containsKey("service_account_keyfile")) { options.setServiceAccountKeyfile(System.getProperty("service_account_keyfile")); } - if (System.getProperties().containsKey("worker_id")) { - options.setWorkerId(System.getProperty("worker_id")); - } if (System.getProperties().containsKey("project_id")) { options.setProject(System.getProperty("project_id")); } - if (System.getProperties().containsKey("job_id")) { - options.setJobId(System.getProperty("job_id")); - } if (System.getProperties().containsKey("path_validator_class")) { try { options.setPathValidatorClass((Class) Class.forName( diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index b072f64fd576b..1a3d27498c892 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -89,6 +89,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -108,6 +109,7 @@ public class DataflowPipelineTranslator { // Must be kept in sync with their internal counterparts. public static final String HARNESS_WORKER_POOL = "harness"; private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineTranslator.class); + private static final ObjectMapper MAPPER = new ObjectMapper(); /** * A map from PTransform class to the corresponding @@ -340,6 +342,14 @@ public Job translate(List packages) { Environment environment = new Environment(); job.setEnvironment(environment); + try { + environment.setSdkPipelineOptions( + MAPPER.readValue(MAPPER.writeValueAsBytes(options), Map.class)); + } catch (IOException e) { + throw new IllegalArgumentException( + "PipelineOptions specified failed to serialize to JSON.", e); + } + WorkerPool workerPool = new WorkerPool(); workerPool.setKind(HARNESS_WORKER_POOL); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java index 0f75fb9cbb42a..25eaa4aa1b100 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -112,8 +112,15 @@ public static void main(String[] args) throws Exception { (WindmillServerStub) Class.forName(WINDMILL_SERVER_CLASS_NAME) .getDeclaredConstructor(String.class).newInstance(hostport); + DataflowWorkerHarnessOptions options = + PipelineOptionsFactory.createFromSystemProperties(); + // TODO: Remove setting these options once we have migrated to passing + // through the pipeline options. + options.setAppName("StreamingWorkerHarness"); + options.setStreaming(true); + StreamingDataflowWorker worker = - new StreamingDataflowWorker(mapTasks, windmillServer); + new StreamingDataflowWorker(mapTasks, windmillServer, options); worker.start(); worker.runStatusServer(statusPort); @@ -134,43 +141,50 @@ public static void main(String[] args) throws Exception { private Server statusServer; private AtomicReference lastException; - /** Regular constructor. */ public StreamingDataflowWorker( - List mapTasks, WindmillServerStub server) { - options = PipelineOptionsFactory.createFromSystemProperties(); - options.setAppName("StreamingWorkerHarness"); - options.setStreaming(true); - - initialize(mapTasks, server); - - if (System.getProperties().containsKey("path_validator_class")) { - try { - options.setPathValidatorClass((Class) Class.forName( - System.getProperty("path_validator_class"))); - } catch (ClassNotFoundException e) { - throw new RuntimeException("Unable to find validator class", e); - } - } - if (System.getProperties().containsKey("credential_factory_class")) { - try { - options.setCredentialFactoryClass((Class) Class.forName( - System.getProperty("credential_factory_class"))); - } catch (ClassNotFoundException e) { - throw new RuntimeException("Unable to find credential factory class", e); - } - } - } - - /** The constructor that takes PipelineOptions. Should be used only by unit tests. */ - StreamingDataflowWorker( List mapTasks, WindmillServerStub server, DataflowWorkerHarnessOptions options) { this.options = options; - initialize(mapTasks, server); + this.instructionMap = new ConcurrentHashMap<>(); + this.outputMap = new ConcurrentHashMap<>(); + this.mapTaskExecutors = new ConcurrentHashMap<>(); + for (MapTask mapTask : mapTasks) { + addComputation(mapTask); + } + this.threadFactory = new ThreadFactory() { + private final Thread.UncaughtExceptionHandler handler = + new Thread.UncaughtExceptionHandler() { + @Override + public void uncaughtException(Thread thread, Throwable e) { + LOG.error("Uncaught exception: ", e); + System.exit(1); + } + }; + + @Override + public Thread newThread(Runnable r) { + Thread t = new Thread(r); + t.setUncaughtExceptionHandler(handler); + t.setDaemon(true); + return t; + } + }; + this.executor = new BoundedQueueExecutor( + MAX_THREAD_POOL_SIZE, THREAD_EXPIRATION_TIME_SEC, TimeUnit.SECONDS, + MAX_THREAD_POOL_QUEUE_SIZE, threadFactory); + this.windmillServer = server; + this.running = new AtomicBoolean(); + this.stateFetcher = new StateFetcher(server); + this.clientId = new Random().nextLong(); + this.lastException = new AtomicReference<>(); + + DataflowWorkerLoggingFormatter.setJobId(options.getJobId()); + DataflowWorkerLoggingFormatter.setWorkerId(options.getWorkerId()); } public void start() { running.set(true); dispatchThread = threadFactory.newThread(new Runnable() { + @Override public void run() { dispatchLoop(); } @@ -180,6 +194,7 @@ public void run() { dispatchThread.start(); commitThread = threadFactory.newThread(new Runnable() { + @Override public void run() { commitLoop(); } @@ -212,42 +227,6 @@ public void stop() { } } - /** Initializes the execution harness. */ - private void initialize(List mapTasks, WindmillServerStub server) { - this.instructionMap = new ConcurrentHashMap<>(); - this.outputMap = new ConcurrentHashMap<>(); - this.mapTaskExecutors = new ConcurrentHashMap<>(); - for (MapTask mapTask : mapTasks) { - addComputation(mapTask); - } - this.threadFactory = new ThreadFactory() { - private final Thread.UncaughtExceptionHandler handler = - new Thread.UncaughtExceptionHandler() { - public void uncaughtException(Thread thread, Throwable e) { - LOG.error("Uncaught exception: ", e); - System.exit(1); - } - }; - public Thread newThread(Runnable r) { - Thread t = new Thread(r); - t.setUncaughtExceptionHandler(handler); - t.setDaemon(true); - return t; - } - }; - this.executor = new BoundedQueueExecutor( - MAX_THREAD_POOL_SIZE, THREAD_EXPIRATION_TIME_SEC, TimeUnit.SECONDS, - MAX_THREAD_POOL_QUEUE_SIZE, threadFactory); - this.windmillServer = server; - this.running = new AtomicBoolean(); - this.stateFetcher = new StateFetcher(server); - this.clientId = new Random().nextLong(); - this.lastException = new AtomicReference<>(); - - DataflowWorkerLoggingFormatter.setJobId(options.getJobId()); - DataflowWorkerLoggingFormatter.setWorkerId(options.getWorkerId()); - } - public void runStatusServer(int statusPort) { statusServer = new Server(statusPort); statusServer.setHandler(new StatusHandler()); @@ -320,6 +299,7 @@ private void dispatchLoop() { getConfig(computation); } executor.execute(new Runnable() { + @Override public void run() { process(computation, work); } @@ -402,6 +382,7 @@ private void process( // Try again, after some delay and at the end of the queue to avoid a tight loop. sleep(10000); executor.forceExecute(new Runnable() { + @Override public void run() { process(computation, work); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java index 8ec5eb25e125b..3cf4e1258f628 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java @@ -59,7 +59,7 @@ public void testAutomaticRegistrationOfRunners() { } @Test - public void testCreationFromSystemProperties() { + public void testCreationFromSystemProperties() throws Exception { System.getProperties().putAll(ImmutableMap .builder() .put("root_url", "test_root_url") @@ -71,7 +71,10 @@ public void testCreationFromSystemProperties() { .put("worker_id", "test_worker_id") .put("project_id", "test_project_id") .put("job_id", "test_job_id") + // Set a non-default value for testing + .put("sdk_pipeline_options", "{\"options\":{\"numWorkers\":999}}") .build()); + DataflowWorkerHarnessOptions options = PipelineOptionsFactory.createFromSystemProperties(); assertEquals("test_root_url", options.getApiRootUrl()); assertEquals("test_service_path", options.getDataflowEndpoint()); @@ -82,6 +85,7 @@ public void testCreationFromSystemProperties() { assertEquals("test_worker_id", options.getWorkerId()); assertEquals("test_project_id", options.getProject()); assertEquals("test_job_id", options.getJobId()); + assertEquals(999, options.getNumWorkers()); } @Test diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java index 21a4a02a770a2..24509b8dd374c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java @@ -56,6 +56,8 @@ import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.PDone; import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import org.junit.Assert; @@ -133,6 +135,7 @@ private static Dataflow buildMockDataflow( private static DataflowPipelineOptions buildPipelineOptions() throws IOException { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setGcpCredential(new TestCredential()); + options.setJobName("some-job-name"); options.setProject("some-project"); options.setTempLocation(GcsPath.fromComponents("somebucket", "some/path").toString()); options.setFilesToStage(new LinkedList()); @@ -140,12 +143,37 @@ private static DataflowPipelineOptions buildPipelineOptions() throws IOException return options; } + @Test + public void testSettingOfSdkPipelineOptions() throws IOException { + DataflowPipelineOptions options = buildPipelineOptions(); + options.setRunner(DataflowPipelineRunner.class); + + Pipeline p = buildPipeline(options); + p.traverseTopologically(new RecordingPipelineVisitor()); + Job job = DataflowPipelineTranslator.fromOptions(options).translate( + p, Collections.emptyList()); + + assertEquals(ImmutableMap.of("options", + ImmutableMap.builder() + .put("appName", "DataflowPipelineTranslatorTest") + .put("project", "some-project") + .put("pathValidatorClass", "com.google.cloud.dataflow.sdk.util.DataflowPathValidator") + .put("runner", "com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner") + .put("jobName", "some-job-name") + .put("tempLocation", "gs://somebucket/some/path") + .put("filesToStage", ImmutableList.of()) + .put("stagingLocation", "gs://somebucket/some/path/staging") + .build()), + job.getEnvironment().getSdkPipelineOptions()); + } + @Test public void testZoneConfig() throws IOException { final String testZone = "test-zone-1"; DataflowPipelineOptions options = buildPipelineOptions(); options.setZone(testZone); + options.setRunner(DataflowPipelineRunner.class); Pipeline p = buildPipeline(options); p.traverseTopologically(new RecordingPipelineVisitor()); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java index 3146c0e72c379..4d11659508985 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java @@ -144,7 +144,7 @@ public void setWorkerProgress(ApproximateProgress progress) { public void initMocksAndWorkflowServiceAndWorkerAndWork() throws IOException { MockitoAnnotations.initMocks(this); - options = PipelineOptionsFactory.createFromSystemProperties(); + options = PipelineOptionsFactory.as(DataflowWorkerHarnessOptions.class); options.setProject(PROJECT_ID); options.setJobId(JOB_ID); options.setWorkerId(WORKER_ID); From 177b0d19f65b26ab0dd4e51e8eee817405de09e4 Mon Sep 17 00:00:00 2001 From: dhuo Date: Wed, 25 Feb 2015 13:46:36 -0800 Subject: [PATCH 0186/1541] Add unittest for premature end-of-stream, add entry to CHANGES.txt. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87187120 --- .../sdk/util/gcsio/GoogleCloudStorageReadChannel.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java index 13c3dc427eaab..ab9c747b32228 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java @@ -262,7 +262,9 @@ public int read(ByteBuffer buffer) // actual size of the data stream when stream compression is used, so we can only ignore // this case here. Preconditions.checkState(isCompressedStream || currentPosition == size, - "Received end of stream result before all the file data has been received"); + "Received end of stream result before all the file data has been received; " + + "totalBytesRead: %s, currentPosition: %s, size: %s", + totalBytesRead, currentPosition, size); break; } totalBytesRead += numBytesRead; @@ -360,7 +362,8 @@ public int read(ByteBuffer buffer) // read against the stream size. Unfortunately we don't have information about the actual size // of the data stream when stream compression is used, so we can only ignore this case here. Preconditions.checkState(isCompressedStream || currentPosition == size, - "Failed to read any data before all the file data has been received"); + "Failed to read any data before all the file data has been received; " + + "currentPosition: %s, size: %s", currentPosition, size); return -1; } else { return totalBytesRead; From 734d441825cdb9fe9e96f2691e43d41e0ece5a9a Mon Sep 17 00:00:00 2001 From: vanya Date: Wed, 25 Feb 2015 13:58:32 -0800 Subject: [PATCH 0187/1541] Improve error message on exception in StandardCoder.getEncodedElementByteSize(). [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87188532 --- .../com/google/cloud/dataflow/sdk/coders/StandardCoder.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java index 2df352757a142..790e96fcf6d53 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java @@ -126,7 +126,7 @@ protected long getEncodedElementByteSize(T value, Context context) return os.size(); } catch (Exception exn) { throw new IllegalArgumentException( - "Unable to encode element " + value + " with coder " + this, exn); + "Unable to encode element '" + value + "' with coder '" + this + "'.", exn); } } From ac6bbdd034902855d04df5d857c8f0e7ebc73e5a Mon Sep 17 00:00:00 2001 From: vanya Date: Wed, 25 Feb 2015 15:07:29 -0800 Subject: [PATCH 0188/1541] Slightly improve invalid bucket name error message. This gets the specific problem description to the beginning, and more information is given after that. Also gets rid of an inconsistent double space at the beginning of one of the sentences. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87195785 --- .../cloud/dataflow/sdk/util/gcsfs/GcsPath.java | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPath.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPath.java index f1da8b767ef2e..9b0c75c09c3d7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPath.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPath.java @@ -203,14 +203,13 @@ public GcsPath(@Nullable FileSystem fs, } Preconditions.checkArgument(!bucket.contains("/"), "GCS bucket may not contain a slash"); - Preconditions - .checkArgument(bucket.isEmpty() - || bucket.matches("[a-z0-9][-_a-z0-9.]+[a-z0-9]"), - "GCS bucket names must contain only lowercase letters, numbers, " - + "dashes (-), underscores (_), and dots (.). Bucket names " - + "must start and end with a number or letter. " - + "See https://developers.google.com/storage/docs/bucketnaming " - + "for more details. Bucket name: " + bucket); + Preconditions.checkArgument( + bucket.isEmpty() || bucket.matches("[a-z0-9][-_a-z0-9.]+[a-z0-9]"), "" + + "Invalid bucket name: '" + bucket + "'. GCS bucket names must contain only " + + "lowercase letters, numbers, dashes (-), underscores (_), and dots (.). " + + "Bucket names must start and end with a number or letter. " + + "See https://developers.google.com/storage/docs/bucketnaming " + + "for more details."); if (object == null) { object = ""; From 1e0afad75dd3f762460493da6a9be6d282763207 Mon Sep 17 00:00:00 2001 From: vanya Date: Wed, 25 Feb 2015 15:51:43 -0800 Subject: [PATCH 0189/1541] Improve error message on CoderException. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87200225 --- .../dataflow/sdk/runners/DataflowPipelineTranslator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index 1a3d27498c892..622de9a4cbd7d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -805,8 +805,8 @@ private void createHelper( // TODO: Put in better element printing: // truncate if too long. throw new IllegalArgumentException( - "unable to encode element " + elem + " of " + transform - + " using " + coder, + "Unable to encode element '" + elem + "' of transform '" + transform + + "' using coder '" + coder + "'.", exn); } String encodedJson = byteArrayToJsonString(encodedBytes); From 48dfb0cecd2a1a2e1fed25d7c047225c4a2127e2 Mon Sep 17 00:00:00 2001 From: peihe Date: Wed, 25 Feb 2015 15:54:38 -0800 Subject: [PATCH 0190/1541] Fix the types in ExtractOutputDoFn java doc. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87200523 --- .../cloud/dataflow/sdk/runners/worker/CombineValuesFn.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java index 31e87649a5d27..b32588081828c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java @@ -203,7 +203,7 @@ public void processElement(ProcessContext c) { } /* - * EXTRACT phase: KV> -> KV. + * EXTRACT phase: KV -> KV. */ private static class ExtractOutputDoFn extends DoFn, KV>{ From b76455fcd29842b6c562de7587cc39d1e87cae13 Mon Sep 17 00:00:00 2001 From: peihe Date: Wed, 25 Feb 2015 17:06:02 -0800 Subject: [PATCH 0191/1541] Mark PartialGroupByKeyOperation can restart. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87206956 --- .../sdk/util/common/worker/PartialGroupByKeyOperation.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperation.java index a4afa5b2820d2..97072a5a18879 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperation.java @@ -194,6 +194,12 @@ public void finish() throws Exception { } } + @Override + public boolean supportsRestart() { + // SizeEstimators are safe to be reused. + return true; + } + /** * Sets the maximum amount of memory the grouping table is allowed to * consume before it has to be flushed. From 19f9f05cd5c9d1dbb215eddd8dbff5d394f94de8 Mon Sep 17 00:00:00 2001 From: earhart Date: Wed, 25 Feb 2015 21:16:26 -0800 Subject: [PATCH 0192/1541] Switch the Dataflow SDK to use the new Dataflow service endpoint. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87220492 --- .../dataflow/sdk/options/DataflowPipelineDebugOptions.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java index 0ec0bbf044375..aba60e1ef5e7e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java @@ -41,7 +41,7 @@ public interface DataflowPipelineDebugOptions extends PipelineOptions { * url. */ @Description("Cloud Dataflow Endpoint") - @Default.String("dataflow/v1b3/projects/") + @Default.String("v1b3/projects/") String getDataflowEndpoint(); void setDataflowEndpoint(String value); @@ -61,7 +61,7 @@ public interface DataflowPipelineDebugOptions extends PipelineOptions { * The API endpoint to use when communicating with the Dataflow service. */ @Description("Google Cloud root API") - @Default.String("https://www.googleapis.com/") + @Default.String("https://dataflow.googleapis.com/") String getApiRootUrl(); void setApiRootUrl(String value); From 92d06ebc0c61bfd3aa9fd6f487d848abe7e9ed79 Mon Sep 17 00:00:00 2001 From: kirpichov Date: Thu, 26 Feb 2015 11:39:30 -0800 Subject: [PATCH 0193/1541] Rollback the change to slightly improve invalid bucket name error message. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87270734 --- .../cloud/dataflow/sdk/util/gcsfs/GcsPath.java | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPath.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPath.java index 9b0c75c09c3d7..f1da8b767ef2e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPath.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsfs/GcsPath.java @@ -203,13 +203,14 @@ public GcsPath(@Nullable FileSystem fs, } Preconditions.checkArgument(!bucket.contains("/"), "GCS bucket may not contain a slash"); - Preconditions.checkArgument( - bucket.isEmpty() || bucket.matches("[a-z0-9][-_a-z0-9.]+[a-z0-9]"), "" - + "Invalid bucket name: '" + bucket + "'. GCS bucket names must contain only " - + "lowercase letters, numbers, dashes (-), underscores (_), and dots (.). " - + "Bucket names must start and end with a number or letter. " - + "See https://developers.google.com/storage/docs/bucketnaming " - + "for more details."); + Preconditions + .checkArgument(bucket.isEmpty() + || bucket.matches("[a-z0-9][-_a-z0-9.]+[a-z0-9]"), + "GCS bucket names must contain only lowercase letters, numbers, " + + "dashes (-), underscores (_), and dots (.). Bucket names " + + "must start and end with a number or letter. " + + "See https://developers.google.com/storage/docs/bucketnaming " + + "for more details. Bucket name: " + bucket); if (object == null) { object = ""; From 0a97da58291c946be8a0beb60aafa71ccc68d89d Mon Sep 17 00:00:00 2001 From: millsd Date: Thu, 26 Feb 2015 11:48:37 -0800 Subject: [PATCH 0194/1541] Pick a sensible default period for SlidingWindows based on the given size of the windows [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87271369 --- .../transforms/windowing/SlidingWindows.java | 18 +++++++++++++-- .../windowing/SlidingWindowsTest.java | 22 +++++++++++++++++++ 2 files changed, 38 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java index d29062e31ef2e..2ab1737ac9975 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java @@ -61,10 +61,11 @@ public class SlidingWindows extends NonMergingWindowFn { * [N * period, N * period + size), where 0 is the epoch. * *

    If {@link SlidingWindows#every} is not called, the period defaults - * to one millisecond. + * to the largest time unit smaller than the given duration. For example, + * specifying a size of 5 seconds will result in a default period of 1 second. */ public static SlidingWindows of(Duration size) { - return new SlidingWindows(new Duration(1), size, Duration.ZERO); + return new SlidingWindows(getDefaultPeriod(size), size, Duration.ZERO); } /** @@ -129,4 +130,17 @@ public boolean isCompatible(WindowFn other) { return false; } } + + static Duration getDefaultPeriod(Duration size) { + if (size.isLongerThan(Duration.standardHours(1))) { + return Duration.standardHours(1); + } + if (size.isLongerThan(Duration.standardMinutes(1))) { + return Duration.standardMinutes(1); + } + if (size.isLongerThan(Duration.standardSeconds(1))) { + return Duration.standardSeconds(1); + } + return Duration.millis(1); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java index 8af9782fc382c..fbad9e790e0ea 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java @@ -110,6 +110,28 @@ public void testTimeUnit() throws Exception { Arrays.asList(1L, 2L, 1000L, 5000L, 5001L, 10000L))); } + @Test + public void testDefaultPeriods() throws Exception { + assertEquals(Duration.standardHours(1), + SlidingWindows.getDefaultPeriod(Duration.standardDays(1))); + assertEquals(Duration.standardHours(1), + SlidingWindows.getDefaultPeriod(Duration.standardHours(2))); + assertEquals(Duration.standardMinutes(1), + SlidingWindows.getDefaultPeriod(Duration.standardHours(1))); + assertEquals(Duration.standardMinutes(1), + SlidingWindows.getDefaultPeriod(Duration.standardMinutes(10))); + assertEquals(Duration.standardSeconds(1), + SlidingWindows.getDefaultPeriod(Duration.standardMinutes(1))); + assertEquals(Duration.standardSeconds(1), + SlidingWindows.getDefaultPeriod(Duration.standardSeconds(10))); + assertEquals(Duration.millis(1), + SlidingWindows.getDefaultPeriod(Duration.standardSeconds(1))); + assertEquals(Duration.millis(1), + SlidingWindows.getDefaultPeriod(Duration.millis(10))); + assertEquals(Duration.millis(1), + SlidingWindows.getDefaultPeriod(Duration.millis(1))); + } + @Test public void testEquality() { assertTrue( From cd845a9594eb6dc87c697c948561bd19dfd28e9e Mon Sep 17 00:00:00 2001 From: relax Date: Fri, 27 Feb 2015 01:12:45 -0800 Subject: [PATCH 0195/1541] Reduce the number of calls to BigQuery table creation to reduce quota issues. Previously every thread tried to create the table at startup. Synchronizing this call prevents this. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87323976 --- .../cloud/dataflow/sdk/io/BigQueryIO.java | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java index 63e29aaf07162..afee01f2fe040 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java @@ -682,11 +682,18 @@ public void startBundle(Context context) { JSON_FACTORY.fromString(jsonTableReference, TableReference.class); if (!createdTables.contains(jsonTableSchema)) { - TableSchema tableSchema = JSON_FACTORY.fromString(jsonTableSchema, TableSchema.class); - Bigquery client = Transport.newBigQueryClient(options).build(); - BigQueryTableInserter inserter = new BigQueryTableInserter(client, tableReference); - inserter.tryCreateTable(tableSchema); - createdTables.add(jsonTableSchema); + synchronized (createdTables) { + // Another thread may have succeeded in creating the table in the meanwhile, so + // check again. This check isn't needed for correctness, but we add it to prevent + // every thread from attempting a create and overwhelming our BigQuery quota. + if (!createdTables.contains(jsonTableSchema)) { + TableSchema tableSchema = JSON_FACTORY.fromString(jsonTableSchema, TableSchema.class); + Bigquery client = Transport.newBigQueryClient(options).build(); + BigQueryTableInserter inserter = new BigQueryTableInserter(client, tableReference); + inserter.tryCreateTable(tableSchema); + createdTables.add(jsonTableSchema); + } + } } } catch (IOException e) { throw new RuntimeException(e); From 0aa4d9d539a23d765a6a0d4f7fa9d82626d960ae Mon Sep 17 00:00:00 2001 From: lcwik Date: Fri, 27 Feb 2015 10:45:54 -0800 Subject: [PATCH 0196/1541] Add support for setting the default log level and also custom log level overrides on the Dataflow worker. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87355253 --- .../sdk/options/DataflowPipelineOptions.java | 2 +- .../options/DataflowWorkerLoggingOptions.java | 154 ++++++++++++++++++ .../runners/worker/DataflowWorkerHarness.java | 4 +- .../worker/StreamingDataflowWorker.java | 16 +- .../DataflowWorkerLoggingInitializer.java | 103 +++++++++--- .../DataflowWorkerLoggingOptionsTest.java | 70 ++++++++ .../DataflowWorkerLoggingInitializerTest.java | 106 ++++++------ 7 files changed, 371 insertions(+), 84 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptionsTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java index b71949ec9947b..6028326788806 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java @@ -34,7 +34,7 @@ public interface DataflowPipelineOptions extends PipelineOptions, GcpOptions, ApplicationNameOptions, DataflowPipelineDebugOptions, DataflowPipelineWorkerPoolOptions, BigQueryOptions, - GcsOptions, StreamingOptions, CloudDebuggerOptions { + GcsOptions, StreamingOptions, CloudDebuggerOptions, DataflowWorkerLoggingOptions { /** * GCS path for temporary files. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java new file mode 100644 index 0000000000000..e14b97e718d24 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java @@ -0,0 +1,154 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.common.base.Preconditions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; + +import java.util.Arrays; + +/** + * Options which are used to control logging configuration on the Dataflow worker. + */ +public interface DataflowWorkerLoggingOptions extends PipelineOptions { + /** + * The set of log levels which can be used on the Dataflow worker. + */ + public enum Level { + DEBUG, ERROR, INFO, TRACE, WARN + } + + /** + * This option controls the default log level of all loggers without a + * log level override. + */ + @Default.Enum("INFO") + Level getDefaultWorkerLogLevel(); + void setDefaultWorkerLogLevel(Level level); + + /** + * This option controls the log levels for specifically named loggers. + *

    + * Later options with equivalent names override earlier options. + *

    + * See {@link WorkerLogLevelOverride} for more information on how to configure logging + * on a per {@link Class}, {@link Package}, or name basis. + */ + WorkerLogLevelOverride[] getWorkerLogLevelOverrides(); + void setWorkerLogLevelOverrides(WorkerLogLevelOverride[] string); + + /** + * Defines a log level override for a specific class, package, or name. + *

    + * {@link java.util.logging} is used on the Dataflow worker harness and supports + * a logging hierarchy based off of names which are "." separated. It is a common + * pattern to have the logger for a given class share the same name as the class itself. + * Given the classes {@code a.b.c.Foo}, {@code a.b.c.Xyz}, and {@code a.b.Bar}, with + * loggers named {@code "a.b.c.Foo"}, {@code "a.b.c.Xyz"}, and {@code "a.b.Bar"} respectively, + * we can override the log levels: + *

      + *
    • for {@code Foo} by specifying the name {@code "a.b.c.Foo"} or the {@link Class} + * representing {@code a.b.c.Foo}. + *
    • for {@code Foo}, {@code Xyz}, and {@code Bar} by specifying the name {@code "a.b"} or + * the {@link Package} representing {@code a.b}. + *
    • for {@code Foo} and {@code Bar} by specifying both of their names or classes. + *
    + * Note that by specifying multiple overrides, the exact name followed by the closest parent + * takes precedence. + */ + public static class WorkerLogLevelOverride { + private static final String SEPARATOR = "#"; + + /** + * Overrides the default log level for the passed in class. + *

    + * This is equivalent to calling {@link #forName(String, Level)} and + * passing in the {@link Class#getName() class name}. + */ + public static WorkerLogLevelOverride forClass(Class klass, Level level) { + Preconditions.checkNotNull(klass, "Expected class to be not null."); + return forName(klass.getName(), level); + } + + /** + * Overrides the default log level for the passed in package. + *

    + * This is equivalent to calling {@link #forName(String, Level)} and + * passing in the {@link Package#getName() package name}. + */ + public static WorkerLogLevelOverride forPackage(Package pkg, Level level) { + Preconditions.checkNotNull(pkg, "Expected package to be not null."); + return forName(pkg.getName(), level); + } + + /** + * Overrides the default log level for the passed in name. + *

    + * Note that because of the hierarchical nature of logger names, this will + * override the log level of all loggers which have the passed in name or + * a parent logger which has the passed in name. + */ + public static WorkerLogLevelOverride forName(String name, Level level) { + Preconditions.checkNotNull(name, "Expected name to be not null."); + Preconditions.checkNotNull(level, + "Expected level to be one of %s.", Arrays.toString(Level.values())); + return new WorkerLogLevelOverride(name, level); + } + + /** + * Expects a value of the form {@code Name#Level}. + */ + @JsonCreator + public static WorkerLogLevelOverride create(String value) { + Preconditions.checkNotNull(value, "Expected value to be not null."); + Preconditions.checkArgument(value.contains(SEPARATOR), + "Expected '#' separator but none found within '%s'.", value); + String[] parts = value.split(SEPARATOR, 2); + Level level; + try { + level = Level.valueOf(parts[1]); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException(String.format( + "Unsupported log level '%s' requested. Must be one of %s.", + parts[1], Arrays.toString(Level.values()))); + } + return forName(parts[0], level); + } + + private final String name; + private final Level level; + private WorkerLogLevelOverride(String name, Level level) { + this.name = name; + this.level = level; + } + + public String getName() { + return name; + } + + public Level getLevel() { + return level; + } + + @JsonValue + @Override + public String toString() { + return name + SEPARATOR + level; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java index 3ac2895afcba4..36f603f7d4127 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java @@ -93,10 +93,12 @@ public void uncaughtException(Thread t, Throwable e) { */ public static void main(String[] args) throws Exception { Thread.currentThread().setUncaughtExceptionHandler(WorkerUncaughtExceptionHandler.INSTANCE); - new DataflowWorkerLoggingInitializer().initialize(); + DataflowWorkerLoggingInitializer.initialize(); DataflowWorkerHarnessOptions pipelineOptions = PipelineOptionsFactory.createFromSystemProperties(); + DataflowWorkerLoggingInitializer.configure(pipelineOptions); + final DataflowWorker worker = create(pipelineOptions); processWork(pipelineOptions, worker); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java index 25eaa4aa1b100..f92edbb0cffa5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -91,8 +91,15 @@ static MapTask parseMapTask(String input) throws IOException { } public static void main(String[] args) throws Exception { - new DataflowWorkerLoggingInitializer().initialize(); + DataflowWorkerLoggingInitializer.initialize(); + DataflowWorkerHarnessOptions options = + PipelineOptionsFactory.createFromSystemProperties(); + // TODO: Remove setting these options once we have migrated to passing + // through the pipeline options. + options.setAppName("StreamingWorkerHarness"); + options.setStreaming(true); + DataflowWorkerLoggingInitializer.configure(options); String hostport = System.getProperty("windmill.hostport"); if (hostport == null) { throw new Exception("-Dwindmill.hostport must be set to the location of the windmill server"); @@ -112,13 +119,6 @@ public static void main(String[] args) throws Exception { (WindmillServerStub) Class.forName(WINDMILL_SERVER_CLASS_NAME) .getDeclaredConstructor(String.class).newInstance(hostport); - DataflowWorkerHarnessOptions options = - PipelineOptionsFactory.createFromSystemProperties(); - // TODO: Remove setting these options once we have migrated to passing - // through the pipeline options. - options.setAppName("StreamingWorkerHarness"); - options.setStreaming(true); - StreamingDataflowWorker worker = new StreamingDataflowWorker(mapTasks, windmillServer, options); worker.start(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java index a513dc75ee359..dbe8c6e1d468d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java @@ -16,10 +16,20 @@ package com.google.cloud.dataflow.sdk.runners.worker.logging; +import static com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.Level.DEBUG; +import static com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.Level.ERROR; +import static com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.Level.INFO; +import static com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.Level.TRACE; +import static com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.Level.WARN; + +import com.google.api.client.util.Lists; +import com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions; +import com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverride; import com.google.common.collect.ImmutableBiMap; import java.io.File; import java.io.IOException; +import java.util.List; import java.util.logging.FileHandler; import java.util.logging.Formatter; import java.util.logging.Handler; @@ -28,47 +38,60 @@ import java.util.logging.Logger; /** - * Sets up java.util.Logging configuration on the Dataflow Worker Harness with a - * console and file logger. The console and file loggers use the - * {@link DataflowWorkerLoggingFormatter} format. A user can override - * the logging level and location by specifying the Java system properties - * "dataflow.worker.logging.level" and "dataflow.worker.logging.location" respectively. - * The default log level is INFO and the default location is a file named dataflow-worker.log - * within the systems temporary directory. + * Sets up {@link java.util.logging} configuration on the Dataflow worker with a + * file logger. The file logger uses the {@link DataflowWorkerLoggingFormatter} format. + * A user can override the logging level by customizing the options found within + * {@link DataflowWorkerLoggingOptions}. A user can override the location by specifying the + * Java system property "dataflow.worker.logging.location". The default log level is INFO + * and the default location is a file named dataflow-worker.log within the systems temporary + * directory. */ public class DataflowWorkerLoggingInitializer { private static final String DEFAULT_LOGGING_LOCATION = new File(System.getProperty("java.io.tmpdir"), "dataflow-worker.log").getPath(); private static final String ROOT_LOGGER_NAME = ""; - public static final String DATAFLOW_WORKER_LOGGING_LEVEL = "dataflow.worker.logging.level"; - public static final String DATAFLOW_WORKER_LOGGING_LOCATION = "dataflow.worker.logging.location"; - public static final ImmutableBiMap LEVELS = - ImmutableBiMap.builder() - .put(Level.SEVERE, "ERROR") - .put(Level.WARNING, "WARNING") - .put(Level.INFO, "INFO") - .put(Level.FINE, "DEBUG") - .put(Level.FINEST, "TRACE") + private static final String DATAFLOW_WORKER_LOGGING_LOCATION = "dataflow.worker.logging.location"; + static final ImmutableBiMap LEVELS = + ImmutableBiMap.builder() + .put(Level.SEVERE, ERROR) + .put(Level.WARNING, WARN) + .put(Level.INFO, INFO) + .put(Level.FINE, DEBUG) + .put(Level.FINEST, TRACE) .build(); - private static final String DEFAULT_LOG_LEVEL = LEVELS.get(Level.INFO); - public void initialize() { - initialize(LogManager.getLogManager()); - } + /** + * This default log level is overridden by the log level found at + * {@code DataflowWorkerLoggingOptions#getDefaultWorkerLogLevel()}. + */ + private static final DataflowWorkerLoggingOptions.Level DEFAULT_LOG_LEVEL = + LEVELS.get(Level.INFO); + + /* We need to store a reference to the configured loggers so that they are not + * garbage collected. java.util.logging only has weak references to the loggers + * so if they are garbage collection, our hierarchical configuration will be lost. */ + private static List configuredLoggers = Lists.newArrayList(); + private static FileHandler fileHandler; - void initialize(LogManager logManager) { + /** + * Sets up the initial logging configuration. + */ + public static synchronized void initialize() { + if (fileHandler != null) { + return; + } try { - Level logLevel = LEVELS.inverse().get( - System.getProperty(DATAFLOW_WORKER_LOGGING_LEVEL, DEFAULT_LOG_LEVEL)); + Level logLevel = LEVELS.inverse().get(DEFAULT_LOG_LEVEL); Formatter formatter = new DataflowWorkerLoggingFormatter(); - FileHandler fileHandler = new FileHandler( + fileHandler = new FileHandler( System.getProperty(DATAFLOW_WORKER_LOGGING_LOCATION, DEFAULT_LOGGING_LOCATION), true /* Append so that we don't squash existing logs */); fileHandler.setFormatter(formatter); - fileHandler.setLevel(logLevel); + fileHandler.setLevel(Level.ALL); // Reset the global log manager, get the root logger and remove the default log handlers. + LogManager logManager = LogManager.getLogManager(); logManager.reset(); Logger rootLogger = logManager.getLogger(ROOT_LOGGER_NAME); for (Handler handler : rootLogger.getHandlers()) { @@ -81,4 +104,34 @@ void initialize(LogManager logManager) { throw new ExceptionInInitializerError(e); } } + + /** + * Reconfigures logging with the passed in options. + */ + public static synchronized void configure(DataflowWorkerLoggingOptions options) { + initialize(); + if (options.getDefaultWorkerLogLevel() != null) { + LogManager.getLogManager().getLogger(ROOT_LOGGER_NAME).setLevel( + LEVELS.inverse().get(options.getDefaultWorkerLogLevel())); + } + /* We store a reference to all the custom loggers the user configured. + * To make sure that these custom levels override the default logger level, + * we break the parent chain and have the logger directly pass log records + * to the file handler. */ + if (options.getWorkerLogLevelOverrides() != null) { + for (WorkerLogLevelOverride loggerOverride : options.getWorkerLogLevelOverrides()) { + Logger logger = Logger.getLogger(loggerOverride.getName()); + logger.setUseParentHandlers(false); + logger.setLevel(LEVELS.inverse().get(loggerOverride.getLevel())); + logger.addHandler(fileHandler); + configuredLoggers.add(logger); + } + } + } + + // Visible for testing + static void reset() { + configuredLoggers = Lists.newArrayList(); + fileHandler = null; + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptionsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptionsTest.java new file mode 100644 index 0000000000000..fffef0e888f28 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptionsTest.java @@ -0,0 +1,70 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import static com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.Level.WARN; +import static org.junit.Assert.assertEquals; + +import com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverride; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link DataflowWorkerLoggingOptions}. */ +@RunWith(JUnit4.class) +public class DataflowWorkerLoggingOptionsTest { + private static final ObjectMapper MAPPER = new ObjectMapper(); + @Rule public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testWorkerLogLevelOverrideWithInvalidLogLevel() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Unsupported log level"); + WorkerLogLevelOverride.create("Name#FakeLevel"); + } + + @Test + public void testWorkerLogLevelOverrideForClass() { + assertEquals("org.junit.Test#WARN", + MAPPER.convertValue(WorkerLogLevelOverride.forClass(Test.class, WARN), String.class)); + } + + @Test + public void testWorkerLogLevelOverrideForPackage() { + assertEquals("org.junit#WARN", + MAPPER.convertValue( + WorkerLogLevelOverride.forPackage(Test.class.getPackage(), WARN), String.class)); + } + + @Test + public void testWorkerLogLevelOverrideForName() { + assertEquals("A#WARN", + MAPPER.convertValue(WorkerLogLevelOverride.forName("A", WARN), String.class)); + } + + @Test + public void testSerializationAndDeserializationOf() { + String testValue = "A#WARN"; + assertEquals(testValue, + MAPPER.convertValue( + MAPPER.convertValue(testValue, WorkerLogLevelOverride.class), String.class)); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java index 50cc1e2d3ed18..8cc25a9e0e7df 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java @@ -16,24 +16,19 @@ package com.google.cloud.dataflow.sdk.runners.worker.logging; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.Mockito.when; -import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties; +import com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions; +import com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverride; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import org.junit.Before; -import org.junit.Rule; +import org.junit.After; import org.junit.Test; -import org.junit.rules.TestRule; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.mockito.ArgumentCaptor; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; -import java.util.List; import java.util.logging.FileHandler; import java.util.logging.Handler; import java.util.logging.Level; @@ -43,52 +38,65 @@ /** Unit tests for {@link DataflowWorkerLoggingInitializer}. */ @RunWith(JUnit4.class) public class DataflowWorkerLoggingInitializerTest { - @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); + @After + public void tearDown() { + LogManager.getLogManager().reset(); + DataflowWorkerLoggingInitializer.reset(); + } - @Mock LogManager mockLogManager; - @Mock Logger mockRootLogger; - @Mock Handler mockHandler; + @Test + public void testWithDefaults() { + DataflowWorkerLoggingOptions options = + PipelineOptionsFactory.as(DataflowWorkerLoggingOptions.class); - @Before - public void setUp() { - MockitoAnnotations.initMocks(this); - when(mockLogManager.getLogger("")).thenReturn(mockRootLogger); - when(mockRootLogger.getHandlers()).thenReturn(new Handler[]{ mockHandler }); + DataflowWorkerLoggingInitializer.initialize(); + DataflowWorkerLoggingInitializer.configure(options); + + Logger rootLogger = LogManager.getLogManager().getLogger(""); + assertEquals(1, rootLogger.getHandlers().length); + assertEquals(Level.INFO, rootLogger.getLevel()); + assertTrue(isFileHandler(rootLogger.getHandlers()[0], Level.ALL)); } @Test - public void testWithDefaults() { - ArgumentCaptor argument = ArgumentCaptor.forClass(Handler.class); - - new DataflowWorkerLoggingInitializer().initialize(mockLogManager); - verify(mockLogManager).getLogger(""); - verify(mockLogManager).reset(); - verify(mockRootLogger).getHandlers(); - verify(mockRootLogger).removeHandler(mockHandler); - verify(mockRootLogger).setLevel(Level.INFO); - verify(mockRootLogger).addHandler(argument.capture()); - verifyNoMoreInteractions(mockLogManager, mockRootLogger); - - List handlers = argument.getAllValues(); - assertTrue(isFileHandler(handlers.get(0), Level.INFO)); + public void testWithConfigurationOverride() { + DataflowWorkerLoggingOptions options = + PipelineOptionsFactory.as(DataflowWorkerLoggingOptions.class); + options.setDefaultWorkerLogLevel(DataflowWorkerLoggingOptions.Level.WARN); + + DataflowWorkerLoggingInitializer.initialize(); + DataflowWorkerLoggingInitializer.configure(options); + + Logger rootLogger = LogManager.getLogManager().getLogger(""); + assertEquals(1, rootLogger.getHandlers().length); + assertEquals(Level.WARNING, rootLogger.getLevel()); + assertTrue(isFileHandler(rootLogger.getHandlers()[0], Level.ALL)); } @Test - public void testWithOverrides() { - ArgumentCaptor argument = ArgumentCaptor.forClass(Handler.class); - System.setProperty("dataflow.worker.logging.level", "WARNING"); - - new DataflowWorkerLoggingInitializer().initialize(mockLogManager); - verify(mockLogManager).getLogger(""); - verify(mockLogManager).reset(); - verify(mockRootLogger).getHandlers(); - verify(mockRootLogger).removeHandler(mockHandler); - verify(mockRootLogger).setLevel(Level.WARNING); - verify(mockRootLogger).addHandler(argument.capture()); - verifyNoMoreInteractions(mockLogManager, mockRootLogger); - - List handlers = argument.getAllValues(); - assertTrue(isFileHandler(handlers.get(0), Level.WARNING)); + public void testWithCustomLogLevels() { + DataflowWorkerLoggingOptions options = + PipelineOptionsFactory.as(DataflowWorkerLoggingOptions.class); + options.setWorkerLogLevelOverrides( + new WorkerLogLevelOverride[] { + WorkerLogLevelOverride.forName("A", DataflowWorkerLoggingOptions.Level.DEBUG), + WorkerLogLevelOverride.forName("B", DataflowWorkerLoggingOptions.Level.ERROR), + }); + + DataflowWorkerLoggingInitializer.initialize(); + DataflowWorkerLoggingInitializer.configure(options); + + Logger aLogger = LogManager.getLogManager().getLogger("A"); + assertEquals(1, aLogger.getHandlers().length); + assertEquals(Level.FINE, aLogger.getLevel()); + assertFalse(aLogger.getUseParentHandlers()); + assertTrue(isFileHandler(aLogger.getHandlers()[0], Level.ALL)); + + Logger bLogger = LogManager.getLogManager().getLogger("B"); + assertEquals(1, bLogger.getHandlers().length); + assertEquals(Level.SEVERE, bLogger.getLevel()); + assertFalse(bLogger.getUseParentHandlers()); + assertTrue(isFileHandler(bLogger.getHandlers()[0], Level.ALL)); } private boolean isFileHandler(Handler handler, Level level) { From 6fffedec103721965c42a17d42b2f3701c4cb9fa Mon Sep 17 00:00:00 2001 From: Derek Perez Date: Wed, 28 Jan 2015 15:08:25 -0800 Subject: [PATCH 0197/1541] Proto2 Coder support. --- .../dataflow/sdk/coders/Proto2Coder.java | 139 ++++++++++++++++++ .../dataflow/sdk/coders/Proto2CoderTest.java | 64 ++++++++ 2 files changed, 203 insertions(+) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/Proto2CoderTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java new file mode 100644 index 0000000000000..6b89ec140fc20 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java @@ -0,0 +1,139 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.protobuf.ExtensionRegistry; +import com.google.protobuf.Message; +import com.google.protobuf.Parser; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.List; + +/** + * An encoder using Google Protocol Buffers 2 binary format. + *

    + * To learn more about Protocol Buffers, visit: + * https://developers.google.com/protocol-buffers + *

    + * To use, specify the {@code Coder} type on a PCollection: + *

    + * {@code
    + * PCollection records =
    + *     input.apply(...)
    + *          .setCoder(Proto2Coder.of(MyProto.Message.class));
    + * }
    + * 
    + *

    + * Custom message extensions are also supported, but the coder must be made + * aware of them explicitly: + *

    + * {@code
    + * PCollection records =
    + *     input.apply(...)
    + *          .setCoder(Proto2Coder.of(MyProto.Message.class)
    + *          .withExtensionsFrom(MyProto.class));
    + * }
    + * 
    + * + * @param the type of elements handled by this coder, must extend {@code Message} + */ +public class Proto2Coder extends CustomCoder { + + /** + * Produces a new Proto2Coder instance, for a given Protobuf message class. + */ + public static Proto2Coder of(Class protoMessageClass) { + return new Proto2Coder<>(protoMessageClass); + } + + private final Class protoMessageClass; + private final List> extensionClassList = new ArrayList<>(); + private transient Parser parser; + private transient ExtensionRegistry extensionRegistry; + + Proto2Coder(Class protoMessageClass) { + this.protoMessageClass = protoMessageClass; + } + + /** + * Adds custom Protobuf extensions to the coder. + * + * @param extensionHosts must be a class that defines a static + * method name {@code registerAllExtensions} + */ + public Proto2Coder withExtensionsFrom(Class... extensionHosts) { + for (Class extensionHost : extensionHosts) { + try { + // Attempt to access the declared method, to make sure its present. + extensionHost + .getDeclaredMethod("registerAllExtensions", ExtensionRegistry.class); + } catch (NoSuchMethodException e) { + throw new IllegalArgumentException(e); + } + extensionClassList.add(extensionHost); + } + return this; + } + + @Override + public void encode(T value, OutputStream outStream, Context context) throws IOException { + value.writeTo(outStream); + } + + @Override + public T decode(InputStream inStream, Context context) throws IOException { + return (T) getParser().parseFrom(inStream, getExtensionRegistry()); + } + + private Parser getParser() { + if (parser != null) { + return parser; + } + try { + T protoMessageInstance = (T) protoMessageClass + .getMethod("getDefaultInstance").invoke(null); + parser = (Parser) protoMessageInstance.getParserForType(); + } catch (IllegalAccessException + | InvocationTargetException + | NoSuchMethodException e) { + throw new IllegalArgumentException(e); + } + return parser; + } + + private ExtensionRegistry getExtensionRegistry() { + if (extensionRegistry != null) { + return extensionRegistry; + } + extensionRegistry = ExtensionRegistry.newInstance(); + for (Class extensionHost : extensionClassList) { + try { + extensionHost + .getDeclaredMethod("registerAllExtensions", ExtensionRegistry.class) + .invoke(null, extensionRegistry); + } catch (IllegalAccessException + | InvocationTargetException + | NoSuchMethodException e) { + throw new IllegalStateException(e); + } + } + return extensionRegistry; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/Proto2CoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/Proto2CoderTest.java new file mode 100644 index 0000000000000..11b5f6e85b7c9 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/Proto2CoderTest.java @@ -0,0 +1,64 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageB; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageC; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for Proto2Coder. + */ +@RunWith(JUnit4.class) +public class Proto2CoderTest { + + @Test + public void testCoderEncodeDecodeEqual() throws Exception { + MessageA value = MessageA.newBuilder() + .setField1("hello") + .addField2(MessageB.newBuilder() + .setField1(true).build()) + .addField2(MessageB.newBuilder() + .setField1(false).build()) + .build(); + CoderProperties.coderDecodeEncodeEqual(Proto2Coder.of(MessageA.class), value); + } + + @Test + public void testCoderEncodeDecodeExtensionsEqual() throws Exception { + MessageC value = MessageC.newBuilder() + .setExtension(Proto2CoderTestMessages.field1, + MessageA.newBuilder() + .setField1("hello") + .addField2(MessageB.newBuilder() + .setField1(true) + .build()) + .build()) + .setExtension(Proto2CoderTestMessages.field2, + MessageB.newBuilder() + .setField1(false) + .build()) + .build(); + CoderProperties.coderDecodeEncodeEqual( + Proto2Coder.of(MessageC.class) + .withExtensionsFrom(Proto2CoderTestMessages.class), value); + } +} From 3f38c967fdb59395c183e7b94d8f1711b335744f Mon Sep 17 00:00:00 2001 From: lcwik Date: Fri, 27 Feb 2015 13:52:08 -0800 Subject: [PATCH 0198/1541] Enable Google API tracing on the worker. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87370988 --- .../dataflow/sdk/options/GoogleApiDebugOptions.java | 7 +++++++ .../sdk/options/GoogleApiDebugOptionsTest.java | 11 +++++++++++ 2 files changed, 18 insertions(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java index 72e4bae0032d8..6675de8b6f335 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java @@ -22,6 +22,7 @@ import com.google.common.base.Preconditions; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; import java.io.IOException; import java.util.regex.Matcher; @@ -99,5 +100,11 @@ public void initialize(AbstractGoogleClientRequest request) throws IOExceptio request.set("trace", token); } } + + @JsonValue + @Override + public String toString() { + return clientRequestName + "#" + token; + } } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptionsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptionsTest.java index 147021a710e12..4df9fe9aaed00 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptionsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptionsTest.java @@ -27,6 +27,8 @@ import com.google.cloud.dataflow.sdk.util.TestCredential; import com.google.cloud.dataflow.sdk.util.Transport; +import com.fasterxml.jackson.databind.ObjectMapper; + import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -131,4 +133,13 @@ public void testMatchingAgainstRequestType() throws Exception { options.getDataflowClient().v1b3().projects().jobs().create("testProjectId", null); assertNull(createRequest.get("trace")); } + + @Test + public void testDeserializationAndSerializationOfGoogleApiTracer() { + String serializedValue = "Api#Token"; + ObjectMapper objectMapper = new ObjectMapper(); + assertEquals(serializedValue, + objectMapper.convertValue( + objectMapper.convertValue(serializedValue, GoogleApiTracer.class), String.class)); + } } From 4f14cb6a5a155d69733f5168f8862a93f0e17447 Mon Sep 17 00:00:00 2001 From: lcwik Date: Fri, 27 Feb 2015 15:02:24 -0800 Subject: [PATCH 0199/1541] Update Copyright notice to 2015. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87377460 --- checkstyle.xml | 4 ++-- examples/pom.xml | 2 +- .../com/google/cloud/dataflow/examples/BigQueryTornadoes.java | 2 +- .../google/cloud/dataflow/examples/CombinePerKeyExamples.java | 2 +- .../google/cloud/dataflow/examples/DatastoreWordCount.java | 2 +- .../java/com/google/cloud/dataflow/examples/DeDupExample.java | 2 +- .../com/google/cloud/dataflow/examples/FilterExamples.java | 2 +- .../java/com/google/cloud/dataflow/examples/JoinExamples.java | 2 +- .../com/google/cloud/dataflow/examples/MaxPerKeyExamples.java | 2 +- .../google/cloud/dataflow/examples/PubsubFileInjector.java | 2 +- .../google/cloud/dataflow/examples/StreamingWordExtract.java | 2 +- .../main/java/com/google/cloud/dataflow/examples/TfIdf.java | 2 +- .../google/cloud/dataflow/examples/TopWikipediaSessions.java | 2 +- .../google/cloud/dataflow/examples/WindowingWordCount.java | 2 +- .../java/com/google/cloud/dataflow/examples/WordCount.java | 2 +- .../google/cloud/dataflow/examples/BigQueryTornadoesTest.java | 2 +- .../cloud/dataflow/examples/CombinePerKeyExamplesTest.java | 2 +- .../com/google/cloud/dataflow/examples/DeDupExampleTest.java | 2 +- .../google/cloud/dataflow/examples/FilterExamplesTest.java | 2 +- .../com/google/cloud/dataflow/examples/JoinExamplesTest.java | 2 +- .../google/cloud/dataflow/examples/MaxPerKeyExamplesTest.java | 2 +- .../java/com/google/cloud/dataflow/examples/TfIdfTest.java | 2 +- .../cloud/dataflow/examples/TopWikipediaSessionsTest.java | 2 +- .../com/google/cloud/dataflow/examples/WordCountTest.java | 2 +- pom.xml | 2 +- sdk/pom.xml | 2 +- sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java | 2 +- .../java/com/google/cloud/dataflow/sdk/PipelineResult.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/AtomicCoder.java | 2 +- .../java/com/google/cloud/dataflow/sdk/coders/AvroCoder.java | 2 +- .../cloud/dataflow/sdk/coders/BigEndianIntegerCoder.java | 2 +- .../google/cloud/dataflow/sdk/coders/BigEndianLongCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/coders/Coder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/CoderException.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/CoderRegistry.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/CollectionCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/CustomCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/DefaultCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/DelegateCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/DoubleCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/EntityCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/InstantCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/IterableCoder.java | 2 +- .../google/cloud/dataflow/sdk/coders/IterableLikeCoder.java | 2 +- .../java/com/google/cloud/dataflow/sdk/coders/KvCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/KvCoderBase.java | 2 +- .../java/com/google/cloud/dataflow/sdk/coders/ListCoder.java | 2 +- .../java/com/google/cloud/dataflow/sdk/coders/MapCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/MapCoderBase.java | 2 +- .../google/cloud/dataflow/sdk/coders/SerializableCoder.java | 2 +- .../java/com/google/cloud/dataflow/sdk/coders/SetCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/StandardCoder.java | 2 +- .../google/cloud/dataflow/sdk/coders/StringDelegateCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/StringUtf8Coder.java | 2 +- .../google/cloud/dataflow/sdk/coders/TableRowJsonCoder.java | 2 +- .../google/cloud/dataflow/sdk/coders/TextualIntegerCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/VarIntCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/VarLongCoder.java | 2 +- .../java/com/google/cloud/dataflow/sdk/coders/VoidCoder.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/package-info.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java | 2 +- .../java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java | 2 +- .../java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java | 2 +- .../java/com/google/cloud/dataflow/sdk/io/ReadSource.java | 2 +- .../com/google/cloud/dataflow/sdk/io/ShardNameTemplate.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/io/Source.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/io/TextIO.java | 2 +- .../java/com/google/cloud/dataflow/sdk/io/package-info.java | 2 +- .../cloud/dataflow/sdk/options/ApplicationNameOptions.java | 2 +- .../google/cloud/dataflow/sdk/options/BigQueryOptions.java | 2 +- .../dataflow/sdk/options/BlockingDataflowPipelineOptions.java | 2 +- .../cloud/dataflow/sdk/options/CloudDebuggerOptions.java | 2 +- .../dataflow/sdk/options/DataflowPipelineDebugOptions.java | 2 +- .../cloud/dataflow/sdk/options/DataflowPipelineOptions.java | 2 +- .../sdk/options/DataflowPipelineWorkerPoolOptions.java | 2 +- .../dataflow/sdk/options/DataflowWorkerHarnessOptions.java | 2 +- .../java/com/google/cloud/dataflow/sdk/options/Default.java | 2 +- .../cloud/dataflow/sdk/options/DefaultValueFactory.java | 2 +- .../com/google/cloud/dataflow/sdk/options/Description.java | 2 +- .../cloud/dataflow/sdk/options/DirectPipelineOptions.java | 2 +- .../com/google/cloud/dataflow/sdk/options/GcpOptions.java | 2 +- .../com/google/cloud/dataflow/sdk/options/GcsOptions.java | 2 +- .../cloud/dataflow/sdk/options/GoogleApiDebugOptions.java | 2 +- .../google/cloud/dataflow/sdk/options/PipelineOptions.java | 2 +- .../cloud/dataflow/sdk/options/PipelineOptionsFactory.java | 2 +- .../cloud/dataflow/sdk/options/PipelineOptionsRegistrar.java | 2 +- .../cloud/dataflow/sdk/options/PipelineOptionsValidator.java | 2 +- .../cloud/dataflow/sdk/options/ProxyInvocationHandler.java | 2 +- .../google/cloud/dataflow/sdk/options/StreamingOptions.java | 2 +- .../com/google/cloud/dataflow/sdk/options/Validation.java | 2 +- .../com/google/cloud/dataflow/sdk/options/package-info.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/package-info.java | 2 +- .../dataflow/sdk/runners/BlockingDataflowPipelineRunner.java | 2 +- .../google/cloud/dataflow/sdk/runners/DataflowPipeline.java | 2 +- .../cloud/dataflow/sdk/runners/DataflowPipelineJob.java | 2 +- .../cloud/dataflow/sdk/runners/DataflowPipelineRegistrar.java | 2 +- .../cloud/dataflow/sdk/runners/DataflowPipelineRunner.java | 2 +- .../dataflow/sdk/runners/DataflowPipelineRunnerHooks.java | 2 +- .../dataflow/sdk/runners/DataflowPipelineTranslator.java | 2 +- .../com/google/cloud/dataflow/sdk/runners/DirectPipeline.java | 2 +- .../cloud/dataflow/sdk/runners/DirectPipelineRegistrar.java | 2 +- .../cloud/dataflow/sdk/runners/DirectPipelineRunner.java | 2 +- .../com/google/cloud/dataflow/sdk/runners/PipelineRunner.java | 2 +- .../cloud/dataflow/sdk/runners/PipelineRunnerRegistrar.java | 2 +- .../cloud/dataflow/sdk/runners/RecordingPipelineVisitor.java | 2 +- .../google/cloud/dataflow/sdk/runners/TransformHierarchy.java | 2 +- .../google/cloud/dataflow/sdk/runners/TransformTreeNode.java | 2 +- .../cloud/dataflow/sdk/runners/dataflow/AvroIOTranslator.java | 2 +- .../sdk/runners/dataflow/BasicSerializableSourceFormat.java | 2 +- .../dataflow/sdk/runners/dataflow/BigQueryIOTranslator.java | 2 +- .../dataflow/sdk/runners/dataflow/DatastoreIOTranslator.java | 2 +- .../dataflow/sdk/runners/dataflow/PubsubIOTranslator.java | 2 +- .../dataflow/sdk/runners/dataflow/ReadSourceTranslator.java | 2 +- .../cloud/dataflow/sdk/runners/dataflow/TextIOTranslator.java | 2 +- .../cloud/dataflow/sdk/runners/dataflow/package-info.java | 2 +- .../com/google/cloud/dataflow/sdk/runners/package-info.java | 2 +- .../dataflow/sdk/runners/worker/ApplianceShuffleReader.java | 2 +- .../dataflow/sdk/runners/worker/ApplianceShuffleWriter.java | 2 +- .../dataflow/sdk/runners/worker/AssignWindowsParDoFn.java | 2 +- .../cloud/dataflow/sdk/runners/worker/AvroByteReader.java | 2 +- .../cloud/dataflow/sdk/runners/worker/AvroByteSink.java | 2 +- .../google/cloud/dataflow/sdk/runners/worker/AvroReader.java | 2 +- .../cloud/dataflow/sdk/runners/worker/AvroReaderFactory.java | 2 +- .../google/cloud/dataflow/sdk/runners/worker/AvroSink.java | 2 +- .../cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java | 2 +- .../cloud/dataflow/sdk/runners/worker/BigQueryReader.java | 2 +- .../dataflow/sdk/runners/worker/BigQueryReaderFactory.java | 2 +- .../dataflow/sdk/runners/worker/ByteArrayShufflePosition.java | 2 +- .../sdk/runners/worker/ChunkingShuffleBatchReader.java | 2 +- .../sdk/runners/worker/ChunkingShuffleEntryWriter.java | 2 +- .../cloud/dataflow/sdk/runners/worker/CombineValuesFn.java | 2 +- .../sdk/runners/worker/CopyableSeekableByteChannel.java | 2 +- .../sdk/runners/worker/DataflowWorkProgressUpdater.java | 2 +- .../cloud/dataflow/sdk/runners/worker/DataflowWorker.java | 2 +- .../dataflow/sdk/runners/worker/DataflowWorkerHarness.java | 2 +- .../cloud/dataflow/sdk/runners/worker/FileBasedReader.java | 2 +- .../sdk/runners/worker/GroupAlsoByWindowsParDoFn.java | 2 +- .../dataflow/sdk/runners/worker/GroupingShuffleReader.java | 2 +- .../sdk/runners/worker/GroupingShuffleReaderFactory.java | 2 +- .../cloud/dataflow/sdk/runners/worker/InMemoryReader.java | 2 +- .../dataflow/sdk/runners/worker/InMemoryReaderFactory.java | 2 +- .../dataflow/sdk/runners/worker/LazyMultiReaderIterator.java | 2 +- .../dataflow/sdk/runners/worker/MapTaskExecutorFactory.java | 2 +- .../cloud/dataflow/sdk/runners/worker/NormalParDoFn.java | 2 +- .../google/cloud/dataflow/sdk/runners/worker/OrderedCode.java | 2 +- .../cloud/dataflow/sdk/runners/worker/ParDoFnFactory.java | 2 +- .../sdk/runners/worker/PartitioningShuffleReader.java | 2 +- .../sdk/runners/worker/PartitioningShuffleReaderFactory.java | 2 +- .../cloud/dataflow/sdk/runners/worker/PubsubReader.java | 2 +- .../google/cloud/dataflow/sdk/runners/worker/PubsubSink.java | 2 +- .../cloud/dataflow/sdk/runners/worker/ReaderFactory.java | 2 +- .../cloud/dataflow/sdk/runners/worker/ShuffleEntryWriter.java | 2 +- .../cloud/dataflow/sdk/runners/worker/ShuffleLibrary.java | 2 +- .../cloud/dataflow/sdk/runners/worker/ShuffleReader.java | 2 +- .../google/cloud/dataflow/sdk/runners/worker/ShuffleSink.java | 2 +- .../cloud/dataflow/sdk/runners/worker/ShuffleSinkFactory.java | 2 +- .../cloud/dataflow/sdk/runners/worker/ShuffleWriter.java | 2 +- .../cloud/dataflow/sdk/runners/worker/SideInputUtils.java | 2 +- .../google/cloud/dataflow/sdk/runners/worker/SinkFactory.java | 2 +- .../dataflow/sdk/runners/worker/SourceFormatFactory.java | 2 +- .../dataflow/sdk/runners/worker/SourceOperationExecutor.java | 2 +- .../sdk/runners/worker/SourceOperationExecutorFactory.java | 2 +- .../dataflow/sdk/runners/worker/SourceTranslationUtils.java | 2 +- .../dataflow/sdk/runners/worker/StreamingDataflowWorker.java | 2 +- .../google/cloud/dataflow/sdk/runners/worker/TextReader.java | 2 +- .../cloud/dataflow/sdk/runners/worker/TextReaderFactory.java | 2 +- .../google/cloud/dataflow/sdk/runners/worker/TextSink.java | 2 +- .../cloud/dataflow/sdk/runners/worker/TextSinkFactory.java | 2 +- .../dataflow/sdk/runners/worker/UngroupedShuffleReader.java | 2 +- .../sdk/runners/worker/UngroupedShuffleReaderFactory.java | 2 +- .../dataflow/sdk/runners/worker/UngroupedWindmillReader.java | 2 +- .../cloud/dataflow/sdk/runners/worker/WindmillSink.java | 2 +- .../dataflow/sdk/runners/worker/WindowingWindmillReader.java | 2 +- .../worker/logging/DataflowWorkerLoggingFormatter.java | 2 +- .../worker/logging/DataflowWorkerLoggingInitializer.java | 2 +- .../cloud/dataflow/sdk/runners/worker/package-info.java | 2 +- .../sdk/runners/worker/windmill/WindmillServerStub.java | 2 +- .../com/google/cloud/dataflow/sdk/testing/DataflowAssert.java | 2 +- .../google/cloud/dataflow/sdk/testing/RunnableOnService.java | 2 +- .../dataflow/sdk/testing/TestDataflowPipelineOptions.java | 2 +- .../dataflow/sdk/testing/TestDataflowPipelineRunner.java | 2 +- .../com/google/cloud/dataflow/sdk/testing/TestPipeline.java | 2 +- .../google/cloud/dataflow/sdk/testing/WindowFnTestUtils.java | 2 +- .../com/google/cloud/dataflow/sdk/testing/package-info.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/Aggregator.java | 2 +- .../cloud/dataflow/sdk/transforms/ApproximateQuantiles.java | 2 +- .../cloud/dataflow/sdk/transforms/ApproximateUnique.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/Combine.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/Count.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/Create.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/DoFn.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/DoFnTester.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/Filter.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/First.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/Flatten.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/GroupByKey.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/Keys.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/KvSwap.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/Max.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/Mean.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/Min.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/PTransform.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/ParDo.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/Partition.java | 2 +- .../google/cloud/dataflow/sdk/transforms/RateLimiting.java | 2 +- .../cloud/dataflow/sdk/transforms/RemoveDuplicates.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/Sample.java | 2 +- .../cloud/dataflow/sdk/transforms/SerializableComparator.java | 2 +- .../cloud/dataflow/sdk/transforms/SerializableFunction.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/Sum.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/Top.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/Values.java | 2 +- .../java/com/google/cloud/dataflow/sdk/transforms/View.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/WithKeys.java | 2 +- .../cloud/dataflow/sdk/transforms/join/CoGbkResult.java | 2 +- .../cloud/dataflow/sdk/transforms/join/CoGbkResultSchema.java | 2 +- .../cloud/dataflow/sdk/transforms/join/CoGroupByKey.java | 2 +- .../dataflow/sdk/transforms/join/KeyedPCollectionTuple.java | 2 +- .../cloud/dataflow/sdk/transforms/join/RawUnionValue.java | 2 +- .../google/cloud/dataflow/sdk/transforms/join/UnionCoder.java | 2 +- .../cloud/dataflow/sdk/transforms/join/package-info.java | 2 +- .../google/cloud/dataflow/sdk/transforms/package-info.java | 2 +- .../dataflow/sdk/transforms/windowing/BoundedWindow.java | 2 +- .../dataflow/sdk/transforms/windowing/CalendarWindows.java | 2 +- .../cloud/dataflow/sdk/transforms/windowing/FixedWindows.java | 2 +- .../cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java | 2 +- .../dataflow/sdk/transforms/windowing/GlobalWindows.java | 2 +- .../dataflow/sdk/transforms/windowing/IntervalWindow.java | 2 +- .../dataflow/sdk/transforms/windowing/InvalidWindows.java | 2 +- .../transforms/windowing/MergeOverlappingIntervalWindows.java | 2 +- .../dataflow/sdk/transforms/windowing/NonMergingWindowFn.java | 2 +- .../sdk/transforms/windowing/PartitioningWindowFn.java | 2 +- .../cloud/dataflow/sdk/transforms/windowing/Sessions.java | 2 +- .../dataflow/sdk/transforms/windowing/SlidingWindows.java | 2 +- .../cloud/dataflow/sdk/transforms/windowing/Window.java | 2 +- .../cloud/dataflow/sdk/transforms/windowing/WindowFn.java | 2 +- .../cloud/dataflow/sdk/transforms/windowing/package-info.java | 2 +- .../com/google/cloud/dataflow/sdk/util/AbstractWindowSet.java | 2 +- .../com/google/cloud/dataflow/sdk/util/AggregatorImpl.java | 2 +- .../com/google/cloud/dataflow/sdk/util/ApiErrorExtractor.java | 2 +- .../google/cloud/dataflow/sdk/util/AppEngineEnvironment.java | 2 +- .../com/google/cloud/dataflow/sdk/util/AssignWindowsDoFn.java | 2 +- .../dataflow/sdk/util/AttemptBoundedExponentialBackOff.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/Base64Utils.java | 2 +- .../cloud/dataflow/sdk/util/BatchModeExecutionContext.java | 2 +- .../google/cloud/dataflow/sdk/util/BigQueryTableInserter.java | 2 +- .../cloud/dataflow/sdk/util/BigQueryTableRowIterator.java | 2 +- .../google/cloud/dataflow/sdk/util/BoundedQueueExecutor.java | 2 +- .../google/cloud/dataflow/sdk/util/BufferingWindowSet.java | 2 +- .../com/google/cloud/dataflow/sdk/util/CloudCounterUtils.java | 2 +- .../com/google/cloud/dataflow/sdk/util/CloudKnownType.java | 2 +- .../com/google/cloud/dataflow/sdk/util/CloudMetricUtils.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/CloudObject.java | 2 +- .../com/google/cloud/dataflow/sdk/util/CloudSourceUtils.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/CoderUtils.java | 2 +- .../com/google/cloud/dataflow/sdk/util/CredentialFactory.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/Credentials.java | 2 +- .../google/cloud/dataflow/sdk/util/DataflowPathValidator.java | 2 +- .../google/cloud/dataflow/sdk/util/DataflowReleaseInfo.java | 2 +- .../cloud/dataflow/sdk/util/DirectModeExecutionContext.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/DoFnContext.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/DoFnInfo.java | 2 +- .../google/cloud/dataflow/sdk/util/DoFnProcessContext.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java | 2 +- .../com/google/cloud/dataflow/sdk/util/ExecutionContext.java | 2 +- .../google/cloud/dataflow/sdk/util/FileIOChannelFactory.java | 2 +- .../com/google/cloud/dataflow/sdk/util/GCloudCredential.java | 2 +- .../google/cloud/dataflow/sdk/util/GcpCredentialFactory.java | 2 +- .../google/cloud/dataflow/sdk/util/GcsIOChannelFactory.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/GcsStager.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/util/GcsUtil.java | 2 +- .../cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java | 2 +- .../com/google/cloud/dataflow/sdk/util/IOChannelFactory.java | 2 +- .../com/google/cloud/dataflow/sdk/util/IOChannelUtils.java | 2 +- .../com/google/cloud/dataflow/sdk/util/InstanceBuilder.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/MimeTypes.java | 2 +- .../com/google/cloud/dataflow/sdk/util/MonitoringUtil.java | 2 +- .../google/cloud/dataflow/sdk/util/NoopCredentialFactory.java | 2 +- .../com/google/cloud/dataflow/sdk/util/NoopPathValidator.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/NoopStager.java | 2 +- .../com/google/cloud/dataflow/sdk/util/OutputReference.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/util/PTuple.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/PackageUtil.java | 2 +- .../cloud/dataflow/sdk/util/PartitionBufferingWindowSet.java | 2 +- .../com/google/cloud/dataflow/sdk/util/PathValidator.java | 2 +- .../com/google/cloud/dataflow/sdk/util/PropertyNames.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/ReaderUtils.java | 2 +- .../cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java | 2 +- .../com/google/cloud/dataflow/sdk/util/SerializableUtils.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/Serializer.java | 2 +- .../cloud/dataflow/sdk/util/ShardingWritableByteChannel.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/util/Stager.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/StateFetcher.java | 2 +- .../dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java | 2 +- .../dataflow/sdk/util/StreamingModeExecutionContext.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/StringUtils.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/util/Structs.java | 2 +- .../com/google/cloud/dataflow/sdk/util/TestCredential.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/TimeUtil.java | 2 +- .../com/google/cloud/dataflow/sdk/util/TimerOrElement.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/Transport.java | 2 +- .../com/google/cloud/dataflow/sdk/util/UserCodeException.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/util/Values.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/util/VarInt.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/WindowUtils.java | 2 +- .../com/google/cloud/dataflow/sdk/util/WindowedValue.java | 2 +- .../com/google/cloud/dataflow/sdk/util/common/Counter.java | 2 +- .../com/google/cloud/dataflow/sdk/util/common/CounterSet.java | 2 +- .../dataflow/sdk/util/common/ElementByteSizeObservable.java | 2 +- .../sdk/util/common/ElementByteSizeObservableIterable.java | 2 +- .../sdk/util/common/ElementByteSizeObservableIterator.java | 2 +- .../dataflow/sdk/util/common/ElementByteSizeObserver.java | 2 +- .../cloud/dataflow/sdk/util/common/ForwardingReiterator.java | 2 +- .../com/google/cloud/dataflow/sdk/util/common/Metric.java | 2 +- .../cloud/dataflow/sdk/util/common/PeekingReiterator.java | 2 +- .../com/google/cloud/dataflow/sdk/util/common/Reiterable.java | 2 +- .../com/google/cloud/dataflow/sdk/util/common/Reiterator.java | 2 +- .../google/cloud/dataflow/sdk/util/common/package-info.java | 2 +- .../sdk/util/common/worker/BatchingShuffleEntryReader.java | 2 +- .../sdk/util/common/worker/CachingShuffleBatchReader.java | 2 +- .../dataflow/sdk/util/common/worker/FlattenOperation.java | 2 +- .../sdk/util/common/worker/GroupingShuffleEntryIterator.java | 2 +- .../sdk/util/common/worker/KeyGroupedShuffleEntries.java | 2 +- .../dataflow/sdk/util/common/worker/MapTaskExecutor.java | 2 +- .../cloud/dataflow/sdk/util/common/worker/Operation.java | 2 +- .../cloud/dataflow/sdk/util/common/worker/OutputReceiver.java | 2 +- .../google/cloud/dataflow/sdk/util/common/worker/ParDoFn.java | 2 +- .../cloud/dataflow/sdk/util/common/worker/ParDoOperation.java | 2 +- .../sdk/util/common/worker/PartialGroupByKeyOperation.java | 2 +- .../dataflow/sdk/util/common/worker/ProgressTracker.java | 2 +- .../dataflow/sdk/util/common/worker/ProgressTrackerGroup.java | 2 +- .../sdk/util/common/worker/ProgressTrackingReiterator.java | 2 +- .../cloud/dataflow/sdk/util/common/worker/ReadOperation.java | 2 +- .../google/cloud/dataflow/sdk/util/common/worker/Reader.java | 2 +- .../cloud/dataflow/sdk/util/common/worker/Receiver.java | 2 +- .../dataflow/sdk/util/common/worker/ReceivingOperation.java | 2 +- .../dataflow/sdk/util/common/worker/ShuffleBatchReader.java | 2 +- .../cloud/dataflow/sdk/util/common/worker/ShuffleEntry.java | 2 +- .../dataflow/sdk/util/common/worker/ShuffleEntryReader.java | 2 +- .../dataflow/sdk/util/common/worker/ShufflePosition.java | 2 +- .../google/cloud/dataflow/sdk/util/common/worker/Sink.java | 2 +- .../cloud/dataflow/sdk/util/common/worker/SourceFormat.java | 2 +- .../cloud/dataflow/sdk/util/common/worker/StateSampler.java | 2 +- .../cloud/dataflow/sdk/util/common/worker/WorkExecutor.java | 2 +- .../dataflow/sdk/util/common/worker/WorkProgressUpdater.java | 2 +- .../cloud/dataflow/sdk/util/common/worker/WriteOperation.java | 2 +- .../cloud/dataflow/sdk/util/common/worker/package-info.java | 2 +- .../com/google/cloud/dataflow/sdk/util/gcsfs/GcsPath.java | 2 +- .../google/cloud/dataflow/sdk/util/gcsfs/package-info.java | 2 +- .../cloud/dataflow/sdk/util/gcsio/ClientRequestHelper.java | 2 +- .../dataflow/sdk/util/gcsio/GoogleCloudStorageExceptions.java | 2 +- .../sdk/util/gcsio/GoogleCloudStorageReadChannel.java | 2 +- .../sdk/util/gcsio/GoogleCloudStorageWriteChannel.java | 2 +- .../util/gcsio/LoggingMediaHttpUploaderProgressListener.java | 2 +- .../cloud/dataflow/sdk/util/gcsio/StorageResourceId.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/package-info.java | 2 +- .../com/google/cloud/dataflow/sdk/values/CodedTupleTag.java | 2 +- .../google/cloud/dataflow/sdk/values/CodedTupleTagMap.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/values/KV.java | 2 +- .../java/com/google/cloud/dataflow/sdk/values/PBegin.java | 2 +- .../com/google/cloud/dataflow/sdk/values/PCollection.java | 2 +- .../com/google/cloud/dataflow/sdk/values/PCollectionList.java | 2 +- .../google/cloud/dataflow/sdk/values/PCollectionTuple.java | 2 +- .../com/google/cloud/dataflow/sdk/values/PCollectionView.java | 2 +- .../main/java/com/google/cloud/dataflow/sdk/values/PDone.java | 2 +- .../java/com/google/cloud/dataflow/sdk/values/PInput.java | 2 +- .../java/com/google/cloud/dataflow/sdk/values/POutput.java | 2 +- .../google/cloud/dataflow/sdk/values/POutputValueBase.java | 2 +- .../java/com/google/cloud/dataflow/sdk/values/PValue.java | 2 +- .../java/com/google/cloud/dataflow/sdk/values/PValueBase.java | 2 +- .../google/cloud/dataflow/sdk/values/TimestampedValue.java | 2 +- .../java/com/google/cloud/dataflow/sdk/values/TupleTag.java | 2 +- .../com/google/cloud/dataflow/sdk/values/TupleTagList.java | 2 +- .../com/google/cloud/dataflow/sdk/values/TypedPValue.java | 2 +- .../com/google/cloud/dataflow/sdk/values/package-info.java | 2 +- sdk/src/main/proto/windmill.proto | 2 +- .../test/java/com/google/cloud/dataflow/sdk/PipelineTest.java | 2 +- .../test/java/com/google/cloud/dataflow/sdk/TestUtils.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java | 2 +- .../cloud/dataflow/sdk/coders/BigEndianIntegerCoderTest.java | 2 +- .../cloud/dataflow/sdk/coders/BigEndianLongCoderTest.java | 2 +- .../google/cloud/dataflow/sdk/coders/ByteArrayCoderTest.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/CoderProperties.java | 2 +- .../google/cloud/dataflow/sdk/coders/CoderRegistryTest.java | 2 +- .../google/cloud/dataflow/sdk/coders/CollectionCoderTest.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/CustomCoderTest.java | 2 +- .../google/cloud/dataflow/sdk/coders/DefaultCoderTest.java | 2 +- .../google/cloud/dataflow/sdk/coders/DelegateCoderTest.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/EntityCoderTest.java | 2 +- .../google/cloud/dataflow/sdk/coders/InstantCoderTest.java | 2 +- .../google/cloud/dataflow/sdk/coders/IterableCoderTest.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/KvCoderTest.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/ListCoderTest.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/MapCoderTest.java | 2 +- .../cloud/dataflow/sdk/coders/SerializableCoderTest.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/SetCoderTest.java | 2 +- .../cloud/dataflow/sdk/coders/StringDelegateCoderTest.java | 2 +- .../google/cloud/dataflow/sdk/coders/StringUtf8CoderTest.java | 2 +- .../cloud/dataflow/sdk/coders/TableRowJsonCoderTest.java | 2 +- .../cloud/dataflow/sdk/coders/TextualIntegerCoderTest.java | 2 +- .../com/google/cloud/dataflow/sdk/coders/VarIntCoderTest.java | 2 +- .../google/cloud/dataflow/sdk/coders/VarLongCoderTest.java | 2 +- .../java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java | 2 +- .../java/com/google/cloud/dataflow/sdk/io/BigQueryIOTest.java | 2 +- .../com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java | 2 +- .../java/com/google/cloud/dataflow/sdk/io/TextIOTest.java | 2 +- .../dataflow/sdk/options/DataflowPipelineOptionsTest.java | 2 +- .../cloud/dataflow/sdk/options/GoogleApiDebugOptionsTest.java | 2 +- .../dataflow/sdk/options/PipelineOptionsFactoryTest.java | 2 +- .../cloud/dataflow/sdk/options/PipelineOptionsTest.java | 2 +- .../dataflow/sdk/options/PipelineOptionsValidatorTest.java | 2 +- .../dataflow/sdk/options/ProxyInvocationHandlerTest.java | 2 +- .../sdk/runners/BlockingDataflowPipelineRunnerTest.java | 2 +- .../cloud/dataflow/sdk/runners/DataflowPipelineJobTest.java | 2 +- .../dataflow/sdk/runners/DataflowPipelineRegistrarTest.java | 2 +- .../dataflow/sdk/runners/DataflowPipelineRunnerTest.java | 2 +- .../dataflow/sdk/runners/DataflowPipelineTranslatorTest.java | 2 +- .../dataflow/sdk/runners/DirectPipelineRegistrarTest.java | 2 +- .../google/cloud/dataflow/sdk/runners/PipelineRunnerTest.java | 2 +- .../google/cloud/dataflow/sdk/runners/TransformTreeTest.java | 2 +- .../runners/dataflow/BasicSerializableSourceFormatTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/AvroByteReaderTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/AvroByteSinkTest.java | 2 +- .../dataflow/sdk/runners/worker/AvroReaderFactoryTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/AvroReaderTest.java | 2 +- .../dataflow/sdk/runners/worker/AvroSinkFactoryTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/AvroSinkTest.java | 2 +- .../sdk/runners/worker/BigQueryReaderFactoryTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java | 2 +- .../dataflow/sdk/runners/worker/CombineValuesFnTest.java | 2 +- .../sdk/runners/worker/CopyableSeekableByteChannelTest.java | 2 +- .../sdk/runners/worker/DataflowWorkProgressUpdaterTest.java | 2 +- .../sdk/runners/worker/DataflowWorkerHarnessTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java | 2 +- .../dataflow/sdk/runners/worker/DatastoreReaderTest.java | 2 +- .../sdk/runners/worker/GroupingShuffleReaderTest.java | 2 +- .../sdk/runners/worker/InMemoryReaderFactoryTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java | 2 +- .../sdk/runners/worker/MapTaskExecutorFactoryTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/NormalParDoFnTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/OrderedCodeTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/ParDoFnFactoryTest.java | 2 +- .../sdk/runners/worker/PartitioningShuffleReaderTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/ReaderFactoryTest.java | 2 +- .../dataflow/sdk/runners/worker/ShuffleReaderFactoryTest.java | 2 +- .../dataflow/sdk/runners/worker/ShuffleSinkFactoryTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/ShuffleSinkTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/SideInputUtilsTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/SinkFactoryTest.java | 2 +- .../sdk/runners/worker/StreamingDataflowWorkerTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/TestShuffleReader.java | 2 +- .../dataflow/sdk/runners/worker/TestShuffleReaderTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/TestShuffleWriter.java | 2 +- .../dataflow/sdk/runners/worker/TextReaderFactoryTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/TextReaderTest.java | 2 +- .../dataflow/sdk/runners/worker/TextSinkFactoryTest.java | 2 +- .../cloud/dataflow/sdk/runners/worker/TextSinkTest.java | 2 +- .../sdk/runners/worker/UngroupedShuffleReaderTest.java | 2 +- .../worker/logging/DataflowWorkerLoggingFormatterTest.java | 2 +- .../worker/logging/DataflowWorkerLoggingInitializerTest.java | 2 +- .../google/cloud/dataflow/sdk/testing/DataflowAssertTest.java | 2 +- .../com/google/cloud/dataflow/sdk/testing/ExpectedLogs.java | 2 +- .../google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java | 2 +- .../cloud/dataflow/sdk/testing/FastNanoClockAndSleeper.java | 2 +- .../dataflow/sdk/testing/FastNanoClockAndSleeperTest.java | 2 +- .../cloud/dataflow/sdk/testing/ResetDateTimeProvider.java | 2 +- .../cloud/dataflow/sdk/testing/ResetDateTimeProviderTest.java | 2 +- .../dataflow/sdk/testing/RestoreDataflowLoggingFormatter.java | 2 +- .../sdk/testing/RestoreDataflowLoggingFormatterTest.java | 2 +- .../cloud/dataflow/sdk/testing/RestoreSystemProperties.java | 2 +- .../dataflow/sdk/testing/RestoreSystemPropertiesTest.java | 2 +- .../google/cloud/dataflow/sdk/testing/TestPipelineTest.java | 2 +- .../dataflow/sdk/transforms/ApproximateQuantilesTest.java | 2 +- .../cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/CombineTest.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/CountTest.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/CreateTest.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/FilterTest.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/FirstTest.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/FlattenTest.java | 2 +- .../google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/KeysTest.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/KvSwapTest.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/ParDoTest.java | 2 +- .../google/cloud/dataflow/sdk/transforms/PartitionTest.java | 2 +- .../cloud/dataflow/sdk/transforms/RateLimitingTest.java | 2 +- .../cloud/dataflow/sdk/transforms/RemoveDuplicatesTest.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/SampleTest.java | 2 +- .../cloud/dataflow/sdk/transforms/SimpleStatsFnsTest.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/TopTest.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/ValuesTest.java | 2 +- .../com/google/cloud/dataflow/sdk/transforms/ViewTest.java | 2 +- .../google/cloud/dataflow/sdk/transforms/WithKeysTest.java | 2 +- .../dataflow/sdk/transforms/join/CoGbkResultCoderTest.java | 2 +- .../cloud/dataflow/sdk/transforms/join/CoGroupByKeyTest.java | 2 +- .../cloud/dataflow/sdk/transforms/join/UnionCoderTest.java | 2 +- .../sdk/transforms/windowing/CalendarWindowsTest.java | 2 +- .../dataflow/sdk/transforms/windowing/FixedWindowsTest.java | 2 +- .../cloud/dataflow/sdk/transforms/windowing/SessionsTest.java | 2 +- .../dataflow/sdk/transforms/windowing/SlidingWindowsTest.java | 2 +- .../dataflow/sdk/transforms/windowing/WindowingTest.java | 2 +- .../google/cloud/dataflow/sdk/util/AggregatorImplTest.java | 2 +- .../sdk/util/AttemptBoundedExponentialBackOffTest.java | 2 +- .../com/google/cloud/dataflow/sdk/util/Base64UtilsTest.java | 2 +- .../com/google/cloud/dataflow/sdk/util/BigQueryUtilTest.java | 2 +- .../google/cloud/dataflow/sdk/util/CloudMetricUtilsTest.java | 2 +- .../google/cloud/dataflow/sdk/util/CloudSourceUtilsTest.java | 2 +- .../com/google/cloud/dataflow/sdk/util/CoderUtilsTest.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/GcsUtilTest.java | 2 +- .../cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java | 2 +- .../google/cloud/dataflow/sdk/util/IOChannelUtilsTest.java | 2 +- .../com/google/cloud/dataflow/sdk/util/IOFactoryTest.java | 2 +- .../google/cloud/dataflow/sdk/util/InstanceBuilderTest.java | 2 +- .../google/cloud/dataflow/sdk/util/MonitoringUtilTest.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/PTupleTest.java | 2 +- .../com/google/cloud/dataflow/sdk/util/PackageUtilTest.java | 2 +- .../dataflow/sdk/util/RetryHttpRequestInitializerTest.java | 2 +- .../google/cloud/dataflow/sdk/util/SerializableUtilsTest.java | 2 +- .../com/google/cloud/dataflow/sdk/util/SerializerTest.java | 2 +- .../sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java | 2 +- .../com/google/cloud/dataflow/sdk/util/StringUtilsTest.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/StructsTest.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/TimeUtilTest.java | 2 +- .../java/com/google/cloud/dataflow/sdk/util/VarIntTest.java | 2 +- .../com/google/cloud/dataflow/sdk/util/WindowedValueTest.java | 2 +- .../google/cloud/dataflow/sdk/util/common/CounterSetTest.java | 2 +- .../google/cloud/dataflow/sdk/util/common/CounterTest.java | 2 +- .../cloud/dataflow/sdk/util/common/CounterTestUtils.java | 2 +- .../com/google/cloud/dataflow/sdk/util/common/MetricTest.java | 2 +- .../dataflow/sdk/util/common/TaggedReiteratorListTest.java | 2 +- .../util/common/worker/BatchingShuffleEntryReaderTest.java | 2 +- .../sdk/util/common/worker/CachingShuffleBatchReaderTest.java | 2 +- .../dataflow/sdk/util/common/worker/ExecutorTestUtils.java | 2 +- .../dataflow/sdk/util/common/worker/FlattenOperationTest.java | 2 +- .../dataflow/sdk/util/common/worker/MapTaskExecutorTest.java | 2 +- .../dataflow/sdk/util/common/worker/OutputReceiverTest.java | 2 +- .../dataflow/sdk/util/common/worker/ParDoOperationTest.java | 2 +- .../util/common/worker/PartialGroupByKeyOperationTest.java | 2 +- .../dataflow/sdk/util/common/worker/ReadOperationTest.java | 2 +- .../dataflow/sdk/util/common/worker/ShuffleEntryTest.java | 2 +- .../dataflow/sdk/util/common/worker/StateSamplerTest.java | 2 +- .../dataflow/sdk/util/common/worker/WorkExecutorTest.java | 2 +- .../dataflow/sdk/util/common/worker/WriteOperationTest.java | 2 +- .../com/google/cloud/dataflow/sdk/util/gcsfs/GcsPathTest.java | 2 +- .../gcsio/LoggingMediaHttpUploaderProgressListenerTest.java | 2 +- .../java/com/google/cloud/dataflow/sdk/values/KVTest.java | 2 +- .../google/cloud/dataflow/sdk/values/PCollectionListTest.java | 2 +- .../java/com/google/cloud/dataflow/sdk/values/PDoneTest.java | 2 +- .../com/google/cloud/dataflow/sdk/values/TupleTagTest.java | 2 +- 551 files changed, 552 insertions(+), 552 deletions(-) diff --git a/checkstyle.xml b/checkstyle.xml index 062f1346d9583..3e79c3d48028c 100644 --- a/checkstyle.xml +++ b/checkstyle.xml @@ -1,6 +1,6 @@ notice, so that this required text appears on the second line:
             /*
    -         * Copyright 2008 Google Inc.
    +         * Copyright 2015 Google Inc.
              *
              * (details of open-source license...)
           
    diff --git a/examples/pom.xml b/examples/pom.xml index 81de0a61629b3..863bf76459e2d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -1,6 +1,6 @@ + -Xlint:-cast + -Xlint:-deprecation + -Xlint:-processing + -Xlint:-rawtypes + -Xlint:-try + -Xlint:-unchecked + -Xlint:-varargs + + + + true - true + + false From db05c447a439a40a542c0d08f9bd9241e9dfee60 Mon Sep 17 00:00:00 2001 From: lcwik Date: Tue, 3 Mar 2015 14:18:24 -0800 Subject: [PATCH 0210/1541] Do not transport options which control credential generation to the worker since the worker does not understand them. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87647672 --- .../com/google/cloud/dataflow/sdk/options/GcpOptions.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java index a51336ae8986a..a884d3ec7ebb4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java @@ -70,6 +70,7 @@ public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions { * This option controls which file to use when attempting to create the credentials using the * OAuth 2 webflow. */ + @JsonIgnore @Description("Path to a file containing Google API secret") String getSecretsFile(); void setSecretsFile(String value); @@ -81,6 +82,7 @@ public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions { * This option if specified, needs be combined with the * {@link GcpOptions#getServiceAccountName() serviceAccountName}. */ + @JsonIgnore @Description("Path to a file containing the P12 service credentials") String getServiceAccountKeyfile(); void setServiceAccountKeyfile(String value); @@ -92,10 +94,12 @@ public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions { * This option if specified, needs be combined with the * {@link GcpOptions#getServiceAccountKeyfile() serviceAccountKeyfile}. */ + @JsonIgnore @Description("Name of the service account for Google APIs") String getServiceAccountName(); void setServiceAccountName(String value); + @JsonIgnore @Description("The path to the gcloud binary. " + " Default is to search the system path.") String getGCloudPath(); From e63dd15137d8348aff6d7629163402de08876267 Mon Sep 17 00:00:00 2001 From: lcwik Date: Tue, 3 Mar 2015 17:29:33 -0800 Subject: [PATCH 0211/1541] Add additional documentation about worker logger configuration from the command line. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87666318 --- .../sdk/options/DataflowWorkerLoggingOptions.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java index e14b97e718d24..a38cd44ab7e02 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java @@ -47,10 +47,12 @@ public enum Level { * Later options with equivalent names override earlier options. *

    * See {@link WorkerLogLevelOverride} for more information on how to configure logging - * on a per {@link Class}, {@link Package}, or name basis. + * on a per {@link Class}, {@link Package}, or name basis. If used from the command line, + * the expected format is {@code Name#Level}, further details on + * {@link WorkerLogLevelOverride#create(String)}. */ WorkerLogLevelOverride[] getWorkerLogLevelOverrides(); - void setWorkerLogLevelOverrides(WorkerLogLevelOverride[] string); + void setWorkerLogLevelOverrides(WorkerLogLevelOverride[] workerLogLevelOverrides); /** * Defines a log level override for a specific class, package, or name. @@ -111,7 +113,11 @@ public static WorkerLogLevelOverride forName(String name, Level level) { } /** - * Expects a value of the form {@code Name#Level}. + * Expects a value of the form {@code Name#Level}. The {@code Name} generally + * represents the fully qualified Java {@link Class#getName() class name}, + * or fully qualified Java {@link Package#getName() package name}, or custom + * logger name. The {@code Level} represents the log level and must be one + * of {@link Level}. */ @JsonCreator public static WorkerLogLevelOverride create(String value) { From 95c496626e69e53d56a9f7af01ec70513f828d95 Mon Sep 17 00:00:00 2001 From: millsd Date: Tue, 3 Mar 2015 17:42:48 -0800 Subject: [PATCH 0212/1541] Use a custom UncaughtExceptionHandler in main in addition to on worker threads in the StreamingDataflowWorker [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87667368 --- .../sdk/runners/worker/DataflowWorkerHarness.java | 2 +- .../sdk/runners/worker/StreamingDataflowWorker.java | 13 +++---------- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java index 96bce707b6111..a078d754abb1f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java @@ -92,7 +92,7 @@ public void uncaughtException(Thread t, Throwable e) { * Fetches and processes work units from the Dataflow service. */ public static void main(String[] args) throws Exception { - Thread.currentThread().setUncaughtExceptionHandler(WorkerUncaughtExceptionHandler.INSTANCE); + Thread.setDefaultUncaughtExceptionHandler(WorkerUncaughtExceptionHandler.INSTANCE); DataflowWorkerLoggingInitializer.initialize(); DataflowWorkerHarnessOptions pipelineOptions = diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java index 612c5646c9ec1..05c7722aeb58b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -93,6 +93,9 @@ static MapTask parseMapTask(String input) throws IOException { } public static void main(String[] args) throws Exception { + Thread.setDefaultUncaughtExceptionHandler( + DataflowWorkerHarness.WorkerUncaughtExceptionHandler.INSTANCE); + DataflowWorkerLoggingInitializer.initialize(); DataflowWorkerHarnessOptions options = PipelineOptionsFactory.createFromSystemProperties(); @@ -153,19 +156,9 @@ public StreamingDataflowWorker( addComputation(mapTask); } this.threadFactory = new ThreadFactory() { - private final Thread.UncaughtExceptionHandler handler = - new Thread.UncaughtExceptionHandler() { - @Override - public void uncaughtException(Thread thread, Throwable e) { - LOG.error("Uncaught exception: ", e); - System.exit(1); - } - }; - @Override public Thread newThread(Runnable r) { Thread t = new Thread(r); - t.setUncaughtExceptionHandler(handler); t.setDaemon(true); return t; } From 7ddf5b12e4590a995da3be3cf600f708c14f8b7f Mon Sep 17 00:00:00 2001 From: millsd Date: Tue, 3 Mar 2015 17:57:24 -0800 Subject: [PATCH 0213/1541] Fix off-by-one bug in GroupAlsoByWindowsDoFn iterator logic. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87668345 --- .../sdk/util/GroupAlsoByWindowsDoFn.java | 2 +- .../sdk/util/GroupAlsoByWindowsDoFnTest.java | 43 +++++++++++++++++++ 2 files changed, 44 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java index bd90a7f85690b..86876bfcf5bdc 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java @@ -261,7 +261,7 @@ public void remove() { private void skipToValidElement() { while (iterator.hasNext()) { WindowedValue peek = iterator.peek(); - if (!peek.getTimestamp().isBefore(window.maxTimestamp())) { + if (peek.getTimestamp().isAfter(window.maxTimestamp())) { // We are past the end of this window, so there can't be any more // elements in this iterator. break; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java index 4e60e41df72a0..677dac3ff8b0e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java @@ -161,6 +161,49 @@ public class GroupAlsoByWindowsDoFnTest { Matchers.contains(window(10, 30))); } + @Test public void testDiscontiguousWindows() throws Exception { + DoFnRunner>>, + KV>, List> runner = + makeRunner(FixedWindows.of(Duration.millis(10))); + + runner.startBundle(); + + runner.processElement(WindowedValue.valueInEmptyWindows( + KV.of("k", (Iterable>) Arrays.asList( + WindowedValue.of( + "v1", + new Instant(1), + Arrays.asList(window(0, 5))), + WindowedValue.of( + "v2", + new Instant(4), + Arrays.asList(window(1, 5))), + WindowedValue.of( + "v3", + new Instant(4), + Arrays.asList(window(0, 5))))))); + + runner.finishBundle(); + + List>>> result = runner.getReceiver(outputTag); + + assertEquals(2, result.size()); + + WindowedValue>> item0 = result.get(0); + assertEquals("k", item0.getValue().getKey()); + assertThat(item0.getValue().getValue(), Matchers.containsInAnyOrder("v1", "v3")); + assertEquals(new Instant(4), item0.getTimestamp()); + assertThat(item0.getWindows(), + Matchers.contains(window(0, 5))); + + WindowedValue>> item1 = result.get(1); + assertEquals("k", item1.getValue().getKey()); + assertThat(item1.getValue().getValue(), Matchers.contains("v2")); + assertEquals(new Instant(4), item1.getTimestamp()); + assertThat(item1.getWindows(), + Matchers.contains(window(1, 5))); + } + @Test public void testSessions() throws Exception { DoFnRunner>>, KV>, List> runner = From 573b31e0819497a1b249dfbabd47823237254fc0 Mon Sep 17 00:00:00 2001 From: zhouyunqing Date: Wed, 4 Mar 2015 01:46:37 -0800 Subject: [PATCH 0214/1541] Change the example in the comment to use non-deprecated pCollections instead of deprecated Flatten.create(). [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87691484 --- .../java/com/google/cloud/dataflow/sdk/transforms/Flatten.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java index d0cf77a5a2bdc..c2a10d375b0d3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java @@ -40,7 +40,7 @@ * PCollection pc2 = ...; * PCollection pc3 = ...; * PCollectionList pcs = PCollectionList.of(pc1).and(pc2).and(pc3); - * PCollection merged = pcs.apply(Flatten..create()); + * PCollection merged = pcs.apply(Flatten..pCollections()); * } * *

    By default, the {@code Coder} of the output {@code PCollection} From a4c45171e69e061f30fe18fd80e78caaaf91523a Mon Sep 17 00:00:00 2001 From: lcwik Date: Wed, 4 Mar 2015 10:20:10 -0800 Subject: [PATCH 0215/1541] Simplify the worker logging configuration for overrides. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87721993 --- .../logging/DataflowWorkerLoggingInitializer.java | 7 +------ .../logging/DataflowWorkerLoggingInitializerTest.java | 11 ++++------- 2 files changed, 5 insertions(+), 13 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java index 23cbc8d44b950..b285586646da6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializer.java @@ -114,16 +114,11 @@ public static synchronized void configure(DataflowWorkerLoggingOptions options) LogManager.getLogManager().getLogger(ROOT_LOGGER_NAME).setLevel( LEVELS.inverse().get(options.getDefaultWorkerLogLevel())); } - /* We store a reference to all the custom loggers the user configured. - * To make sure that these custom levels override the default logger level, - * we break the parent chain and have the logger directly pass log records - * to the file handler. */ + if (options.getWorkerLogLevelOverrides() != null) { for (WorkerLogLevelOverride loggerOverride : options.getWorkerLogLevelOverrides()) { Logger logger = Logger.getLogger(loggerOverride.getName()); - logger.setUseParentHandlers(false); logger.setLevel(LEVELS.inverse().get(loggerOverride.getLevel())); - logger.addHandler(fileHandler); configuredLoggers.add(logger); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java index 5d6fa6687a1d3..665c4f129a421 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingInitializerTest.java @@ -17,7 +17,6 @@ package com.google.cloud.dataflow.sdk.runners.worker.logging; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.options.DataflowWorkerLoggingOptions; @@ -87,16 +86,14 @@ public void testWithCustomLogLevels() { DataflowWorkerLoggingInitializer.configure(options); Logger aLogger = LogManager.getLogManager().getLogger("A"); - assertEquals(1, aLogger.getHandlers().length); + assertEquals(0, aLogger.getHandlers().length); assertEquals(Level.FINE, aLogger.getLevel()); - assertFalse(aLogger.getUseParentHandlers()); - assertTrue(isFileHandler(aLogger.getHandlers()[0], Level.ALL)); + assertTrue(aLogger.getUseParentHandlers()); Logger bLogger = LogManager.getLogManager().getLogger("B"); - assertEquals(1, bLogger.getHandlers().length); assertEquals(Level.SEVERE, bLogger.getLevel()); - assertFalse(bLogger.getUseParentHandlers()); - assertTrue(isFileHandler(bLogger.getHandlers()[0], Level.ALL)); + assertEquals(0, bLogger.getHandlers().length); + assertTrue(aLogger.getUseParentHandlers()); } private boolean isFileHandler(Handler handler, Level level) { From 95abb3fc7c36263406cb5ed6c62b5411c87886ae Mon Sep 17 00:00:00 2001 From: millsd Date: Wed, 4 Mar 2015 11:44:01 -0800 Subject: [PATCH 0216/1541] Cap TestPipeline job names at 40 chars; they'll throw an IllegalArgumentException otherwise. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87732028 --- .../cloud/dataflow/sdk/testing/TestPipeline.java | 3 ++- .../dataflow/sdk/testing/TestPipelineTest.java | 13 +++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java index dd8eb8866a470..7545e7ee10521 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java @@ -136,7 +136,8 @@ private static String getAppName() { private static String getJobName() { Optional stackTraceElement = findCallersStackTrace(); if (stackTraceElement.isPresent()) { - return stackTraceElement.get().getMethodName(); + String name = stackTraceElement.get().getMethodName(); + return name.substring(0, Math.min(40, name.length())); } return "unittestjob"; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java index 730efdd677fa3..e01b8b5c2b54d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java @@ -71,4 +71,17 @@ public void testCreationOfPipelineOptions() throws Exception { assertEquals("testZone", options.getZone()); assertEquals(2, options.getDiskSizeGb()); } + + @Test + public void testCreationOfPipelineOptionsFromReallyVerboselyNamedTestCase() throws Exception { + ObjectMapper mapper = new ObjectMapper(); + String stringOptions = mapper.writeValueAsString( + ImmutableMap.of("options", + ImmutableMap.builder() + .build())); + System.getProperties().put("dataflowOptions", stringOptions); + TestDataflowPipelineOptions options = TestPipeline.getPipelineOptions(); + assertEquals("TestPipelineTest", options.getAppName()); + assertEquals("testCreationOfPipelineOptionsFromReallyV", options.getJobName()); + } } From f5c63011acac19a77f7ae5d8483eefebff9af59b Mon Sep 17 00:00:00 2001 From: sisk Date: Wed, 4 Mar 2015 16:36:28 -0800 Subject: [PATCH 0217/1541] Add code to one of the examples demonstrating how to properly log from a dataflow job. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87766081 --- .../com/google/cloud/dataflow/examples/TfIdf.java | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java b/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java index 422ffdf7dbbf2..79033eb9eede2 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/TfIdf.java @@ -51,6 +51,9 @@ import com.google.cloud.dataflow.sdk.values.PInput; import com.google.cloud.dataflow.sdk.values.TupleTag; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.File; import java.io.IOException; import java.net.URI; @@ -61,7 +64,7 @@ /** * An example that computes a basic TF-IDF search table for a directory or GCS prefix. * - *

    Concepts: joining data; side inputs + *

    Concepts: joining data; side inputs; logging * *

    To execute this pipeline locally, specify general pipeline configuration: * --project= @@ -227,6 +230,11 @@ public void processElement(ProcessContext c) { URI uri = c.element().getKey(); String line = c.element().getValue(); for (String word : line.split("\\W+")) { + // Log INFO messages when the word “love” is found. + if (word.toLowerCase().equals("love")) { + LOG.info("Found {}", word.toLowerCase()); + } + if (!word.isEmpty()) { c.output(KV.of(uri, word.toLowerCase())); } @@ -373,6 +381,11 @@ public void processElement(ProcessContext c) { return wordToUriAndTfIdf; } + + // Instantiate Logger. + // It is suggested that the user specify the class name of the containing class + // (in this case ComputeTfIdf). + private static final Logger LOG = LoggerFactory.getLogger(ComputeTfIdf.class); } /** From 474eb6ba5b63e2e1c2fdc2a71fe626132a9755f9 Mon Sep 17 00:00:00 2001 From: vanya Date: Wed, 4 Mar 2015 17:12:55 -0800 Subject: [PATCH 0218/1541] Do not stop ignoring HTTP 307/308 when extra ignored status codes are provided. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87769846 --- .../cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java index 1301d6c010fc7..c0278e1ba5f0f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java @@ -137,11 +137,11 @@ public RetryHttpRequestInitializer(@Nullable HttpRequestInitializer chained) { } public RetryHttpRequestInitializer(@Nullable HttpRequestInitializer chained, - NanoClock nanoClock, Sleeper sleeper, Collection ignoredResponseCodes) { + NanoClock nanoClock, Sleeper sleeper, Collection additionalIgnoredResponseCodes) { this.chained = chained; this.nanoClock = nanoClock; this.sleeper = sleeper; - this.ignoredResponseCodes = new HashSet<>(ignoredResponseCodes); + this.ignoredResponseCodes.addAll(additionalIgnoredResponseCodes); } @Override From 636b38795326ffd9e6fcab63786536ac5697d314 Mon Sep 17 00:00:00 2001 From: peihe Date: Wed, 4 Mar 2015 21:07:37 -0800 Subject: [PATCH 0219/1541] CL resubmit: Windowing: java worker code to handle CombineFn in StreamingGroupAlsoByWindowsDoFn [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87786512 --- .../worker/GroupAlsoByWindowsParDoFn.java | 21 ++- .../dataflow/sdk/transforms/Combine.java | 2 +- .../cloud/dataflow/sdk/transforms/DoFn.java | 5 + .../sdk/util/BatchModeExecutionContext.java | 8 + .../dataflow/sdk/util/CombiningWindowSet.java | 140 ++++++++++++++++++ .../dataflow/sdk/util/DoFnProcessContext.java | 14 +- .../util/StreamingGroupAlsoByWindowsDoFn.java | 15 +- .../util/StreamingModeExecutionContext.java | 6 + .../StreamingGroupAlsoByWindowsDoFnTest.java | 84 +++++++++-- 9 files changed, 274 insertions(+), 21 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombiningWindowSet.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java index c3434b75d9834..eb05e336bfb9c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java @@ -19,6 +19,7 @@ import static com.google.cloud.dataflow.sdk.util.Structs.getBytes; import static com.google.cloud.dataflow.sdk.util.Structs.getObject; +import com.google.api.client.util.Preconditions; import com.google.api.services.dataflow.model.MultiOutputInfo; import com.google.api.services.dataflow.model.SideInputInfo; import com.google.cloud.dataflow.sdk.coders.Coder; @@ -73,13 +74,15 @@ public static GroupAlsoByWindowsParDoFn create( } byte[] serializedCombineFn = getBytes(cloudUserFn, PropertyNames.COMBINE_FN, null); - Object combineFn = null; + final Object combineFn; if (serializedCombineFn != null) { combineFn = SerializableUtils.deserializeFromByteArray(serializedCombineFn, "serialized combine fn"); if (!(combineFn instanceof KeyedCombineFn)) { throw new Exception("unexpected kind of KeyedCombineFn: " + combineFn.getClass().getName()); } + } else { + combineFn = null; } Map inputCoderObject = getObject(cloudUserFn, PropertyNames.INPUT_CODER); @@ -106,18 +109,26 @@ public static GroupAlsoByWindowsParDoFn create( fnFactory = new DoFnInfoFactory() { @Override public DoFnInfo createDoFnInfo() { - return new DoFnInfo(StreamingGroupAlsoByWindowsDoFn.create( - (WindowFn) windowFn, - ((KvCoder) elemCoder).getValueCoder()), + return new DoFnInfo( + StreamingGroupAlsoByWindowsDoFn.create( + (WindowFn) windowFn, + (KeyedCombineFn) combineFn, + ((KvCoder) elemCoder).getValueCoder()), null); } }; } else { + //TODO: handle CombineFn in batch GroupAlsoByWindowsDoFn. + Preconditions.checkArgument( + combineFn == null, + "combineFn is expected to be null in batch, but it is " + combineFn); fnFactory = new DoFnInfoFactory() { @Override public DoFnInfo createDoFnInfo() { return new DoFnInfo( - new GroupAlsoByWindowsDoFn((WindowFn) windowFn, elemCoder), + new GroupAlsoByWindowsDoFn( + (WindowFn) windowFn, + elemCoder), null); } }; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java index 14ff00c49f810..3e7f2895631df 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -818,7 +818,7 @@ protected String getKindString() { * {@link #perKey(SerializableFunction)}, and * {@link #groupedValues(SerializableFunction)}. */ - static class SimpleCombineFn extends CombineFn, V> { + public static class SimpleCombineFn extends CombineFn, V> { /** * Returns a {@code CombineFn} that uses the given * {@code SerializableFunction} to combine values. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java index fa5fc4ed1ed30..3b71738ad7970 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java @@ -281,6 +281,11 @@ public interface KeyedState { */ public void store(CodedTupleTag tag, T value) throws IOException; + /** + * Removes the data associated with the given tag from {@code KeyedState}. + */ + public void remove(CodedTupleTag tag); + /** * Returns the value associated with the given tag in this * {@code KeyedState}, or {@code null} if the tag has no asssociated diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java index f524b23ad1c78..051fe792db9cd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java @@ -106,6 +106,14 @@ public void store(CodedTupleTag tag, T value) { perKeyState.put(tag, value); } + @Override + public void remove(CodedTupleTag tag) { + Map, Object> perKeyState = state.get(getKey()); + if (perKeyState != null) { + perKeyState.remove(tag); + } + } + @Override public CodedTupleTagMap lookup(List> tags) { Map, Object> perKeyState = state.get(getKey()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombiningWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombiningWindowSet.java new file mode 100644 index 0000000000000..fe754bcb7cd96 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombiningWindowSet.java @@ -0,0 +1,140 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.cloud.dataflow.sdk.util.WindowUtils.bufferTag; + +import com.google.api.client.util.Lists; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.collect.Iterators; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * A WindowSet for combine accumulators. + * It merges accumulators when windows are added or merged. + * + * @param key tyep + * @param accumulator type + * @param window type + */ +public class CombiningWindowSet + extends AbstractWindowSet { + + private final CodedTupleTag> windowListTag = + CodedTupleTag.of("liveWindowsList", IterableCoder.of(windowFn.windowCoder())); + + private final KeyedCombineFn combineFn; + private final Set liveWindows; + private boolean liveWindowsModified; + + protected CombiningWindowSet( + K key, + WindowFn windowFn, + KeyedCombineFn combineFn, + Coder inputCoder, + DoFnProcessContext> context, + ActiveWindowManager activeWindowManager) throws Exception { + super(key, windowFn, inputCoder, context, activeWindowManager); + this.combineFn = combineFn; + liveWindows = new HashSet(); + Iterators.addAll(liveWindows, + emptyIfNull(context.keyedState().lookup(windowListTag)).iterator()); + liveWindowsModified = false; + } + + @Override + protected Collection windows() { + return Collections.unmodifiableSet(liveWindows); + } + + @Override + protected VA finalValue(W window) throws Exception { + return context.keyedState().lookup( + bufferTag(window, windowFn.windowCoder(), inputCoder)); + } + + @Override + protected void put(W window, VA value) throws Exception { + CodedTupleTag tag = bufferTag(window, windowFn.windowCoder(), inputCoder); + VA va = context.keyedState().lookup(tag); + VA newValue; + if (va == null) { + newValue = value; + } else { + newValue = combineFn.mergeAccumulators(key, Arrays.asList(value, va)); + } + context.keyedState().store(tag, newValue); + activeWindowManager.addWindow(window); + liveWindowsModified = liveWindows.add(window); + } + + @Override + protected void remove(W window) throws Exception { + context.keyedState().remove(bufferTag(window, windowFn.windowCoder(), inputCoder)); + activeWindowManager.addWindow(window); + liveWindowsModified = liveWindows.remove(window); + } + + @Override + protected void merge(Collection toBeMerged, W mergeResult) throws Exception { + List accumulators = Lists.newArrayList(); + for (W w : toBeMerged) { + VA va = context.keyedState().lookup( + bufferTag(w, windowFn.windowCoder(), inputCoder)); + // TODO: determine whether null means no value associated with the tag, b/19201776. + if (va != null) { + accumulators.add(va); + } + remove(w); + } + VA mergedVa = combineFn.mergeAccumulators(key, accumulators); + put(mergeResult, mergedVa); + } + + @Override + protected boolean contains(W window) { + return liveWindows.contains(window); + } + + private static Iterable emptyIfNull(Iterable list) { + if (list == null) { + return Collections.emptyList(); + } else { + return list; + } + } + + @Override + protected void flush() throws Exception { + if (liveWindowsModified) { + context.keyedState().store(windowListTag, liveWindows); + liveWindowsModified = false; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnProcessContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnProcessContext.java index dc6ea63faa776..9470e76c5762c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnProcessContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnProcessContext.java @@ -68,7 +68,7 @@ public I element() { @Override public KeyedState keyedState() { if (!(fn instanceof RequiresKeyedState) - || (element() != null && !(element() instanceof KV))) { + || !equivalentToKV(element())) { throw new UnsupportedOperationException( "Keyed state is only available in the context of a keyed DoFn marked as requiring state"); } @@ -139,4 +139,16 @@ private void checkTimestamp(Instant timestamp) { Preconditions.checkArgument( !timestamp.isBefore(windowedValue.getTimestamp().minus(fn.getAllowedTimestampSkew()))); } + + private boolean equivalentToKV(I input) { + if (input == null) { + return true; + } else if (input instanceof KV) { + return true; + } else if (input instanceof TimerOrElement) { + return ((TimerOrElement) input).isTimer() + || ((TimerOrElement) input).element() instanceof KV; + } + return false; + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java index 01503f783a900..741cd7130bfaf 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.util; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.PartitioningWindowFn; @@ -39,31 +40,39 @@ public class StreamingGroupAlsoByWindowsDoFn extends DoFn>, KV> implements DoFn.RequiresKeyedState { protected WindowFn windowFn; + protected KeyedCombineFn combineFn; protected Coder inputCoder; protected StreamingGroupAlsoByWindowsDoFn( WindowFn windowFn, + KeyedCombineFn combineFn, Coder inputCoder) { this.windowFn = windowFn; + this.combineFn = combineFn; this.inputCoder = inputCoder; } public static StreamingGroupAlsoByWindowsDoFn create( WindowFn windowFn, + KeyedCombineFn combineFn, Coder inputCoder) { - return new StreamingGroupAlsoByWindowsDoFn<>(windowFn, inputCoder); + return new StreamingGroupAlsoByWindowsDoFn<>(windowFn, combineFn, inputCoder); } private AbstractWindowSet createWindowSet( K key, DoFnProcessContext> context, AbstractWindowSet.ActiveWindowManager activeWindowManager) throws Exception { - if (windowFn instanceof PartitioningWindowFn) { + if (combineFn != null) { + return new CombiningWindowSet( + key, windowFn, combineFn, inputCoder, context, activeWindowManager); + } else if (windowFn instanceof PartitioningWindowFn) { return new PartitionBufferingWindowSet( key, windowFn, inputCoder, context, activeWindowManager); } else { - return new BufferingWindowSet(key, windowFn, inputCoder, context, activeWindowManager); + return new BufferingWindowSet( + key, windowFn, inputCoder, context, activeWindowManager); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingModeExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingModeExecutionContext.java index 69f13f4de5709..61e00ac101bb5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingModeExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingModeExecutionContext.java @@ -118,6 +118,12 @@ public void store(CodedTupleTag tag, T value) throws CoderException, IOEx stateCache.put(tag, KV.of(value, stream.toByteString())); } + @Override + public void remove(CodedTupleTag tag) { + // Write ByteString.EMPTY to indicate the value associated with the tag is removed. + stateCache.put(tag, KV.of(null, ByteString.EMPTY)); + } + @Override public CodedTupleTagMap lookup(List> tags) throws CoderException, IOException { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java index c1bb363bad2f8..5d5c03406f29c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java @@ -23,6 +23,10 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; +import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; +import com.google.cloud.dataflow.sdk.transforms.Sum; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; @@ -63,7 +67,7 @@ public class StreamingGroupAlsoByWindowsDoFnTest { @Test public void testEmpty() throws Exception { DoFnRunner>, KV>, List> runner = - makeRunner(FixedWindows.of(Duration.millis(10))); + makeRunner(FixedWindows.of(Duration.millis(10)), null); runner.startBundle(); @@ -77,7 +81,7 @@ public class StreamingGroupAlsoByWindowsDoFnTest { @Test public void testFixedWindows() throws Exception { DoFnRunner>, KV>, List> runner = - makeRunner(FixedWindows.of(Duration.millis(10))); + makeRunner(FixedWindows.of(Duration.millis(10)), null); Coder windowCoder = FixedWindows.of(Duration.millis(10)).windowCoder(); @@ -135,7 +139,7 @@ public class StreamingGroupAlsoByWindowsDoFnTest { @Test public void testSlidingWindows() throws Exception { DoFnRunner>, KV>, List> runner = - makeRunner(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10))); + makeRunner(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10)), null); Coder windowCoder = SlidingWindows.of(Duration.millis(10)).every(Duration.millis(10)).windowCoder(); @@ -200,7 +204,7 @@ public class StreamingGroupAlsoByWindowsDoFnTest { @Test public void testSessions() throws Exception { DoFnRunner>, KV>, List> runner = - makeRunner(Sessions.withGapDuration(Duration.millis(10))); + makeRunner(Sessions.withGapDuration(Duration.millis(10)), null); Coder windowCoder = Sessions.withGapDuration(Duration.millis(10)).windowCoder(); @@ -256,16 +260,74 @@ public class StreamingGroupAlsoByWindowsDoFnTest { assertThat(item1.getWindows(), Matchers.contains(window(15, 25))); } + @Test public void testSessionsCombine() throws Exception { + CombineFn combineFn = Combine.SimpleCombineFn.of(new Sum.SumLongFn()); + DoFnRunner>>, + KV>, List> runner = + makeRunner(Sessions.withGapDuration(Duration.millis(10)), + combineFn.asKeyedFn()); - private DoFnRunner>, - KV>, List> makeRunner( - WindowFn windowingStrategy) { + Coder windowCoder = + Sessions.withGapDuration(Duration.millis(10)).windowCoder(); - StreamingGroupAlsoByWindowsDoFn, IntervalWindow> fn = - StreamingGroupAlsoByWindowsDoFn.create(windowingStrategy, StringUtf8Coder.of()); + runner.startBundle(); - DoFnRunner>, - KV>, List> runner = + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", (Iterable) Arrays.asList(1L))), + new Instant(0), + Arrays.asList(window(0, 10)))); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", (Iterable) Arrays.asList(2L))), + new Instant(5), + Arrays.asList(window(5, 15)))); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", (Iterable) Arrays.asList(3L))), + new Instant(15), + Arrays.asList(window(15, 25)))); + + runner.processElement(WindowedValue.of( + TimerOrElement.element(KV.of("k", (Iterable) Arrays.asList(4L))), + new Instant(3), + Arrays.asList(window(3, 13)))); + + runner.processElement(WindowedValue.valueInEmptyWindows( + TimerOrElement.>>timer( + windowToString((IntervalWindow) window(0, 15), windowCoder), + new Instant(14), "k"))); + + runner.processElement(WindowedValue.valueInEmptyWindows( + TimerOrElement.>>timer( + windowToString((IntervalWindow) window(15, 25), windowCoder), + new Instant(24), "k"))); + + runner.finishBundle(); + + List>>> result = runner.getReceiver(outputTag); + + assertEquals(2, result.size()); + + WindowedValue>> item0 = result.get(0); + assertEquals("k", item0.getValue().getKey()); + assertThat(item0.getValue().getValue(), Matchers.containsInAnyOrder(7L)); + assertEquals(new Instant(14), item0.getTimestamp()); + assertThat(item0.getWindows(), Matchers.contains(window(0, 15))); + + WindowedValue>> item1 = result.get(1); + assertEquals("k", item1.getValue().getKey()); + assertThat(item1.getValue().getValue(), Matchers.containsInAnyOrder(3L)); + assertEquals(new Instant(24), item1.getTimestamp()); + assertThat(item1.getWindows(), Matchers.contains(window(15, 25))); + } + + private DoFnRunner makeRunner( + WindowFn windowingStrategy, + KeyedCombineFn combineFn) { + StreamingGroupAlsoByWindowsDoFn fn = + StreamingGroupAlsoByWindowsDoFn.create(windowingStrategy, combineFn, StringUtf8Coder.of()); + + DoFnRunner runner = DoFnRunner.createWithListOutputs( PipelineOptionsFactory.create(), fn, From 01d143774ba0988b9afbcacb95a63a69f1ec93e8 Mon Sep 17 00:00:00 2001 From: lcwik Date: Thu, 5 Mar 2015 10:34:32 -0800 Subject: [PATCH 0220/1541] Slight improvement of RetryHttpRequestInitializer interface. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87841386 --- .../sdk/util/RetryHttpRequestInitializer.java | 21 ++++++++++++++++++- .../cloud/dataflow/sdk/util/Transport.java | 4 +--- 2 files changed, 21 insertions(+), 4 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java index c0278e1ba5f0f..f01d5ffd62db6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java @@ -133,10 +133,29 @@ public boolean handleResponse(HttpRequest request, HttpResponse response, * applied to HttpRequest initialization. May be null. */ public RetryHttpRequestInitializer(@Nullable HttpRequestInitializer chained) { - this(chained, NanoClock.SYSTEM, Sleeper.DEFAULT, Collections.emptyList()); + this(chained, Collections.emptyList()); } + /** + * @param chained a downstream HttpRequestInitializer, which will also be + * applied to HttpRequest initialization. May be null. + * @param additionalIgnoredResponseCodes a list of HTTP status codes which should not be logged. + */ public RetryHttpRequestInitializer(@Nullable HttpRequestInitializer chained, + Collection additionalIgnoredResponseCodes) { + this(chained, NanoClock.SYSTEM, Sleeper.DEFAULT, additionalIgnoredResponseCodes); + } + + /** + * Visible for testing. + * + * @param chained a downstream HttpRequestInitializer, which will also be + * applied to HttpRequest initialization. May be null. + * @param nanoClock used as a timing source for knowing how much time has elapsed. + * @param sleeper used to sleep between retries. + * @param additionalIgnoredResponseCodes a list of HTTP status codes which should not be logged. + */ + RetryHttpRequestInitializer(@Nullable HttpRequestInitializer chained, NanoClock nanoClock, Sleeper sleeper, Collection additionalIgnoredResponseCodes) { this.chained = chained; this.nanoClock = nanoClock; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java index a5e9fe8ad6981..57876ae7fa4ca 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java @@ -23,8 +23,6 @@ import com.google.api.client.http.HttpTransport; import com.google.api.client.json.JsonFactory; import com.google.api.client.json.jackson2.JacksonFactory; -import com.google.api.client.util.NanoClock; -import com.google.api.client.util.Sleeper; import com.google.api.services.bigquery.Bigquery; import com.google.api.services.dataflow.Dataflow; import com.google.api.services.pubsub.Pubsub; @@ -153,7 +151,7 @@ public static Dataflow.Builder newDataflowClient(DataflowPipelineOptions options new RetryHttpRequestInitializer( // Do not log the code 404. Code up the stack will deal with 404's if needed, and // logging it by default clutters the output during file staging. - options.getGcpCredential(), NanoClock.SYSTEM, Sleeper.DEFAULT, Arrays.asList(404))) + options.getGcpCredential(), Arrays.asList(404))) .setApplicationName(options.getAppName()) .setGoogleClientRequestInitializer( new ChainedGoogleClientRequestInitializer(options.getGoogleApiTrace())); From ad389cb2869595eb964b6c5999ccc43acaf39df9 Mon Sep 17 00:00:00 2001 From: robertwb Date: Thu, 5 Mar 2015 11:26:27 -0800 Subject: [PATCH 0221/1541] Fix misleading member name (no externally visible change). [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87847659 --- .../util/common/worker/PartialGroupByKeyOperation.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperation.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperation.java index a1315d5d1a9f5..dc8c00ffe651c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperation.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/PartialGroupByKeyOperation.java @@ -369,18 +369,18 @@ public static class CombiningGroupingTable extends GroupingTable combiner; private final SizeEstimator keySizer; - private final SizeEstimator valueSizer; + private final SizeEstimator accumulatorSizer; public CombiningGroupingTable(long maxSize, GroupingKeyCreator groupingKeyCreator, PairInfo pairInfo, Combiner combineFn, SizeEstimator keySizer, - SizeEstimator valueSizer) { + SizeEstimator accumulatorSizer) { super(maxSize, groupingKeyCreator, pairInfo); this.combiner = combineFn; this.keySizer = keySizer; - this.valueSizer = valueSizer; + this.accumulatorSizer = accumulatorSizer; } @Override @@ -394,7 +394,7 @@ public GroupingTableEntry createTableEntry(final K key) throws Except public long getSize() { return keySize + accumulatorSize; } public void add(VI value) throws Exception { accumulator = combiner.add(key, accumulator, value); - accumulatorSize = valueSizer.estimateSize(accumulator); + accumulatorSize = accumulatorSizer.estimateSize(accumulator); } }; } From e1b3e43ca0789bc1d9d9c496ba53fc8339ec18f3 Mon Sep 17 00:00:00 2001 From: ckuhn Date: Thu, 5 Mar 2015 14:11:42 -0800 Subject: [PATCH 0222/1541] Removing the precondition that job names have 40 characters or less. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87864968 --- .../dataflow/sdk/runners/DataflowPipelineRunner.java | 2 -- .../dataflow/sdk/runners/DataflowPipelineRunnerTest.java | 9 ++++----- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index 896c60bcb7d15..fffec136c5a20 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -123,8 +123,6 @@ public static DataflowPipelineRunner fromOptions(PipelineOptions options) { "JobName invalid; the name must consist of only the characters " + "[-a-z0-9], starting with a letter and ending with a letter " + "or number"); - Preconditions.checkArgument(jobName.length() <= 40, - "JobName too long; must be no more than 40 characters in length"); return new DataflowPipelineRunner(dataflowOptions); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java index f31362d92be1b..79e17cec7d595 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java @@ -389,13 +389,11 @@ public void testInvalidJobName() throws IOException { List invalidNames = Arrays.asList( "invalid_name", "0invalid", - "invalid-", - "this-one-is-too-long-01234567890123456789"); + "invalid-"); List expectedReason = Arrays.asList( "JobName invalid", "JobName invalid", - "JobName invalid", - "JobName too long"); + "JobName invalid"); for (int i = 0; i < invalidNames.size(); ++i) { ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); @@ -416,7 +414,8 @@ public void testInvalidJobName() throws IOException { @Test public void testValidJobName() throws IOException { - List names = Arrays.asList("ok", "Ok", "A-Ok", "ok-123"); + List names = Arrays.asList("ok", "Ok", "A-Ok", "ok-123", + "this-one-is-fairly-long-01234567890123456789"); for (String name : names) { ArgumentCaptor jobCaptor = ArgumentCaptor.forClass(Job.class); From e8da55c4b38307bedbc2dabd19a2d953120b4284 Mon Sep 17 00:00:00 2001 From: peihe Date: Thu, 5 Mar 2015 14:36:09 -0800 Subject: [PATCH 0223/1541] FIX: sideinput expects WindowedValues, update BigQueryReader to support BigqueryIO directly as sideinput. [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87867783 --- .../cloud/dataflow/sdk/io/BigQueryIO.java | 6 ++-- .../sdk/runners/DirectPipelineRunner.java | 29 +++++++++++++++++-- .../sdk/runners/worker/BigQueryReader.java | 11 +++---- .../runners/worker/BigQueryReaderTest.java | 7 +++-- 4 files changed, 41 insertions(+), 12 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java index ae017680a1e26..061bf68cd5049 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java @@ -35,6 +35,7 @@ import com.google.cloud.dataflow.sdk.util.BigQueryTableInserter; import com.google.cloud.dataflow.sdk.util.ReaderUtils; import com.google.cloud.dataflow.sdk.util.Transport; +import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PDone; @@ -841,9 +842,10 @@ private static void evaluateReadHelper( } LOG.info("Reading from BigQuery table {}", toTableSpec(ref)); - List elems = ReaderUtils.readElemsFromReader(new BigQueryReader(client, ref)); + List> elems = + ReaderUtils.readElemsFromReader(new BigQueryReader(client, ref)); LOG.info("Number of records read from BigQuery: {}", elems.size()); - context.setPCollection(transform.getOutput(), elems); + context.setPCollectionWindowedValue(transform.getOutput(), elems); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java index c665e0d3c5227..6c481d624701d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java @@ -487,6 +487,13 @@ public interface EvaluationContext extends EvaluationResults { void setPCollectionValuesWithMetadata( PCollection pc, List> elements); + /** + * Sets the value of the given PCollection, where each element also has a timestamp + * and collection of windows. + * Throws an exception if the PCollection's value has already been set. + */ + void setPCollectionWindowedValue(PCollection pc, List> elements); + /** * Shorthand for setting the value of a PCollection where the elements do not have * timestamps or windows. @@ -655,7 +662,7 @@ Object getPValue(PValue pvalue) { * Convert a list of T to a list of {@code ValueWithMetadata}, with a timestamp of 0 * and null windows. */ - List> toValuesWithMetadata(List values) { + List> toValueWithMetadata(List values) { List> result = new ArrayList<>(values.size()); for (T value : values) { result.add(ValueWithMetadata.of(WindowedValue.valueInGlobalWindow(value))); @@ -663,9 +670,27 @@ List> toValuesWithMetadata(List values) { return result; } + /** + * Convert a list of {@code WindowedValue} to a list of {@code ValueWithMetadata}. + */ + List> toValueWithMetadataFromWindowedValue( + List> values) { + List> result = new ArrayList<>(values.size()); + for (WindowedValue value : values) { + result.add(ValueWithMetadata.of(value)); + } + return result; + } + @Override public void setPCollection(PCollection pc, List elements) { - setPCollectionValuesWithMetadata(pc, toValuesWithMetadata(elements)); + setPCollectionValuesWithMetadata(pc, toValueWithMetadata(elements)); + } + + @Override + public void setPCollectionWindowedValue( + PCollection pc, List> elements) { + setPCollectionValuesWithMetadata(pc, toValueWithMetadataFromWindowedValue(elements)); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReader.java index c5a27c71e7fa1..149c33a331e06 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReader.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.options.BigQueryOptions; import com.google.cloud.dataflow.sdk.util.BigQueryTableRowIterator; import com.google.cloud.dataflow.sdk.util.Transport; +import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import java.io.IOException; @@ -35,7 +36,7 @@ * progress reporting because the source is used only in situations where the entire table must be * read by each worker (i.e. the source is used as a side input). */ -public class BigQueryReader extends Reader { +public class BigQueryReader extends Reader> { final TableReference tableRef; final BigQueryOptions bigQueryOptions; final Bigquery bigQueryClient; @@ -57,7 +58,7 @@ public BigQueryReader(Bigquery bigQueryClient, TableReference tableRef) { } @Override - public ReaderIterator iterator() throws IOException { + public ReaderIterator> iterator() throws IOException { return new BigQueryReaderIterator( bigQueryClient != null ? bigQueryClient : Transport.newBigQueryClient(bigQueryOptions).build(), @@ -67,7 +68,7 @@ public ReaderIterator iterator() throws IOException { /** * A ReaderIterator that yields TableRow objects for each row of a BigQuery table. */ - class BigQueryReaderIterator extends AbstractReaderIterator { + class BigQueryReaderIterator extends AbstractReaderIterator> { private BigQueryTableRowIterator rowIterator; public BigQueryReaderIterator(Bigquery bigQueryClient, TableReference tableRef) { @@ -80,11 +81,11 @@ public boolean hasNext() { } @Override - public TableRow next() throws IOException { + public WindowedValue next() throws IOException { if (!hasNext()) { throw new NoSuchElementException(); } - return rowIterator.next(); + return WindowedValue.valueInGlobalWindow(rowIterator.next()); } @Override diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java index 41ee8fb83d79d..ee308f42a5707 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/BigQueryReaderTest.java @@ -32,6 +32,7 @@ import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.dataflow.sdk.util.Transport; +import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import com.google.common.collect.Lists; @@ -342,10 +343,10 @@ public void testRead() throws Exception { bigQueryClient, new TableReference().setProjectId(PROJECT_ID).setDatasetId(DATASET).setTableId(TABLE)); - Reader.ReaderIterator iterator = reader.iterator(); + Reader.ReaderIterator> iterator = reader.iterator(); Assert.assertTrue(iterator.hasNext()); - TableRow row = iterator.next(); + TableRow row = iterator.next().getValue(); Assert.assertEquals("Arthur", row.get("name")); Assert.assertEquals("42", row.get("integer")); @@ -360,7 +361,7 @@ public void testRead() throws Exception { Assert.assertTrue(((List) row.get("repeatedFloat")).isEmpty()); Assert.assertTrue(((List) row.get("repeatedRecord")).isEmpty()); - row = iterator.next(); + row = iterator.next().getValue(); Assert.assertEquals("Allison", row.get("name")); Assert.assertEquals("79", row.get("integer")); From 0e165fa1af3f48fe954725c8100875121570bb9c Mon Sep 17 00:00:00 2001 From: millsd Date: Thu, 5 Mar 2015 16:43:27 -0800 Subject: [PATCH 0224/1541] Fix bug in selection of AvroSink vs AvroByteSink and AvroReader vs AvroByteReader ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87881403 --- .../dataflow/sdk/runners/worker/AvroReader.java | 5 +---- .../sdk/runners/worker/AvroReaderFactory.java | 16 +++++----------- .../dataflow/sdk/runners/worker/AvroSink.java | 9 ++------- .../sdk/runners/worker/AvroSinkFactory.java | 15 ++++----------- .../cloud/dataflow/sdk/util/WindowedValue.java | 4 ++-- .../runners/worker/AvroReaderFactoryTest.java | 7 +++++-- .../sdk/runners/worker/AvroSinkFactoryTest.java | 3 ++- 7 files changed, 21 insertions(+), 38 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java index ac0c250e7d4f7..cf51cc7344034 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java @@ -54,10 +54,7 @@ public class AvroReader extends Reader> { private final Schema schema; public AvroReader(String filename, @Nullable Long startPosition, @Nullable Long endPosition, - WindowedValue.WindowedValueCoder coder) { - if (!(coder instanceof WindowedValue.ValueOnlyWindowedValueCoder)) { - throw new IllegalArgumentException("Expected ValueOnlyWindowedValueCoder"); - } + WindowedValue.ValueOnlyWindowedValueCoder coder) { if (!(coder.getValueCoder() instanceof AvroCoder)) { throw new IllegalArgumentException("AvroReader requires an AvroCoder"); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactory.java index 7d9f2277dd84f..1f3d320e16c5d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactory.java @@ -19,13 +19,12 @@ import static com.google.cloud.dataflow.sdk.util.Structs.getLong; import static com.google.cloud.dataflow.sdk.util.Structs.getString; -import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.PropertyNames; -import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.WindowedValue.ValueOnlyWindowedValueCoder; import com.google.cloud.dataflow.sdk.util.common.worker.Reader; /** @@ -46,16 +45,11 @@ static Reader create(CloudObject spec, Coder coder) throws Exception { Long startOffset = getLong(spec, PropertyNames.START_OFFSET, null); Long endOffset = getLong(spec, PropertyNames.END_OFFSET, null); - if (!(coder instanceof WindowedValueCoder)) { - return new AvroByteReader<>(filename, startOffset, endOffset, coder); - //throw new IllegalArgumentException("Expected WindowedValueCoder"); - } - - WindowedValueCoder windowedCoder = (WindowedValueCoder) coder; - if (windowedCoder.getValueCoder() instanceof AvroCoder) { - return new AvroReader(filename, startOffset, endOffset, windowedCoder); + if (coder instanceof ValueOnlyWindowedValueCoder) { + return (Reader) new AvroReader( + filename, startOffset, endOffset, (ValueOnlyWindowedValueCoder) coder); } else { - return new AvroByteReader<>(filename, startOffset, endOffset, windowedCoder); + return new AvroByteReader<>(filename, startOffset, endOffset, coder); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSink.java index bae4cd843be48..9ecf1c3cdebe2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSink.java @@ -17,7 +17,6 @@ package com.google.cloud.dataflow.sdk.runners.worker; import static com.google.cloud.dataflow.sdk.util.WindowedValue.ValueOnlyWindowedValueCoder; -import static com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.util.CoderUtils; @@ -51,16 +50,12 @@ public class AvroSink extends Sink> { final AvroCoder avroCoder; final Schema schema; - public AvroSink(String filename, WindowedValueCoder coder) { + public AvroSink(String filename, ValueOnlyWindowedValueCoder coder) { this(filename, "", "", 1, coder); } public AvroSink(String filenamePrefix, String shardFormat, String filenameSuffix, int shardCount, - WindowedValueCoder coder) { - if (!(coder instanceof ValueOnlyWindowedValueCoder)) { - throw new IllegalArgumentException("Expected ValueOnlyWindowedValueCoder"); - } - + ValueOnlyWindowedValueCoder coder) { if (!(coder.getValueCoder() instanceof AvroCoder)) { throw new IllegalArgumentException("AvroSink requires an AvroCoder"); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java index 82dbed23d06fe..001d624223b37 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java @@ -18,13 +18,12 @@ import static com.google.cloud.dataflow.sdk.util.Structs.getString; -import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.PropertyNames; -import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; +import com.google.cloud.dataflow.sdk.util.WindowedValue.ValueOnlyWindowedValueCoder; import com.google.cloud.dataflow.sdk.util.common.worker.Sink; /** @@ -47,16 +46,10 @@ static Sink create(CloudObject spec, Coder coder) throws Exception { String filename = getString(spec, PropertyNames.FILENAME); - if (!(coder instanceof WindowedValueCoder)) { - return new AvroByteSink<>(filename, coder); - //throw new IllegalArgumentException("Expected WindowedValueCoder"); - } - - WindowedValueCoder windowedCoder = (WindowedValueCoder) coder; - if (windowedCoder.getValueCoder() instanceof AvroCoder) { - return new AvroSink(filename, windowedCoder); + if (coder instanceof ValueOnlyWindowedValueCoder) { + return (Sink) new AvroSink(filename, (ValueOnlyWindowedValueCoder) coder); } else { - return new AvroByteSink<>(filename, windowedCoder); + return new AvroByteSink<>(filename, coder); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java index c031b63c61fa8..595d1b0985795 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java @@ -127,7 +127,7 @@ public Collection getWindows() { * Returns the {@code Coder} to use for a {@code WindowedValue}, * using the given valueCoder and windowCoder. */ - public static WindowedValueCoder getFullCoder( + public static FullWindowedValueCoder getFullCoder( Coder valueCoder, Coder windowCoder) { return FullWindowedValueCoder.of(valueCoder, windowCoder); @@ -136,7 +136,7 @@ public static WindowedValueCoder getFullCoder( /** * Returns the {@code ValueOnlyCoder} from the given valueCoder. */ - public static WindowedValueCoder getValueOnlyCoder(Coder valueCoder) { + public static ValueOnlyWindowedValueCoder getValueOnlyCoder(Coder valueCoder) { return ValueOnlyWindowedValueCoder.of(valueCoder); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactoryTest.java index 25b3c01072910..06486696e4050 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactoryTest.java @@ -24,6 +24,7 @@ import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.WindowedValue; @@ -67,7 +68,8 @@ Reader runTestCreateAvroReader(String filename, @Nullable Long start, @Nullab @Test public void testCreatePlainAvroByteReader() throws Exception { - Coder coder = WindowedValue.getValueOnlyCoder(BigEndianIntegerCoder.of()); + Coder coder = WindowedValue.getFullCoder( + BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); Reader reader = runTestCreateAvroReader(pathToAvroFile, null, null, coder.asCloudObject()); Assert.assertThat(reader, new IsInstanceOf(AvroByteReader.class)); @@ -80,7 +82,8 @@ public void testCreatePlainAvroByteReader() throws Exception { @Test public void testCreateRichAvroByteReader() throws Exception { - Coder coder = WindowedValue.getValueOnlyCoder(BigEndianIntegerCoder.of()); + Coder coder = WindowedValue.getFullCoder( + BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); Reader reader = runTestCreateAvroReader(pathToAvroFile, 200L, 500L, coder.asCloudObject()); Assert.assertThat(reader, new IsInstanceOf(AvroByteReader.class)); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactoryTest.java index cb064c8371226..50309f3bb6ff8 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactoryTest.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.WindowedValue; @@ -59,7 +60,7 @@ Sink runTestCreateAvroSink(String filename, @Test public void testCreateAvroByteSink() throws Exception { Coder coder = - WindowedValue.getValueOnlyCoder(BigEndianIntegerCoder.of()); + WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); Sink sink = runTestCreateAvroSink( pathToAvroFile, coder.asCloudObject()); From bc188f03996bf4ca177a431b2adcc57557b37b88 Mon Sep 17 00:00:00 2001 From: peihe Date: Fri, 6 Mar 2015 02:48:20 -0800 Subject: [PATCH 0225/1541] Windowing: java worker code to handle CombineFn in batch GroupAlsoByWindowsDoFn. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87922554 --- .../worker/GroupAlsoByWindowsParDoFn.java | 1 + .../dataflow/sdk/transforms/GroupByKey.java | 4 +- .../sdk/util/GroupAlsoByWindowsDoFn.java | 51 +++++++++++-------- .../sdk/util/GroupAlsoByWindowsDoFnTest.java | 6 +-- 4 files changed, 37 insertions(+), 25 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java index eb05e336bfb9c..8b74253bd7fa6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java @@ -128,6 +128,7 @@ public DoFnInfo createDoFnInfo() { return new DoFnInfo( new GroupAlsoByWindowsDoFn( (WindowFn) windowFn, + (KeyedCombineFn) combineFn, elemCoder), null); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index 63e98a58dd5e8..93d657a800cf2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -255,8 +255,8 @@ public PCollection>> apply( KvCoder.of(keyCoder, outputValueCoder); return input.apply(ParDo.of( - new GroupAlsoByWindowsDoFn( - (WindowFn) windowFn, inputIterableElementValueCoder))) + new GroupAlsoByWindowsDoFn, BoundedWindow>( + (WindowFn) windowFn, null, inputIterableElementValueCoder))) .setCoder(outputKvCoder); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java index 86876bfcf5bdc..b493e75d2dda6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.util; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.NonMergingWindowFn; @@ -44,30 +45,34 @@ * combining values. * * @param key type - * @param input value element type + * @param input value element type + * @param output value element type * @param window type */ @SuppressWarnings("serial") -public class GroupAlsoByWindowsDoFn - extends DoFn>>, KV>> { +public class GroupAlsoByWindowsDoFn + extends DoFn>>, KV> { // TODO: Add back RequiresKeyed state once that is supported. protected WindowFn windowFn; - protected Coder inputCoder; + protected KeyedCombineFn combineFn; + protected Coder inputCoder; public GroupAlsoByWindowsDoFn( WindowFn windowFn, - Coder inputCoder) { + KeyedCombineFn combineFn, + Coder inputCoder) { this.windowFn = windowFn; + this.combineFn = combineFn; this.inputCoder = inputCoder; } @Override public void processElement(ProcessContext processContext) throws Exception { - DoFnProcessContext>>, KV>> context = - (DoFnProcessContext>>, KV>>) processContext; + DoFnProcessContext>>, KV> context = + (DoFnProcessContext>>, KV>) processContext; - if (windowFn instanceof NonMergingWindowFn) { + if (windowFn instanceof NonMergingWindowFn && combineFn == null) { processElementViaIterators(context); } else { processElementViaWindowSet(context); @@ -75,15 +80,21 @@ public void processElement(ProcessContext processContext) throws Exception { } private void processElementViaWindowSet( - DoFnProcessContext>>, KV>> context) + DoFnProcessContext>>, KV> context) throws Exception { K key = context.element().getKey(); BatchActiveWindowManager activeWindowManager = new BatchActiveWindowManager<>(); - AbstractWindowSet, W> windowSet = - new BufferingWindowSet(key, windowFn, inputCoder, context, activeWindowManager); + AbstractWindowSet windowSet; + if (combineFn == null) { + windowSet = new BufferingWindowSet( + key, windowFn, inputCoder, context, activeWindowManager); + } else { + windowSet = new CombiningWindowSet( + key, windowFn, combineFn, inputCoder, context, activeWindowManager); + } - for (WindowedValue e : context.element().getValue()) { + for (WindowedValue e : context.element().getValue()) { for (BoundedWindow window : e.getWindows()) { windowSet.put((W) window, e.getValue()); } @@ -127,17 +138,17 @@ private void maybeOutputWindows( } private void processElementViaIterators( - DoFnProcessContext>>, KV>> context) + DoFnProcessContext>>, KV> context) throws Exception { K key = context.element().getKey(); - Iterable> value = context.element().getValue(); - PeekingReiterator> iterator; + Iterable> value = context.element().getValue(); + PeekingReiterator> iterator; if (value instanceof Collection) { - iterator = new PeekingReiterator<>(new ListReiterator>( - new ArrayList>((Collection>) value), 0)); + iterator = new PeekingReiterator<>(new ListReiterator>( + new ArrayList>((Collection>) value), 0)); } else if (value instanceof Reiterable) { - iterator = new PeekingReiterator(((Reiterable>) value).iterator()); + iterator = new PeekingReiterator(((Reiterable>) value).iterator()); } else { throw new IllegalArgumentException( "Input to GroupAlsoByWindowsDoFn must be a Collection or Reiterable"); @@ -148,7 +159,7 @@ private void processElementViaIterators( ListMultimap windows = ArrayListMultimap.create(); while (iterator.hasNext()) { - WindowedValue e = iterator.peek(); + WindowedValue e = iterator.peek(); for (BoundedWindow window : e.getWindows()) { // If this window is not already in the active set, emit a new WindowReiterable // corresponding to this window, starting at this element in the input Reiterable. @@ -157,7 +168,7 @@ private void processElementViaIterators( // for as long as it detects that there are no new windows. windows.put(window.maxTimestamp(), window); context.outputWindowedValue( - KV.of(key, (Iterable) new WindowReiterable(iterator, window)), + KV.of(key, (VO) new WindowReiterable(iterator, window)), window.maxTimestamp(), Arrays.asList((W) window)); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java index 677dac3ff8b0e..464fcf9b868a7 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java @@ -252,9 +252,9 @@ public class GroupAlsoByWindowsDoFnTest { KV>, List> makeRunner( WindowFn windowFn) { - GroupAlsoByWindowsDoFn fn = - new GroupAlsoByWindowsDoFn( - windowFn, StringUtf8Coder.of()); + GroupAlsoByWindowsDoFn, IntervalWindow> fn = + new GroupAlsoByWindowsDoFn, IntervalWindow>( + windowFn, null, StringUtf8Coder.of()); DoFnRunner>>, KV>, List> runner = From 8a3e55d2b8fdf6b7f800976cf20610bd400e7965 Mon Sep 17 00:00:00 2001 From: davor Date: Fri, 6 Mar 2015 10:50:21 -0800 Subject: [PATCH 0226/1541] Setting up 'contrib' directory and rules. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87958147 --- CONTRIBUTING.md | 7 ++++++- contrib/README.md | 53 +++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 59 insertions(+), 1 deletion(-) create mode 100644 contrib/README.md diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 26147ec273b71..8b0fcba196f81 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -19,7 +19,12 @@ frustration later on. ### Code reviews All submissions, including submissions by project members, require review. We -use Github pull requests for this purpose. +use GitHub pull requests for this purpose. + +### Organization +During our review and triage of incoming pull requests, we'll advise whether to +include your contribution into the mainline SDK, or to maintain it within the +separate group of [community-contributed modules](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/tree/master/contrib). ### The small print Contributions made by corporations are covered by a different agreement than diff --git a/contrib/README.md b/contrib/README.md new file mode 100644 index 0000000000000..e3381b5e41d4f --- /dev/null +++ b/contrib/README.md @@ -0,0 +1,53 @@ +# Community contributions + +This directory hosts a wide variety of community contributions that may be +useful to other users of +[Google Cloud Dataflow](https://cloud.google.com/dataflow/), +but may not be appropriate or ready yet for inclusion into the +[mainline SDK](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/) or a +separate Google-maintained artifact. + +## Organization + +Each subdirectory represents a logically separate and independent module. +Preferably, the code is hosted directly in this repository. When appropriate, we +are also open to linking external repositories via +[`submodule`](http://git-scm.com/docs/git-submodule/) functionality within Git. + +While we are happy to host individual modules to provide additional value to all +Cloud Dataflow users, the modules are _maintanted solely by their respective +authors_. We will make sure that modules are related to Cloud Dataflow, that +they are distributed under the same license as the mainline SDK, and provide +some guidance to the authors to make the quality as high as possible. + +We __cannot__, however, provide _any_ guarantees about correctness, +compatibility, performance, support, test coverage, maintenance or future +availability of individual modules hosted here. + +## Process + +In general, we recommend to get in touch with us through the issue tracker +first. That way we can help out and possibly guide you. Coordinating up front +makes it much easier to avoid frustration later on. + +We welcome pull requests with a new module from everyone. Every module must be +related to Cloud Dataflow and must have an informative README.md file. We will +provide general guidance, but usually won't be reviewing the module in detail. +We reserve the right to refuse acceptance to any module, or remove it at any +time in the future. + +We also welcome improvements to an existing module from everyone. We'll often +wait for comments from the primary author of the module before merging a pull +request from a non-primary author. + +As the module matures, we may choose to pull it directly into the mainline SDK +or promote it to a Google-managed artifact. + +## Licensing + +We require all contributors to sign the Contributor License Agreement, exactly +as we require for any contributions to the mainline SDK. More information is +available in our [CONTRIBUTING.md](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/CONTRIBUTING.md) +file. + +_Thank you for your contribution to the Cloud Dataflow community!_ From fa9ecb9b95b0c6462ee2d97ea38f44253ac6f4d8 Mon Sep 17 00:00:00 2001 From: earhart Date: Fri, 6 Mar 2015 14:06:59 -0800 Subject: [PATCH 0227/1541] Update Proto2Coder.java Add serialVersionUID for this serializable class. --- .../java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java | 1 + 1 file changed, 1 insertion(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java index 6b89ec140fc20..7e19cce9eb44a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java @@ -55,6 +55,7 @@ * @param the type of elements handled by this coder, must extend {@code Message} */ public class Proto2Coder extends CustomCoder { + private static final long serialVersionUID = 0; /** * Produces a new Proto2Coder instance, for a given Protobuf message class. From e1926f3fe5394dae0287bb15abe23331a65241a1 Mon Sep 17 00:00:00 2001 From: earhart Date: Fri, 6 Mar 2015 14:28:27 -0800 Subject: [PATCH 0228/1541] Add -Xlint:-options to pom.xml Cross-compiling with JDK 8 causes an innocuous warning; this flag disables that warning. --- pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pom.xml b/pom.xml index e8d531f600c93..e5da472cd1257 100644 --- a/pom.xml +++ b/pom.xml @@ -84,6 +84,8 @@ -Xlint:all -Werror + + -Xlint:-options -Xlint:-cast -Xlint:-deprecation From 2f7daac4fd8553d26c3fbbe6d65a9786fa7ff106 Mon Sep 17 00:00:00 2001 From: Rob Earhart Date: Fri, 6 Mar 2015 14:45:24 -0800 Subject: [PATCH 0229/1541] Add serialVersionUID to serializable abstract base classes. --- .../java/com/google/cloud/dataflow/sdk/coders/AtomicCoder.java | 2 ++ .../java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java | 1 + .../com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java | 1 + .../java/com/google/cloud/dataflow/sdk/coders/KvCoderBase.java | 2 ++ .../java/com/google/cloud/dataflow/sdk/coders/MapCoderBase.java | 2 ++ .../com/google/cloud/dataflow/sdk/coders/StandardCoder.java | 1 + sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java | 2 ++ .../com/google/cloud/dataflow/sdk/transforms/PTransform.java | 1 + .../java/com/google/cloud/dataflow/sdk/transforms/View.java | 1 + .../dataflow/sdk/transforms/windowing/NonMergingWindowFn.java | 1 + .../dataflow/sdk/transforms/windowing/PartitioningWindowFn.java | 2 ++ .../cloud/dataflow/sdk/transforms/windowing/WindowFn.java | 1 + .../java/com/google/cloud/dataflow/sdk/util/WindowedValue.java | 2 ++ 13 files changed, 19 insertions(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AtomicCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AtomicCoder.java index ae72091737bbf..15380ca380d94 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AtomicCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/AtomicCoder.java @@ -26,6 +26,8 @@ * @param the type of the values being transcoded */ public abstract class AtomicCoder extends StandardCoder { + private static final long serialVersionUID = 0; + protected AtomicCoder() {} @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java index f9adaf96c3242..5ad93521f172e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java @@ -39,6 +39,7 @@ */ public abstract class CustomCoder extends AtomicCoder implements Serializable { + private static final long serialVersionUID = 0; @JsonCreator public static CustomCoder of( diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java index c5202b22b286e..0aa24b6725742 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java @@ -39,6 +39,7 @@ */ public abstract class IterableLikeCoder> extends StandardCoder { + private static final long serialVersionUID = 0; public Coder getElemCoder() { return elementCoder; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoderBase.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoderBase.java index 212c6339a7010..f418e574c592b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoderBase.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/KvCoderBase.java @@ -31,6 +31,8 @@ * @param the type of values being transcoded */ public abstract class KvCoderBase extends StandardCoder { + private static final long serialVersionUID = 0; + @JsonCreator public static KvCoderBase of( // N.B. typeId is a required parameter here, since a field named "@type" diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoderBase.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoderBase.java index 7a6a99790e906..d187fa3b04fe7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoderBase.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoderBase.java @@ -31,6 +31,8 @@ * @param the type of values being transcoded */ public abstract class MapCoderBase extends StandardCoder { + private static final long serialVersionUID = 0; + @JsonCreator public static MapCoderBase of( // N.B. typeId is a required parameter here, since a field named "@type" diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java index ff27fd2be6dc8..8d7229168e94a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StandardCoder.java @@ -35,6 +35,7 @@ * @param the type of the values being transcoded */ public abstract class StandardCoder implements Coder { + private static final long serialVersionUID = 0; protected StandardCoder() {} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java index a4bf54f40651f..6ddf2a6c8642a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java @@ -52,6 +52,8 @@ * @param Type of elements read by the source. */ public abstract class Source implements Serializable { + private static final long serialVersionUID = 0; + /** * Splits the source into shards. * diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java index bf2f4bd4d1ff6..8f927c6e37639 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java @@ -169,6 +169,7 @@ */ public abstract class PTransform implements Serializable /* See the note above */ { + private static final long serialVersionUID = 0; /** * Applies this {@code PTransform} on the given {@code Input}, and returns its diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java index 760bccf0229f6..4f7fcd543628e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java @@ -335,6 +335,7 @@ public Map fromIterableInternal(Iterable> contents) { private abstract static class PCollectionViewBase extends PValueBase implements PCollectionView { + private static final long serialVersionUID = 0; @Override public TupleTag>> getTagInternal() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowFn.java index 0caaabe72281e..c42c4ff09a28b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/NonMergingWindowFn.java @@ -25,6 +25,7 @@ */ public abstract class NonMergingWindowFn extends WindowFn { + private static final long serialVersionUID = 0; @Override public final void mergeWindows(MergeContext c) { } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowFn.java index 7ec97bc40d5e8..85282084bfb72 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowFn.java @@ -30,6 +30,8 @@ */ public abstract class PartitioningWindowFn extends NonMergingWindowFn { + private static final long serialVersionUID = 0; + /** * Returns the single window to which elements with this timestamp belong. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java index 9eab380dba208..fdfa136ba5251 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java @@ -40,6 +40,7 @@ */ public abstract class WindowFn implements Serializable { + private static final long serialVersionUID = 0; /** * Information available when running {@link #assignWindows}. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java index 595d1b0985795..2147ef4867364 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java @@ -178,6 +178,8 @@ public String toString() { */ public abstract static class WindowedValueCoder extends StandardCoder> { + private static final long serialVersionUID = 0; + final Coder valueCoder; WindowedValueCoder(Coder valueCoder) { From 45b973d0ed9fc772ac3176b2699b6597e68e174b Mon Sep 17 00:00:00 2001 From: Rob Earhart Date: Fri, 6 Mar 2015 14:59:09 -0800 Subject: [PATCH 0230/1541] Add trivial hashCode() functions for tests which override equals() --- .../cloud/dataflow/sdk/coders/AvroCoderTest.java | 5 +++++ .../sdk/options/ProxyInvocationHandlerTest.java | 10 ++++++++++ 2 files changed, 15 insertions(+) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java index 513cf8ba0dc3c..2bd506cdf3549 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/AvroCoderTest.java @@ -112,6 +112,11 @@ public boolean equals(Object o) { return true; } + @Override + public int hashCode() { + return 0; + } + @Override public String toString() { return "Pojo{" diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java index 8d0beb7f11f31..e7f9d0e483978 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java @@ -525,6 +525,11 @@ static InnerType of(double value) { return rval; } + @Override + public int hashCode() { + return 0; + } + @Override public boolean equals(Object obj) { return obj != null @@ -540,6 +545,11 @@ private static class ComplexType { public List genericType; public InnerType innerType; + @Override + public int hashCode() { + return 0; + } + @Override public boolean equals(Object obj) { return obj != null From d1271e608f85fd2c01502ac00b955e93a89674e9 Mon Sep 17 00:00:00 2001 From: Rob Earhart Date: Fri, 6 Mar 2015 15:08:13 -0800 Subject: [PATCH 0231/1541] Add trivial hashCode() functions for tests which override equals() --- .../com/google/cloud/dataflow/examples/AutoComplete.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/AutoComplete.java b/examples/src/main/java/com/google/cloud/dataflow/examples/AutoComplete.java index 79d5515e35668..19d6984a66b56 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/AutoComplete.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/AutoComplete.java @@ -324,6 +324,11 @@ public boolean equals(Object other) { } } + @Override + public int hashCode() { + return Long.valueOf(count).hashCode() ^ value.hashCode(); + } + @Override public String toString() { return "CompletionCandidate[" + value + ", " + count + "]"; From 73a03a29ee272af23dca09da869843adfa7ddcac Mon Sep 17 00:00:00 2001 From: Magnus Runesson Date: Mon, 9 Mar 2015 09:12:12 +0100 Subject: [PATCH 0232/1541] Ignore files created by Intellij. --- .gitignore | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.gitignore b/.gitignore index 2f7896d1d1365..42ce01d2ce26b 100644 --- a/.gitignore +++ b/.gitignore @@ -1 +1,5 @@ target/ + +# Ignore Intellij files +.idea +*.iml From f8765aa576d60bc2a01c95a36bcf69495501c00a Mon Sep 17 00:00:00 2001 From: Magnus Runesson Date: Tue, 3 Mar 2015 08:52:51 +0100 Subject: [PATCH 0233/1541] Generic implementation of inner and outer left/right join. This is a patch to make it easier for user of DataFlow to do joins in the most common circumstances. It implements inner join and outer left and right join. In the cases of outer join. Since null cannot be serialized the user have to provide a value that represent null for that particular usecase. --- .gitignore | 2 + .../dataflow/sdk/transforms/join/Join.java | 176 ++++++++++++++++++ .../sdk/transforms/join/InnerJoinTest.java | 134 +++++++++++++ .../transforms/join/OuterLeftJoinTest.java | 141 ++++++++++++++ .../transforms/join/OuterRightJoinTest.java | 141 ++++++++++++++ 5 files changed, 594 insertions(+) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/Join.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/InnerJoinTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterLeftJoinTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterRightJoinTest.java diff --git a/.gitignore b/.gitignore index 2f7896d1d1365..e02456b1ee1f0 100644 --- a/.gitignore +++ b/.gitignore @@ -1 +1,3 @@ target/ +.idea +*.iml diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/Join.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/Join.java new file mode 100644 index 0000000000000..b3e3196105487 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/Join.java @@ -0,0 +1,176 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.base.Preconditions; + +/** + * Utility class with different versions of joins. All methods join two collections of + * key/value pairs (KV). + */ +public class Join { + + /** + * Inner join of two collections of KV elements. + * @param leftCollection Left side collection to join. + * @param rightCollection Right side collection to join. + * @param Type of the key for both collections + * @param Type of the values for the left collection. + * @param Type of the values for the right collection. + * @return A joined collection of KV where Key is of type V1 and Value is type V2. + */ + public static PCollection> innerJoin( + final PCollection> leftCollection, final PCollection> rightCollection) { + Preconditions.checkNotNull(leftCollection); + Preconditions.checkNotNull(rightCollection); + + final TupleTag v1Tuple = new TupleTag<>(); + final TupleTag v2Tuple = new TupleTag<>(); + + PCollection> coGbkResultCollection = + KeyedPCollectionTuple.of(v1Tuple, leftCollection) + .and(v2Tuple, rightCollection) + .apply(CoGroupByKey.create()); + + return coGbkResultCollection.apply(ParDo.of( + new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) { + KV e = c.element(); + + Iterable leftValuesIterable = e.getValue().getAll(v1Tuple); + Iterable rightValuesIterable = e.getValue().getAll(v2Tuple); + + for (V1 leftValue : leftValuesIterable) { + for (V2 rightValue : rightValuesIterable) { + c.output(KV.of(leftValue, rightValue)); + } + } + } + })) + .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(), + ((KvCoder) rightCollection.getCoder()).getValueCoder())); + } + + /** + * Left Outer Join of two collections of KV elements. + * @param leftCollection Left side collection to join. + * @param rightCollection Right side collection to join. + * @param nullValue Value to use as null value when right side do not match left side. + * @param Type of the key for both collections + * @param Type of the values for the left collection. + * @param Type of the values for the right collection. + * @return A joined collection of KV where Key is of type V1 and Value is type V2. Values that + * should be null or empty is replaced with nullValue. + */ + public static PCollection> leftOuterJoin( + final PCollection> leftCollection, + final PCollection> rightCollection, + final V2 nullValue) { + Preconditions.checkNotNull(leftCollection); + Preconditions.checkNotNull(rightCollection); + Preconditions.checkNotNull(nullValue); + + final TupleTag v1Tuple = new TupleTag<>(); + final TupleTag v2Tuple = new TupleTag<>(); + + PCollection> coGbkResultCollection = + KeyedPCollectionTuple.of(v1Tuple, leftCollection) + .and(v2Tuple, rightCollection) + .apply(CoGroupByKey.create()); + + return coGbkResultCollection.apply(ParDo.of( + new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) { + KV e = c.element(); + + Iterable leftValuesIterable = e.getValue().getAll(v1Tuple); + Iterable rightValuesIterable = e.getValue().getAll(v2Tuple); + + for (V1 leftValue : leftValuesIterable) { + if (rightValuesIterable.iterator().hasNext()) { + for (V2 rightValue : rightValuesIterable) { + c.output(KV.of(leftValue, rightValue)); + } + } else { + c.output(KV.of(leftValue, nullValue)); + } + } + } + })) + .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(), + ((KvCoder) rightCollection.getCoder()).getValueCoder())); + } + + /** + * Right Outer Join of two collections of KV elements. + * @param leftCollection Left side collection to join. + * @param rightCollection Right side collection to join. + * @param nullValue Value to use as null value when left side do not match right side. + * @param Type of the key for both collections + * @param Type of the values for the left collection. + * @param Type of the values for the right collection. + * @return A joined collection of KV where Key is of type V1 and Value is type V2. Keys that + * should be null or empty is replaced with nullValue. + */ + public static PCollection> rightOuterJoin( + final PCollection> leftCollection, + final PCollection> rightCollection, + final V1 nullValue) { + Preconditions.checkNotNull(leftCollection); + Preconditions.checkNotNull(rightCollection); + Preconditions.checkNotNull(nullValue); + + final TupleTag v1Tuple = new TupleTag<>(); + final TupleTag v2Tuple = new TupleTag<>(); + + PCollection> coGbkResultCollection = + KeyedPCollectionTuple.of(v1Tuple, leftCollection) + .and(v2Tuple, rightCollection) + .apply(CoGroupByKey.create()); + + return coGbkResultCollection.apply(ParDo.of( + new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) { + KV e = c.element(); + + Iterable leftValuesIterable = e.getValue().getAll(v1Tuple); + Iterable rightValuesIterable = e.getValue().getAll(v2Tuple); + + for (V2 rightValue : rightValuesIterable) { + if (leftValuesIterable.iterator().hasNext()) { + for (V1 leftValue : leftValuesIterable) { + c.output(KV.of(leftValue, rightValue)); + } + } else { + c.output(KV.of(nullValue, rightValue)); + } + } + } + })) + .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(), + ((KvCoder) rightCollection.getCoder()).getValueCoder())); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/InnerJoinTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/InnerJoinTest.java new file mode 100644 index 0000000000000..900835f172cf4 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/InnerJoinTest.java @@ -0,0 +1,134 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +/** + * This test Inner Join functionality. + */ +public class InnerJoinTest { + + Pipeline p; + List> leftListOfKv; + List> listRightOfKv; + List> expectedResult; + + @Before + public void setup() { + + p = TestPipeline.create(); + leftListOfKv = new ArrayList<>(); + listRightOfKv = new ArrayList<>(); + + expectedResult = new ArrayList<>(); + } + + @Test + public void testJoinOneToOneMapping() { + leftListOfKv.add(KV.of("Key1", 5L)); + leftListOfKv.add(KV.of("Key2", 4L)); + PCollection> leftCollection = p.apply(Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key1", "foo")); + listRightOfKv.add(KV.of("Key2", "bar")); + PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); + + PCollection> output = Join.innerJoin( + leftCollection, rightCollection); + + expectedResult.add(KV.of(5L, "foo")); + expectedResult.add(KV.of(4L, "bar")); + DataflowAssert.that(output).containsInAnyOrder(expectedResult); + + p.run(); + } + + @Test + public void testJoinOneToManyMapping() { + leftListOfKv.add(KV.of("Key2", 4L)); + PCollection> leftCollection = p.apply(Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key2", "bar")); + listRightOfKv.add(KV.of("Key2", "gazonk")); + PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); + + PCollection> output = Join.innerJoin( + leftCollection, rightCollection); + + expectedResult.add(KV.of(4L, "bar")); + expectedResult.add(KV.of(4L, "gazonk")); + DataflowAssert.that(output).containsInAnyOrder(expectedResult); + + p.run(); + } + + @Test + public void testJoinManyToOneMapping() { + leftListOfKv.add(KV.of("Key2", 4L)); + leftListOfKv.add(KV.of("Key2", 6L)); + PCollection> leftCollection = p.apply(Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key2", "bar")); + PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); + + PCollection> output = Join.innerJoin( + leftCollection, rightCollection); + + expectedResult.add(KV.of(4L, "bar")); + expectedResult.add(KV.of(6L, "bar")); + DataflowAssert.that(output).containsInAnyOrder(expectedResult); + + p.run(); + } + + @Test + public void testJoinNoneToNoneMapping() { + leftListOfKv.add(KV.of("Key2", 4L)); + PCollection> leftCollection = p.apply(Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key3", "bar")); + PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); + + PCollection> output = Join.innerJoin( + leftCollection, rightCollection); + + DataflowAssert.that(output).containsInAnyOrder(expectedResult); + p.run(); + } + + @Test(expected = NullPointerException.class) + public void testJoinLeftCollectionNull() { + Join.innerJoin(null, p.apply(Create.of(listRightOfKv))); + } + + @Test(expected = NullPointerException.class) + public void testJoinRightCollectionNull() { + Join.innerJoin(p.apply(Create.of(leftListOfKv)), null); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterLeftJoinTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterLeftJoinTest.java new file mode 100644 index 0000000000000..81c27157cf97b --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterLeftJoinTest.java @@ -0,0 +1,141 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + + +/** + * This test Outer Left Join functionality. + */ +public class OuterLeftJoinTest { + + Pipeline p; + List> leftListOfKv; + List> listRightOfKv; + List> expectedResult; + + @Before + public void setup() { + + p = TestPipeline.create(); + leftListOfKv = new ArrayList<>(); + listRightOfKv = new ArrayList<>(); + + expectedResult = new ArrayList<>(); + } + + @Test + public void testJoinOneToOneMapping() { + leftListOfKv.add(KV.of("Key1", 5L)); + leftListOfKv.add(KV.of("Key2", 4L)); + PCollection> leftCollection = p.apply(Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key1", "foo")); + listRightOfKv.add(KV.of("Key2", "bar")); + PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); + + PCollection> output = Join.leftOuterJoin( + leftCollection, rightCollection, ""); + + expectedResult.add(KV.of(5L, "foo")); + expectedResult.add(KV.of(4L, "bar")); + DataflowAssert.that(output).containsInAnyOrder(expectedResult); + + p.run(); + } + + @Test + public void testJoinOneToManyMapping() { + leftListOfKv.add(KV.of("Key2", 4L)); + PCollection> leftCollection = p.apply(Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key2", "bar")); + listRightOfKv.add(KV.of("Key2", "gazonk")); + PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); + + PCollection> output = Join.leftOuterJoin( + leftCollection, rightCollection, ""); + + expectedResult.add(KV.of(4L, "bar")); + expectedResult.add(KV.of(4L, "gazonk")); + DataflowAssert.that(output).containsInAnyOrder(expectedResult); + + p.run(); + } + + @Test + public void testJoinManyToOneMapping() { + leftListOfKv.add(KV.of("Key2", 4L)); + leftListOfKv.add(KV.of("Key2", 6L)); + PCollection> leftCollection = p.apply(Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key2", "bar")); + PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); + + PCollection> output = Join.leftOuterJoin( + leftCollection, rightCollection, ""); + + expectedResult.add(KV.of(4L, "bar")); + expectedResult.add(KV.of(6L, "bar")); + DataflowAssert.that(output).containsInAnyOrder(expectedResult); + + p.run(); + } + + @Test + public void testJoinOneToNoneMapping() { + leftListOfKv.add(KV.of("Key2", 4L)); + PCollection> leftCollection = p.apply(Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key3", "bar")); + PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); + + PCollection> output = Join.leftOuterJoin( + leftCollection, rightCollection, ""); + + expectedResult.add(KV.of(4L, "")); + DataflowAssert.that(output).containsInAnyOrder(expectedResult); + p.run(); + } + + @Test(expected = NullPointerException.class) + public void testJoinLeftCollectionNull() { + Join.leftOuterJoin(null, p.apply(Create.of(listRightOfKv)), ""); + } + + @Test(expected = NullPointerException.class) + public void testJoinRightCollectionNull() { + Join.leftOuterJoin(p.apply(Create.of(leftListOfKv)), null, ""); + } + + @Test(expected = NullPointerException.class) + public void testJoinNullValueIsNull() { + Join.leftOuterJoin(p.apply(Create.of(leftListOfKv)), p.apply(Create.of(listRightOfKv)), null); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterRightJoinTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterRightJoinTest.java new file mode 100644 index 0000000000000..5d668aaed4d5a --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterRightJoinTest.java @@ -0,0 +1,141 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + + +/** + * This test Outer Right Join functionality. + */ +public class OuterRightJoinTest { + + Pipeline p; + List> leftListOfKv; + List> listRightOfKv; + List> expectedResult; + + @Before + public void setup() { + + p = TestPipeline.create(); + leftListOfKv = new ArrayList<>(); + listRightOfKv = new ArrayList<>(); + + expectedResult = new ArrayList<>(); + } + + @Test + public void testJoinOneToOneMapping() { + leftListOfKv.add(KV.of("Key1", 5L)); + leftListOfKv.add(KV.of("Key2", 4L)); + PCollection> leftCollection = p.apply(Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key1", "foo")); + listRightOfKv.add(KV.of("Key2", "bar")); + PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); + + PCollection> output = Join.rightOuterJoin( + leftCollection, rightCollection, -1L); + + expectedResult.add(KV.of(5L, "foo")); + expectedResult.add(KV.of(4L, "bar")); + DataflowAssert.that(output).containsInAnyOrder(expectedResult); + + p.run(); + } + + @Test + public void testJoinOneToManyMapping() { + leftListOfKv.add(KV.of("Key2", 4L)); + PCollection> leftCollection = p.apply(Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key2", "bar")); + listRightOfKv.add(KV.of("Key2", "gazonk")); + PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); + + PCollection> output = Join.rightOuterJoin( + leftCollection, rightCollection, -1L); + + expectedResult.add(KV.of(4L, "bar")); + expectedResult.add(KV.of(4L, "gazonk")); + DataflowAssert.that(output).containsInAnyOrder(expectedResult); + + p.run(); + } + + @Test + public void testJoinManyToOneMapping() { + leftListOfKv.add(KV.of("Key2", 4L)); + leftListOfKv.add(KV.of("Key2", 6L)); + PCollection> leftCollection = p.apply(Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key2", "bar")); + PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); + + PCollection> output = Join.rightOuterJoin( + leftCollection, rightCollection, -1L); + + expectedResult.add(KV.of(4L, "bar")); + expectedResult.add(KV.of(6L, "bar")); + DataflowAssert.that(output).containsInAnyOrder(expectedResult); + + p.run(); + } + + @Test + public void testJoinNoneToOneMapping() { + leftListOfKv.add(KV.of("Key2", 4L)); + PCollection> leftCollection = p.apply(Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key3", "bar")); + PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); + + PCollection> output = Join.rightOuterJoin( + leftCollection, rightCollection, -1L); + + expectedResult.add(KV.of(-1L, "bar")); + DataflowAssert.that(output).containsInAnyOrder(expectedResult); + p.run(); + } + + @Test(expected = NullPointerException.class) + public void testJoinLeftCollectionNull() { + Join.rightOuterJoin(null, p.apply(Create.of(listRightOfKv)), ""); + } + + @Test(expected = NullPointerException.class) + public void testJoinRightCollectionNull() { + Join.rightOuterJoin(p.apply(Create.of(leftListOfKv)), null, -1L); + } + + @Test(expected = NullPointerException.class) + public void testJoinNullValueIsNull() { + Join.rightOuterJoin(p.apply(Create.of(leftListOfKv)), p.apply(Create.of(listRightOfKv)), null); + } +} From 314596aebc956fc6ba60c89f833e7cea71b58524 Mon Sep 17 00:00:00 2001 From: Magnus Runesson Date: Wed, 4 Mar 2015 13:19:48 +0100 Subject: [PATCH 0234/1541] Join key is keept Decided to keep the key in after the join is done. Original usecase did not require the key. It is a breaking change but worth doing. --- .../dataflow/sdk/transforms/join/Join.java | 49 +++++++++++-------- .../sdk/transforms/join/InnerJoinTest.java | 22 ++++----- .../transforms/join/OuterLeftJoinTest.java | 24 ++++----- .../transforms/join/OuterRightJoinTest.java | 24 ++++----- 4 files changed, 63 insertions(+), 56 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/Join.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/Join.java index b3e3196105487..2ada8c9b0f8b5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/Join.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/Join.java @@ -37,10 +37,12 @@ public class Join { * @param Type of the key for both collections * @param Type of the values for the left collection. * @param Type of the values for the right collection. - * @return A joined collection of KV where Key is of type V1 and Value is type V2. + * @return A joined collection of KV where Key is the key and value is a + * KV where Key is of type V1 and Value is type V2. */ - public static PCollection> innerJoin( - final PCollection> leftCollection, final PCollection> rightCollection) { + public static PCollection>> innerJoin( + final PCollection> leftCollection, final PCollection> rightCollection) + { Preconditions.checkNotNull(leftCollection); Preconditions.checkNotNull(rightCollection); @@ -53,7 +55,7 @@ public static PCollection> innerJoin( .apply(CoGroupByKey.create()); return coGbkResultCollection.apply(ParDo.of( - new DoFn, KV>() { + new DoFn, KV>>() { @Override public void processElement(ProcessContext c) { KV e = c.element(); @@ -63,13 +65,14 @@ public void processElement(ProcessContext c) { for (V1 leftValue : leftValuesIterable) { for (V2 rightValue : rightValuesIterable) { - c.output(KV.of(leftValue, rightValue)); + c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue))); } } } })) - .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(), - ((KvCoder) rightCollection.getCoder()).getValueCoder())); + .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getKeyCoder(), + KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(), + ((KvCoder) rightCollection.getCoder()).getValueCoder()))); } /** @@ -80,10 +83,11 @@ public void processElement(ProcessContext c) { * @param Type of the key for both collections * @param Type of the values for the left collection. * @param Type of the values for the right collection. - * @return A joined collection of KV where Key is of type V1 and Value is type V2. Values that + * @return A joined collection of KV where Key is the key and value is a + * KV where Key is of type V1 and Value is type V2. Values that * should be null or empty is replaced with nullValue. */ - public static PCollection> leftOuterJoin( + public static PCollection>> leftOuterJoin( final PCollection> leftCollection, final PCollection> rightCollection, final V2 nullValue) { @@ -100,7 +104,7 @@ public static PCollection> leftOuterJoin( .apply(CoGroupByKey.create()); return coGbkResultCollection.apply(ParDo.of( - new DoFn, KV>() { + new DoFn, KV>>() { @Override public void processElement(ProcessContext c) { KV e = c.element(); @@ -111,16 +115,17 @@ public void processElement(ProcessContext c) { for (V1 leftValue : leftValuesIterable) { if (rightValuesIterable.iterator().hasNext()) { for (V2 rightValue : rightValuesIterable) { - c.output(KV.of(leftValue, rightValue)); + c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue))); } } else { - c.output(KV.of(leftValue, nullValue)); + c.output(KV.of(e.getKey(), KV.of(leftValue, nullValue))); } } } })) - .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(), - ((KvCoder) rightCollection.getCoder()).getValueCoder())); + .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getKeyCoder(), + KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(), + ((KvCoder) rightCollection.getCoder()).getValueCoder()))); } /** @@ -131,10 +136,11 @@ public void processElement(ProcessContext c) { * @param Type of the key for both collections * @param Type of the values for the left collection. * @param Type of the values for the right collection. - * @return A joined collection of KV where Key is of type V1 and Value is type V2. Keys that + * @return A joined collection of KV where Key is the key and value is a + * KV where Key is of type V1 and Value is type V2. Keys that * should be null or empty is replaced with nullValue. */ - public static PCollection> rightOuterJoin( + public static PCollection>> rightOuterJoin( final PCollection> leftCollection, final PCollection> rightCollection, final V1 nullValue) { @@ -151,7 +157,7 @@ public static PCollection> rightOuterJoin( .apply(CoGroupByKey.create()); return coGbkResultCollection.apply(ParDo.of( - new DoFn, KV>() { + new DoFn, KV>>() { @Override public void processElement(ProcessContext c) { KV e = c.element(); @@ -162,15 +168,16 @@ public void processElement(ProcessContext c) { for (V2 rightValue : rightValuesIterable) { if (leftValuesIterable.iterator().hasNext()) { for (V1 leftValue : leftValuesIterable) { - c.output(KV.of(leftValue, rightValue)); + c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue))); } } else { - c.output(KV.of(nullValue, rightValue)); + c.output(KV.of(e.getKey(), KV.of(nullValue, rightValue))); } } } })) - .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(), - ((KvCoder) rightCollection.getCoder()).getValueCoder())); + .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getKeyCoder(), + KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(), + ((KvCoder) rightCollection.getCoder()).getValueCoder()))); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/InnerJoinTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/InnerJoinTest.java index 900835f172cf4..574a19323b6c0 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/InnerJoinTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/InnerJoinTest.java @@ -37,7 +37,7 @@ public class InnerJoinTest { Pipeline p; List> leftListOfKv; List> listRightOfKv; - List> expectedResult; + List>> expectedResult; @Before public void setup() { @@ -59,11 +59,11 @@ public void testJoinOneToOneMapping() { listRightOfKv.add(KV.of("Key2", "bar")); PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); - PCollection> output = Join.innerJoin( + PCollection>> output = Join.innerJoin( leftCollection, rightCollection); - expectedResult.add(KV.of(5L, "foo")); - expectedResult.add(KV.of(4L, "bar")); + expectedResult.add(KV.of("Key1", KV.of(5L, "foo"))); + expectedResult.add(KV.of("Key2", KV.of(4L, "bar"))); DataflowAssert.that(output).containsInAnyOrder(expectedResult); p.run(); @@ -78,11 +78,11 @@ public void testJoinOneToManyMapping() { listRightOfKv.add(KV.of("Key2", "gazonk")); PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); - PCollection> output = Join.innerJoin( + PCollection>> output = Join.innerJoin( leftCollection, rightCollection); - expectedResult.add(KV.of(4L, "bar")); - expectedResult.add(KV.of(4L, "gazonk")); + expectedResult.add(KV.of("Key2", KV.of(4L, "bar"))); + expectedResult.add(KV.of("Key2", KV.of(4L, "gazonk"))); DataflowAssert.that(output).containsInAnyOrder(expectedResult); p.run(); @@ -97,11 +97,11 @@ public void testJoinManyToOneMapping() { listRightOfKv.add(KV.of("Key2", "bar")); PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); - PCollection> output = Join.innerJoin( + PCollection>> output = Join.innerJoin( leftCollection, rightCollection); - expectedResult.add(KV.of(4L, "bar")); - expectedResult.add(KV.of(6L, "bar")); + expectedResult.add(KV.of("Key2", KV.of(4L, "bar"))); + expectedResult.add(KV.of("Key2", KV.of(6L, "bar"))); DataflowAssert.that(output).containsInAnyOrder(expectedResult); p.run(); @@ -115,7 +115,7 @@ public void testJoinNoneToNoneMapping() { listRightOfKv.add(KV.of("Key3", "bar")); PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); - PCollection> output = Join.innerJoin( + PCollection>> output = Join.innerJoin( leftCollection, rightCollection); DataflowAssert.that(output).containsInAnyOrder(expectedResult); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterLeftJoinTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterLeftJoinTest.java index 81c27157cf97b..ce8d531b6f3c1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterLeftJoinTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterLeftJoinTest.java @@ -38,7 +38,7 @@ public class OuterLeftJoinTest { Pipeline p; List> leftListOfKv; List> listRightOfKv; - List> expectedResult; + List>> expectedResult; @Before public void setup() { @@ -60,11 +60,11 @@ public void testJoinOneToOneMapping() { listRightOfKv.add(KV.of("Key2", "bar")); PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); - PCollection> output = Join.leftOuterJoin( + PCollection>> output = Join.leftOuterJoin( leftCollection, rightCollection, ""); - expectedResult.add(KV.of(5L, "foo")); - expectedResult.add(KV.of(4L, "bar")); + expectedResult.add(KV.of("Key1", KV.of(5L, "foo"))); + expectedResult.add(KV.of("Key2", KV.of(4L, "bar"))); DataflowAssert.that(output).containsInAnyOrder(expectedResult); p.run(); @@ -79,11 +79,11 @@ public void testJoinOneToManyMapping() { listRightOfKv.add(KV.of("Key2", "gazonk")); PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); - PCollection> output = Join.leftOuterJoin( + PCollection>> output = Join.leftOuterJoin( leftCollection, rightCollection, ""); - expectedResult.add(KV.of(4L, "bar")); - expectedResult.add(KV.of(4L, "gazonk")); + expectedResult.add(KV.of("Key2", KV.of(4L, "bar"))); + expectedResult.add(KV.of("Key2", KV.of(4L, "gazonk"))); DataflowAssert.that(output).containsInAnyOrder(expectedResult); p.run(); @@ -98,11 +98,11 @@ public void testJoinManyToOneMapping() { listRightOfKv.add(KV.of("Key2", "bar")); PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); - PCollection> output = Join.leftOuterJoin( + PCollection>> output = Join.leftOuterJoin( leftCollection, rightCollection, ""); - expectedResult.add(KV.of(4L, "bar")); - expectedResult.add(KV.of(6L, "bar")); + expectedResult.add(KV.of("Key2", KV.of(4L, "bar"))); + expectedResult.add(KV.of("Key2", KV.of(6L, "bar"))); DataflowAssert.that(output).containsInAnyOrder(expectedResult); p.run(); @@ -116,10 +116,10 @@ public void testJoinOneToNoneMapping() { listRightOfKv.add(KV.of("Key3", "bar")); PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); - PCollection> output = Join.leftOuterJoin( + PCollection>> output = Join.leftOuterJoin( leftCollection, rightCollection, ""); - expectedResult.add(KV.of(4L, "")); + expectedResult.add(KV.of("Key2", KV.of(4L, ""))); DataflowAssert.that(output).containsInAnyOrder(expectedResult); p.run(); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterRightJoinTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterRightJoinTest.java index 5d668aaed4d5a..f6c245d7afdfc 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterRightJoinTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterRightJoinTest.java @@ -38,7 +38,7 @@ public class OuterRightJoinTest { Pipeline p; List> leftListOfKv; List> listRightOfKv; - List> expectedResult; + List>> expectedResult; @Before public void setup() { @@ -60,11 +60,11 @@ public void testJoinOneToOneMapping() { listRightOfKv.add(KV.of("Key2", "bar")); PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); - PCollection> output = Join.rightOuterJoin( + PCollection>> output = Join.rightOuterJoin( leftCollection, rightCollection, -1L); - expectedResult.add(KV.of(5L, "foo")); - expectedResult.add(KV.of(4L, "bar")); + expectedResult.add(KV.of("Key1", KV.of(5L, "foo"))); + expectedResult.add(KV.of("Key2", KV.of(4L, "bar"))); DataflowAssert.that(output).containsInAnyOrder(expectedResult); p.run(); @@ -79,11 +79,11 @@ public void testJoinOneToManyMapping() { listRightOfKv.add(KV.of("Key2", "gazonk")); PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); - PCollection> output = Join.rightOuterJoin( + PCollection>> output = Join.rightOuterJoin( leftCollection, rightCollection, -1L); - expectedResult.add(KV.of(4L, "bar")); - expectedResult.add(KV.of(4L, "gazonk")); + expectedResult.add(KV.of("Key2", KV.of(4L, "bar"))); + expectedResult.add(KV.of("Key2", KV.of(4L, "gazonk"))); DataflowAssert.that(output).containsInAnyOrder(expectedResult); p.run(); @@ -98,11 +98,11 @@ public void testJoinManyToOneMapping() { listRightOfKv.add(KV.of("Key2", "bar")); PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); - PCollection> output = Join.rightOuterJoin( + PCollection>> output = Join.rightOuterJoin( leftCollection, rightCollection, -1L); - expectedResult.add(KV.of(4L, "bar")); - expectedResult.add(KV.of(6L, "bar")); + expectedResult.add(KV.of("Key2", KV.of(4L, "bar"))); + expectedResult.add(KV.of("Key2", KV.of(6L, "bar"))); DataflowAssert.that(output).containsInAnyOrder(expectedResult); p.run(); @@ -116,10 +116,10 @@ public void testJoinNoneToOneMapping() { listRightOfKv.add(KV.of("Key3", "bar")); PCollection> rightCollection = p.apply(Create.of(listRightOfKv)); - PCollection> output = Join.rightOuterJoin( + PCollection>> output = Join.rightOuterJoin( leftCollection, rightCollection, -1L); - expectedResult.add(KV.of(-1L, "bar")); + expectedResult.add(KV.of("Key3", KV.of(-1L, "bar"))); DataflowAssert.that(output).containsInAnyOrder(expectedResult); p.run(); } From ecc44e1d9f444ea6b78d145896c4e668b9f038aa Mon Sep 17 00:00:00 2001 From: Magnus Runesson Date: Mon, 9 Mar 2015 12:38:31 +0100 Subject: [PATCH 0235/1541] Move join code to contrib module join-library. Adding pom.xml and README.md. --- contrib/join-library/README.md | 25 +++++ contrib/join-library/pom.xml | 91 +++++++++++++++++++ .../dataflow/contrib/joinlibrary}/Join.java | 5 +- .../contrib/joinlibrary}/InnerJoinTest.java | 2 +- .../joinlibrary}/OuterLeftJoinTest.java | 2 +- .../joinlibrary}/OuterRightJoinTest.java | 2 +- 6 files changed, 123 insertions(+), 4 deletions(-) create mode 100644 contrib/join-library/README.md create mode 100644 contrib/join-library/pom.xml rename {sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join => contrib/join-library/src/main/java/com/google/cloud/dataflow/contrib/joinlibrary}/Join.java (96%) rename {sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join => contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary}/InnerJoinTest.java (98%) rename {sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join => contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary}/OuterLeftJoinTest.java (98%) rename {sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join => contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary}/OuterRightJoinTest.java (98%) diff --git a/contrib/join-library/README.md b/contrib/join-library/README.md new file mode 100644 index 0000000000000..b5766df50c6fd --- /dev/null +++ b/contrib/join-library/README.md @@ -0,0 +1,25 @@ +Join-library +============ + +Join-library provide inner join, outer left and right join functions to +Google DataFlow. The aim is to simplify the most common cases of join to a +simple function call. + +The functions are generic so it supports join of any types supported by +DataFlow. Input to the join functions are PCollections of Key/Values. Both the +left and right PCollections need the same type for the key. All the join +functions returns a Key/Value where Key is the join key and value is +a Key/Value where the key is the left value and right is the value. + +In the cases of outer join, since null cannot be serialized the user have +to provide a value that represent null for that particular use case. + +Example how to use join-library: + + PCollection> leftPcollection = ... + PCollection> rightPcollection = ... + + PCollection>> joinedPcollection = + Join.innerJoin(leftPcollection, rightPcollection); + +Questions or comments: diff --git a/contrib/join-library/pom.xml b/contrib/join-library/pom.xml new file mode 100644 index 0000000000000..2ae8bad3a2301 --- /dev/null +++ b/contrib/join-library/pom.xml @@ -0,0 +1,91 @@ + + + + 4.0.0 + + com.google.cloud.dataflow + join-library + Join library + Library with generic join functions for DataFlow. + 0.0.2-SNAPSHOT + jar + + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + UTF-8 + 0.3.150227 + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.2 + + 1.7 + 1.7 + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.12 + + ../../checkstyle.xml + true + true + true + + + + validate + + check + + + + + + + + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-all + ${google-cloud-dataflow-version} + + + + com.google.guava + guava + 18.0 + + + + + org.hamcrest + hamcrest-all + 1.3 + test + + + + junit + junit + 4.12 + test + + + + diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/Join.java b/contrib/join-library/src/main/java/com/google/cloud/dataflow/contrib/joinlibrary/Join.java similarity index 96% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/Join.java rename to contrib/join-library/src/main/java/com/google/cloud/dataflow/contrib/joinlibrary/Join.java index 2ada8c9b0f8b5..e13f9747c7942 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/Join.java +++ b/contrib/join-library/src/main/java/com/google/cloud/dataflow/contrib/joinlibrary/Join.java @@ -14,11 +14,14 @@ * the License. */ -package com.google.cloud.dataflow.sdk.transforms.join; +package com.google.cloud.dataflow.contrib.joinlibrary; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; +import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; +import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.TupleTag; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/InnerJoinTest.java b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/InnerJoinTest.java similarity index 98% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/InnerJoinTest.java rename to contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/InnerJoinTest.java index 574a19323b6c0..c031cba7956a4 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/InnerJoinTest.java +++ b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/InnerJoinTest.java @@ -14,7 +14,7 @@ * the License. */ -package com.google.cloud.dataflow.sdk.transforms.join; +package com.google.cloud.dataflow.contrib.joinlibrary; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterLeftJoinTest.java b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterLeftJoinTest.java similarity index 98% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterLeftJoinTest.java rename to contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterLeftJoinTest.java index ce8d531b6f3c1..c342ef706b0e1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterLeftJoinTest.java +++ b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterLeftJoinTest.java @@ -14,7 +14,7 @@ * the License. */ -package com.google.cloud.dataflow.sdk.transforms.join; +package com.google.cloud.dataflow.contrib.joinlibrary; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterRightJoinTest.java b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterRightJoinTest.java similarity index 98% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterRightJoinTest.java rename to contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterRightJoinTest.java index f6c245d7afdfc..23bdf301dc90a 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/OuterRightJoinTest.java +++ b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterRightJoinTest.java @@ -14,7 +14,7 @@ * the License. */ -package com.google.cloud.dataflow.sdk.transforms.join; +package com.google.cloud.dataflow.contrib.joinlibrary; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; From 87dd6286db01702ab29741c54474254fc634f2b7 Mon Sep 17 00:00:00 2001 From: Magnus Runesson Date: Mon, 9 Mar 2015 13:01:04 +0100 Subject: [PATCH 0236/1541] Gitignor should not be changed in this pullrequest. --- .gitignore | 2 -- 1 file changed, 2 deletions(-) diff --git a/.gitignore b/.gitignore index e02456b1ee1f0..2f7896d1d1365 100644 --- a/.gitignore +++ b/.gitignore @@ -1,3 +1 @@ target/ -.idea -*.iml From a99854466247bf541b6e288c6a7ac3fb8d1ba96f Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Mon, 9 Mar 2015 10:46:28 -0700 Subject: [PATCH 0237/1541] Update comment in .gitignore. --- .gitignore | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index 42ce01d2ce26b..1fed5fc45db84 100644 --- a/.gitignore +++ b/.gitignore @@ -1,5 +1,5 @@ target/ -# Ignore Intellij files -.idea +# Ignore IntelliJ files. +.idea/ *.iml From 70b0005485a96609fb1e48d92f0cd891968df2c8 Mon Sep 17 00:00:00 2001 From: peihe Date: Fri, 6 Mar 2015 11:16:01 -0800 Subject: [PATCH 0238/1541] Explicitly check cached mapTaskExecutor's all operations could restart. Previously it fails at operation level. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87960788 --- .../sdk/runners/worker/StreamingDataflowWorker.java | 4 +++- .../dataflow/sdk/util/common/worker/MapTaskExecutor.java | 9 +++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java index 05c7722aeb58b..e8c2758bb37f6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -34,11 +34,11 @@ import com.google.cloud.dataflow.sdk.util.common.Counter; import com.google.cloud.dataflow.sdk.util.common.CounterSet; import com.google.cloud.dataflow.sdk.util.common.worker.MapTaskExecutor; +import com.google.common.base.Preconditions; import org.eclipse.jetty.server.Request; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.AbstractHandler; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -331,6 +331,8 @@ private void process( if (workerAndContext == null) { context = new StreamingModeExecutionContext(computation, stateFetcher); worker = MapTaskExecutorFactory.create(options, mapTask, context); + Preconditions.checkState(worker.supportsRestart(), + "Streaming runner requires all operations support restart."); } else { worker = workerAndContext.getWorker(); context = workerAndContext.getContext(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java index 2da7f9a21998b..8f83e4bdc7f21 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutor.java @@ -107,4 +107,13 @@ public void close() throws Exception { stateSampler.close(); super.close(); } + + public boolean supportsRestart() { + for (Operation op : operations) { + if (!op.supportsRestart()) { + return false; + } + } + return true; + } } From 29936902ea80079f2bfecdab47b0e77c9660c661 Mon Sep 17 00:00:00 2001 From: peihe Date: Fri, 6 Mar 2015 12:18:43 -0800 Subject: [PATCH 0239/1541] Create: java sdk code and integration test for supporting Create in streaming. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87967100 --- .../cloud/dataflow/sdk/io/PubsubIO.java | 4 +- .../sdk/runners/DataflowPipelineRunner.java | 3 + .../cloud/dataflow/sdk/transforms/Create.java | 83 ++++++++++++++++--- 3 files changed, 79 insertions(+), 11 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index f501a316eb6c0..af9c7b9d60b03 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -64,6 +64,7 @@ public class PubsubIO { private static final int PUBSUB_NAME_MAX_LENGTH = 255; private static final String SUBSCRIPTION_RANDOM_TEST_PREFIX = "_random/"; + private static final String SUBSCRIPTION_STARTING_SIGNAL = "_starting_signal/"; private static final String TOPIC_DEV_NULL_TEST_NAME = "/topics/dev/null"; /** @@ -85,7 +86,8 @@ public static void validateTopicName(String topic) { } public static void validateSubscriptionName(String subscription) { - if (subscription.startsWith(SUBSCRIPTION_RANDOM_TEST_PREFIX)) { + if (subscription.startsWith(SUBSCRIPTION_RANDOM_TEST_PREFIX) + || subscription.startsWith(SUBSCRIPTION_STARTING_SIGNAL)) { return; } Matcher match = SUBSCRIPTION_REGEXP.matcher(subscription); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index fffec136c5a20..4283614c5e5bc 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -26,6 +26,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator; import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.GroupByKey; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo; @@ -150,6 +151,8 @@ public Output apply( boolean runnerSortsByTimestamp = true; return (Output) ((GroupByKey) transform).applyHelper( (PCollection) input, options.isStreaming(), runnerSortsByTimestamp); + } else if (transform instanceof Create) { + return (Output) ((Create) transform).applyHelper(input, options.isStreaming()); } else { return super.apply(transform, input); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java index 05a2ebf43a958..6cdb11b342b4e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java @@ -17,10 +17,17 @@ package com.google.cloud.dataflow.sdk.transforms; import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.io.PubsubIO; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.values.CodedTupleTag; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PBegin; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -31,6 +38,7 @@ import org.joda.time.Instant; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -184,9 +192,60 @@ public static CreateTimestamped timestamped( @Override public PCollection apply(PInput input) { - return PCollection.createPrimitiveOutputInternal(new GlobalWindows()); + return applyHelper(input, false); } + public PCollection applyHelper(PInput input, boolean isStreaming) { + if (isStreaming) { + Coder elemCoder = (Coder) getElementCoder(); + return Pipeline.applyTransform( + input, PubsubIO.Read.named("StartingSignal").subscription("_starting_signal/")) + .apply(ParDo.of(new DoFn>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(DoFn>.ProcessContext c) + throws Exception { + c.output(KV.of((Void) null, (Void) null)); + } + })) + .apply(ParDo.of(new OutputOnceDoFn<>(elems, elemCoder))); + } else { + return PCollection.createPrimitiveOutputInternal(new GlobalWindows()); + } + } + + private static class OutputOnceDoFn extends DoFn, T> + implements DoFn.RequiresKeyedState { + private static final long serialVersionUID = 0; + + private final CodedTupleTag outputOnceTag = + CodedTupleTag.of("outputOnce", StringUtf8Coder.of()); + private final byte[] encodedBytes; + private final IterableCoder iterableCoder; + + public OutputOnceDoFn(Iterable elems, Coder coder) { + this.iterableCoder = IterableCoder.of(coder); + try { + this.encodedBytes = CoderUtils.encodeToByteArray(iterableCoder, elems); + } catch (CoderException e) { + throw new IllegalArgumentException( + "Unable to encode element '" + elems + "' using coder '" + coder + "'.", e); + } + } + + @Override + public void processElement(ProcessContext c) throws IOException { + String state = c.keyedState().lookup(outputOnceTag); + if (state == null || state.isEmpty()) { + Iterable elems = CoderUtils.decodeFromByteArray(iterableCoder, encodedBytes); + for (T t : elems) { + c.output(t); + } + c.keyedState().store(outputOnceTag, "done"); + } + } + } ///////////////////////////////////////////////////////////////////////////// @@ -207,8 +266,7 @@ public Iterable getElements() { return elems; } - @Override - protected Coder getDefaultOutputCoder() { + private Coder getElementCoder() { // First try to deduce a coder using the types of the elements. Class elementType = null; for (T elem : elems) { @@ -222,7 +280,7 @@ protected Coder getDefaultOutputCoder() { } } if (elementType == null) { - return super.getDefaultOutputCoder(); + return null; } if (elementType.getTypeParameters().length == 0) { Coder candidate = getCoderRegistry().getDefaultCoder(TypeToken.of(elementType)); @@ -242,11 +300,17 @@ protected Coder getDefaultOutputCoder() { break; } } - if (coder != null) { - return coder; - } + return coder; + } - return super.getDefaultOutputCoder(); + @Override + protected Coder getDefaultOutputCoder() { + Coder elemCoder = getElementCoder(); + if (elemCoder == null) { + return super.getDefaultOutputCoder(); + } else { + return elemCoder; + } } /** @@ -279,13 +343,12 @@ public PCollection apply(PBegin input) { private static class ConvertTimestamps extends DoFn, T> { @Override - public void processElement(ProcessContext c) { + public void processElement(ProcessContext c) { c.outputWithTimestamp(c.element().getValue(), c.element().getTimestamp()); } } } - ///////////////////////////////////////////////////////////////////////////// static { From dc816e3847e2bb33f49fed0cf8e9c22f3dbee70b Mon Sep 17 00:00:00 2001 From: jeremiele Date: Fri, 6 Mar 2015 14:12:06 -0800 Subject: [PATCH 0240/1541] Added a getter to access the WindowFn from the Window.Bound PTransform. This is needed for third party runner implementations wanting to support windowing. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87978148 --- .../cloud/dataflow/sdk/transforms/windowing/Window.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java index 7e3a318eecf79..7521db8582baf 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java @@ -189,6 +189,13 @@ public Bound named(String name) { return new Bound<>(name, fn); } + /** + * Returns the user-specified {@code WindowFn}. + */ + public WindowFn getWindowFn() { + return fn; + } + @Override public PCollection apply(PCollection input) { return PCollection.createPrimitiveOutputInternal(fn); From c9f072d30262691ebf555c7b08ff527b5d22c8a5 Mon Sep 17 00:00:00 2001 From: sisk Date: Fri, 6 Mar 2015 15:27:27 -0800 Subject: [PATCH 0241/1541] Reorder the log line to move the user's message earlier in the log line, making it easier to read in the Cloud Logging UI. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=87985958 --- .../logging/DataflowWorkerLoggingFormatter.java | 6 +++--- .../DataflowWorkerLoggingFormatterTest.java | 16 ++++++++-------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java index 5af5cafa74aee..24a7e7ff82d2b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatter.java @@ -91,12 +91,12 @@ public String format(LogRecord record) { return DATE_FORMATTER.print(record.getMillis()) + " " + MoreObjects.firstNonNull(LEVELS.get(record.getLevel()), record.getLevel().getName()) - + " " + MoreObjects.firstNonNull(jobId.get(), "unknown") + + " " + record.getMessage() + + " [" + MoreObjects.firstNonNull(jobId.get(), "unknown") + " " + MoreObjects.firstNonNull(workerId.get(), "unknown") + " " + MoreObjects.firstNonNull(workId.get(), "unknown") + " " + record.getThreadID() - + " " + record.getLoggerName() - + " " + record.getMessage() + System.lineSeparator() + + "] " + record.getLoggerName() + System.lineSeparator() + (exception != null ? exception : ""); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java index 916a38567d58f..97650831acaac 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/logging/DataflowWorkerLoggingFormatterTest.java @@ -37,8 +37,8 @@ public class DataflowWorkerLoggingFormatterTest { @Test public void testWithUnsetValuesInMDC() { assertEquals( - "1970-01-01T00:00:00.001Z INFO unknown unknown unknown 2 LoggerName " - + "test.message" + System.lineSeparator(), + "1970-01-01T00:00:00.001Z INFO test.message [unknown unknown unknown 2] LoggerName" + + System.lineSeparator(), new DataflowWorkerLoggingFormatter().format( createLogRecord("test.message", null))); } @@ -50,8 +50,8 @@ public void testWithMessage() { DataflowWorkerLoggingFormatter.setWorkId("testWorkId"); assertEquals( - "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName " - + "test.message" + System.lineSeparator(), + "1970-01-01T00:00:00.001Z INFO test.message [testJobId testWorkerId testWorkId 2] " + + "LoggerName" + System.lineSeparator(), new DataflowWorkerLoggingFormatter().format( createLogRecord("test.message", null))); } @@ -63,8 +63,8 @@ public void testWithMessageAndException() { DataflowWorkerLoggingFormatter.setWorkId("testWorkId"); assertEquals( - "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName " - + "test.message" + System.lineSeparator() + "1970-01-01T00:00:00.001Z INFO test.message [testJobId testWorkerId testWorkId 2] " + + "LoggerName" + System.lineSeparator() + "java.lang.Throwable: exception.test.message" + System.lineSeparator() + "\tat declaringClass1.method1(file1.java:1)" + System.lineSeparator() + "\tat declaringClass2.method2(file2.java:1)" + System.lineSeparator() @@ -80,7 +80,7 @@ public void testWithException() { DataflowWorkerLoggingFormatter.setWorkId("testWorkId"); assertEquals( - "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName null" + "1970-01-01T00:00:00.001Z INFO null [testJobId testWorkerId testWorkId 2] LoggerName" + System.lineSeparator() + "java.lang.Throwable: exception.test.message" + System.lineSeparator() + "\tat declaringClass1.method1(file1.java:1)" + System.lineSeparator() @@ -97,7 +97,7 @@ public void testWithoutExceptionOrMessage() { DataflowWorkerLoggingFormatter.setWorkId("testWorkId"); assertEquals( - "1970-01-01T00:00:00.001Z INFO testJobId testWorkerId testWorkId 2 LoggerName null" + "1970-01-01T00:00:00.001Z INFO null [testJobId testWorkerId testWorkId 2] LoggerName" + System.lineSeparator(), new DataflowWorkerLoggingFormatter().format( createLogRecord(null, null))); From 6d0d1cd28280313a51283adef899164a42bf3bd0 Mon Sep 17 00:00:00 2001 From: chamikara Date: Sun, 8 Mar 2015 18:22:15 -0700 Subject: [PATCH 0242/1541] Adds a method to IOChannelFactory that can be used to efficiently determine if a read channel created using the factory is seekable. This is a backward incompatible change. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88082111 --- .../sdk/util/FileIOChannelFactory.java | 5 ++++ .../sdk/util/GcsIOChannelFactory.java | 6 +++++ .../dataflow/sdk/util/IOChannelFactory.java | 26 ++++++++++++++----- 3 files changed, 31 insertions(+), 6 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java index b43924273c98a..82a3f9ea37521 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/FileIOChannelFactory.java @@ -99,4 +99,9 @@ public WritableByteChannel create(String spec, String mimeType) public long getSizeBytes(String spec) throws IOException { return Files.size(FileSystems.getDefault().getPath(spec)); } + + @Override + public boolean isReadSeekEfficient(String spec) throws IOException { + return true; + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsIOChannelFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsIOChannelFactory.java index c0bb4624e1f5c..2f90428daa282 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsIOChannelFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GcsIOChannelFactory.java @@ -72,4 +72,10 @@ public long getSizeBytes(String spec) throws IOException { GcsUtil util = options.getGcsUtil(); return util.fileSize(path); } + + @Override + public boolean isReadSeekEfficient(String spec) throws IOException { + // TODO It is incorrect to return true here for files with content encoding set to gzip. + return true; + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/IOChannelFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/IOChannelFactory.java index 3661ffd3b1208..9e37907d233d9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/IOChannelFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/IOChannelFactory.java @@ -35,20 +35,20 @@ public interface IOChannelFactory { * Matches a specification, which may contain globs, against available * resources. * - * Glob handling is dependent on the implementation. Implementations should + *

    Glob handling is dependent on the implementation. Implementations should * all support globs in the final component of a path (eg /foo/bar/*.txt), * however they are not required to support globs in the directory paths. * - * The result is the (possibly empty) set of specifications which match. + *

    The result is the (possibly empty) set of specifications which match. */ Collection match(String spec) throws IOException; /** * Returns a read channel for the given specification. * - * The specification is not expanded; it is used verbatim. + *

    The specification is not expanded; it is used verbatim. * - * If seeking is supported, then this returns a + *

    If seeking is supported, then this returns a * {@link java.nio.channels.SeekableByteChannel}. */ ReadableByteChannel open(String spec) throws IOException; @@ -56,14 +56,28 @@ public interface IOChannelFactory { /** * Returns a write channel for the given specification. * - * The specification is not expanded; is it used verbatim. + *

    The specification is not expanded; is it used verbatim. */ WritableByteChannel create(String spec, String mimeType) throws IOException; /** * Returns the size in bytes for the given specification. * - * The specification is not expanded; it is used verbatim. + *

    The specification is not expanded; it is used verbatim. */ long getSizeBytes(String spec) throws IOException; + + /** + * Returns {@code true} if the channel created when invoking method {@link #open} for the given + * file specification is guaranteed to be of type {@link java.nio.channels.SeekableByteChannel + * SeekableByteChannel} and if seeking into positions of the channel is recommended. Returns + * {@code false} if the channel returned is not a {@code SeekableByteChannel}. May return + * {@code false} even if the channel returned is a {@code SeekableByteChannel}, if seeking is not + * efficient for the given file specification. + * + *

    Only efficiently seekable files can be split into offset ranges. + * + *

    The specification is not expanded; it is used verbatim. + */ + boolean isReadSeekEfficient(String spec) throws IOException; } From 73dc3715c382a2441895b8556d2e2553840ad8c9 Mon Sep 17 00:00:00 2001 From: bchambers Date: Mon, 9 Mar 2015 14:32:08 -0700 Subject: [PATCH 0243/1541] Cleanup warnings about use of generics in GroupAlsoByWindowsDoFn. There were a bunch of warnings around unsafe casts, etc. These look to be safe, subject to assumptions about how the DoFn is being used. This change attempts to put all the assumptions into the creation of the DoFn based on presence of a combineFn and the type of windowFn being used. It also documents the assumptions behind each of the @SuppressWarnings. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88152959 --- .../worker/GroupAlsoByWindowsParDoFn.java | 2 +- .../dataflow/sdk/transforms/GroupByKey.java | 9 +- .../sdk/util/GroupAlsoByWindowsDoFn.java | 290 +++++++++++------- .../sdk/util/GroupAlsoByWindowsDoFnTest.java | 3 +- 4 files changed, 184 insertions(+), 120 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java index 8b74253bd7fa6..62c185a06f286 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java @@ -126,7 +126,7 @@ public DoFnInfo createDoFnInfo() { @Override public DoFnInfo createDoFnInfo() { return new DoFnInfo( - new GroupAlsoByWindowsDoFn( + GroupAlsoByWindowsDoFn.create( (WindowFn) windowFn, (KeyedCombineFn) combineFn, elemCoder), diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index 93d657a800cf2..1f7024d4314b9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -25,7 +25,6 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.ValueWithMetadata; -import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.NonMergingWindowFn; @@ -254,10 +253,10 @@ public PCollection>> apply( Coder>> outputKvCoder = KvCoder.of(keyCoder, outputValueCoder); - return input.apply(ParDo.of( - new GroupAlsoByWindowsDoFn, BoundedWindow>( - (WindowFn) windowFn, null, inputIterableElementValueCoder))) - .setCoder(outputKvCoder); + GroupAlsoByWindowsDoFn, ?> fn = + GroupAlsoByWindowsDoFn.create(windowFn, inputIterableElementValueCoder); + + return input.apply(ParDo.of(fn)).setCoder(outputKvCoder); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java index b493e75d2dda6..896dc98b5d033 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java @@ -50,142 +50,207 @@ * @param window type */ @SuppressWarnings("serial") -public class GroupAlsoByWindowsDoFn +public abstract class GroupAlsoByWindowsDoFn extends DoFn>>, KV> { - // TODO: Add back RequiresKeyed state once that is supported. - protected WindowFn windowFn; - protected KeyedCombineFn combineFn; - protected Coder inputCoder; - - public GroupAlsoByWindowsDoFn( - WindowFn windowFn, - KeyedCombineFn combineFn, - Coder inputCoder) { - this.windowFn = windowFn; - this.combineFn = combineFn; - this.inputCoder = inputCoder; - } - - @Override - public void processElement(ProcessContext processContext) throws Exception { - DoFnProcessContext>>, KV> context = - (DoFnProcessContext>>, KV>) processContext; - - if (windowFn instanceof NonMergingWindowFn && combineFn == null) { - processElementViaIterators(context); + /** + * Create a {@link GroupAlsoByWindowsDoFn} without a combine function. Depending on the + * {@code windowFn} this will either use iterators or window sets to implement the grouping. + * + * @param windowFn The window function to use for grouping + * @param inputCoder the input coder to use + */ + public static GroupAlsoByWindowsDoFn, W> + create(final WindowFn windowFn, final Coder inputCoder) { + if (windowFn instanceof NonMergingWindowFn) { + return new GABWViaIteratorsDoFn(); } else { - processElementViaWindowSet(context); + return new GABWViaWindowSetDoFn, W>(windowFn) { + @Override + AbstractWindowSet, W> createWindowSet(K key, + DoFnProcessContext>>, KV>> context, + BatchActiveWindowManager activeWindowManager) throws Exception { + return new BufferingWindowSet(key, windowFn, inputCoder, + context, activeWindowManager); + } + }; } } - private void processElementViaWindowSet( - DoFnProcessContext>>, KV> context) - throws Exception { + /** + * Create a {@link GroupAlsoByWindowsDoFn} using the specified combineFn. + */ + private static GroupAlsoByWindowsDoFn + createCombine(final WindowFn windowFn, + final KeyedCombineFn combineFn, + final Coder inputCoder) { + return new GABWViaWindowSetDoFn(windowFn) { + @Override + AbstractWindowSet createWindowSet(K key, + DoFnProcessContext>>, KV> context, + BatchActiveWindowManager activeWindowManager) throws Exception { + return new CombiningWindowSet( + key, windowFn, combineFn, inputCoder, context, activeWindowManager); + } + }; + } - K key = context.element().getKey(); - BatchActiveWindowManager activeWindowManager = new BatchActiveWindowManager<>(); - AbstractWindowSet windowSet; + /** + * Construct a {@link GroupAlsoByWindowsDoFn} using the {@code combineFn} if available. + */ + public static GroupAlsoByWindowsDoFn + create(WindowFn windowFn, KeyedCombineFn combineFn, + Coder inputCoder) { if (combineFn == null) { - windowSet = new BufferingWindowSet( - key, windowFn, inputCoder, context, activeWindowManager); + // Without combineFn, it should be the case that VO = Iterable, so this is safe + @SuppressWarnings("unchecked") + GroupAlsoByWindowsDoFn fn = + (GroupAlsoByWindowsDoFn) create(windowFn, inputCoder); + return fn; } else { - windowSet = new CombiningWindowSet( - key, windowFn, combineFn, inputCoder, context, activeWindowManager); + // With a combineFn, then VI = VO, and we'll use those as the type of the accumulator + @SuppressWarnings("unchecked") + GroupAlsoByWindowsDoFn fn = + (GroupAlsoByWindowsDoFn) createCombine(windowFn, combineFn, inputCoder); + return fn; } + } - for (WindowedValue e : context.element().getValue()) { - for (BoundedWindow window : e.getWindows()) { - windowSet.put((W) window, e.getValue()); + private static class GABWViaIteratorsDoFn + extends GroupAlsoByWindowsDoFn, W> { + + @Override + public void processElement(ProcessContext c) throws Exception { + @SuppressWarnings({"unchecked", "rawtypes"}) + DoFnProcessContext>> internal = (DoFnProcessContext) c; + + K key = c.element().getKey(); + Iterable> value = c.element().getValue(); + PeekingReiterator> iterator; + + if (value instanceof Collection) { + iterator = new PeekingReiterator<>(new ListReiterator>( + new ArrayList>((Collection>) value), 0)); + } else if (value instanceof Reiterable) { + iterator = new PeekingReiterator<>(((Reiterable>) value).iterator()); + } else { + throw new IllegalArgumentException( + "Input to GroupAlsoByWindowsDoFn must be a Collection or Reiterable"); } - ((WindowFn) windowFn) - .mergeWindows(new AbstractWindowSet.WindowMergeContext(windowSet, windowFn)); - maybeOutputWindows(activeWindowManager, windowSet, windowFn, e.getTimestamp()); - } + // This ListMultimap is a map of window maxTimestamps to the list of active + // windows with that maxTimestamp. + ListMultimap windows = ArrayListMultimap.create(); - maybeOutputWindows(activeWindowManager, windowSet, windowFn, null); + while (iterator.hasNext()) { + WindowedValue e = iterator.peek(); + for (BoundedWindow window : e.getWindows()) { + // If this window is not already in the active set, emit a new WindowReiterable + // corresponding to this window, starting at this element in the input Reiterable. + if (!windows.containsEntry(window.maxTimestamp(), window)) { + // Iterating through the WindowReiterable may advance iterator as an optimization + // for as long as it detects that there are no new windows. + windows.put(window.maxTimestamp(), window); + internal.outputWindowedValue( + KV.of(key, (Iterable) new WindowReiterable(iterator, window)), + window.maxTimestamp(), + Arrays.asList(window)); + } + } + // Copy the iterator in case the next DoFn cached its version of the iterator instead + // of immediately iterating through it. + // And, only advance the iterator if the consuming operation hasn't done so. + iterator = iterator.copy(); + if (iterator.hasNext() && iterator.peek() == e) { + iterator.next(); + } - windowSet.flush(); + // Remove all windows with maxTimestamp behind the current timestamp. + Iterator windowIterator = windows.keys().iterator(); + while (windowIterator.hasNext() + && windowIterator.next().isBefore(e.getTimestamp())) { + windowIterator.remove(); + } + } + } } - /** - * Outputs any windows that are complete, with their corresponding elemeents. - * If there are potentially complete windows, try merging windows first. - */ - private void maybeOutputWindows( - BatchActiveWindowManager activeWindowManager, - AbstractWindowSet windowSet, - WindowFn windowFn, - Instant nextTimestamp) throws Exception { - if (activeWindowManager.hasMoreWindows() - && (nextTimestamp == null - || activeWindowManager.nextTimestamp().isBefore(nextTimestamp))) { - // There is at least one window ready to emit. Merge now in case that window should be merged - // into a not yet completed one. - ((WindowFn) windowFn) - .mergeWindows(new AbstractWindowSet.WindowMergeContext(windowSet, windowFn)); - } + private abstract static class GABWViaWindowSetDoFn + extends GroupAlsoByWindowsDoFn { - while (activeWindowManager.hasMoreWindows() - && (nextTimestamp == null - || activeWindowManager.nextTimestamp().isBefore(nextTimestamp))) { - W window = activeWindowManager.getWindow(); - if (windowSet.contains(window)) { - windowSet.markCompleted(window); - } + private WindowFn windowFn; + + public GABWViaWindowSetDoFn(WindowFn windowFn) { + @SuppressWarnings("unchecked") + WindowFn noWildcard = (WindowFn) windowFn; + this.windowFn = noWildcard; } - } - private void processElementViaIterators( - DoFnProcessContext>>, KV> context) - throws Exception { - K key = context.element().getKey(); - Iterable> value = context.element().getValue(); - PeekingReiterator> iterator; - - if (value instanceof Collection) { - iterator = new PeekingReiterator<>(new ListReiterator>( - new ArrayList>((Collection>) value), 0)); - } else if (value instanceof Reiterable) { - iterator = new PeekingReiterator(((Reiterable>) value).iterator()); - } else { - throw new IllegalArgumentException( - "Input to GroupAlsoByWindowsDoFn must be a Collection or Reiterable"); + abstract AbstractWindowSet createWindowSet( + K key, + DoFnProcessContext>>, KV> context, + BatchActiveWindowManager activeWindowManager) + throws Exception; + + @Override + public void processElement( + DoFn>>, KV>.ProcessContext c) throws Exception { + @SuppressWarnings("unchecked") + DoFnProcessContext>>, KV> context = + (DoFnProcessContext>>, KV>) c; + processElementViaWindowSet(context); } - // This ListMultimap is a map of window maxTimestamps to the list of active - // windows with that maxTimestamp. - ListMultimap windows = ArrayListMultimap.create(); - - while (iterator.hasNext()) { - WindowedValue e = iterator.peek(); - for (BoundedWindow window : e.getWindows()) { - // If this window is not already in the active set, emit a new WindowReiterable - // corresponding to this window, starting at this element in the input Reiterable. - if (!windows.containsEntry(window.maxTimestamp(), window)) { - // Iterating through the WindowReiterable may advance iterator as an optimization - // for as long as it detects that there are no new windows. - windows.put(window.maxTimestamp(), window); - context.outputWindowedValue( - KV.of(key, (VO) new WindowReiterable(iterator, window)), - window.maxTimestamp(), - Arrays.asList((W) window)); + public void processElementViaWindowSet( + DoFnProcessContext>>, KV> context) + throws Exception { + K key = context.element().getKey(); + BatchActiveWindowManager activeWindowManager = new BatchActiveWindowManager<>(); + AbstractWindowSet windowSet = + createWindowSet(key, context, activeWindowManager); + + for (WindowedValue e : context.element().getValue()) { + for (BoundedWindow window : e.getWindows()) { + @SuppressWarnings("unchecked") + W w = (W) window; + windowSet.put(w, e.getValue()); } + windowFn.mergeWindows( + new AbstractWindowSet.WindowMergeContext(windowSet, windowFn)); + + maybeOutputWindows(activeWindowManager, windowSet, e.getTimestamp()); } - // Copy the iterator in case the next DoFn cached its version of the iterator instead - // of immediately iterating through it. - // And, only advance the iterator if the consuming operation hasn't done so. - iterator = iterator.copy(); - if (iterator.hasNext() && iterator.peek() == e) { - iterator.next(); + + maybeOutputWindows(activeWindowManager, windowSet, null); + + windowSet.flush(); + } + + + /** + * Outputs any windows that are complete, with their corresponding elemeents. + * If there are potentially complete windows, try merging windows first. + */ + private void maybeOutputWindows( + BatchActiveWindowManager activeWindowManager, + AbstractWindowSet windowSet, + Instant nextTimestamp) throws Exception { + if (activeWindowManager.hasMoreWindows() + && (nextTimestamp == null + || activeWindowManager.nextTimestamp().isBefore(nextTimestamp))) { + // There is at least one window ready to emit. Merge now in case that window should be + // merged into a not yet completed one. + windowFn.mergeWindows( + new AbstractWindowSet.WindowMergeContext(windowSet, windowFn)); } - // Remove all windows with maxTimestamp behind the current timestamp. - Iterator windowIterator = windows.keys().iterator(); - while (windowIterator.hasNext() - && windowIterator.next().isBefore(e.getTimestamp())) { - windowIterator.remove(); + while (activeWindowManager.hasMoreWindows() + && (nextTimestamp == null + || activeWindowManager.nextTimestamp().isBefore(nextTimestamp))) { + W window = activeWindowManager.getWindow(); + if (windowSet.contains(window)) { + windowSet.markCompleted(window); + } } } } @@ -228,7 +293,8 @@ public String toString() { } /** - * The {@link Reiterator} used by {@link WindowReiterable}. + * The {@link Reiterator} used by + * {@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn.WindowReiterable}. */ private static class WindowReiterator implements Reiterator { private PeekingReiterator> iterator; @@ -322,7 +388,7 @@ public void remove() { @Override public Reiterator copy() { - return new ListReiterator(list, index); + return new ListReiterator(list, index); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java index 464fcf9b868a7..c3482f074fd74 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFnTest.java @@ -253,8 +253,7 @@ KV>, List> makeRunner( WindowFn windowFn) { GroupAlsoByWindowsDoFn, IntervalWindow> fn = - new GroupAlsoByWindowsDoFn, IntervalWindow>( - windowFn, null, StringUtf8Coder.of()); + GroupAlsoByWindowsDoFn.create(windowFn, StringUtf8Coder.of()); DoFnRunner>>, KV>, List> runner = From 241e58bda1df52d983666d78b4356294aa9b7f7a Mon Sep 17 00:00:00 2001 From: davor Date: Mon, 9 Mar 2015 15:27:58 -0700 Subject: [PATCH 0244/1541] Temporarily disable the unit test. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88158689 --- .../com/google/cloud/dataflow/sdk/coders/Proto2CoderTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/Proto2CoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/Proto2CoderTest.java index 11b5f6e85b7c9..bd9535f1f11c0 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/Proto2CoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/Proto2CoderTest.java @@ -20,6 +20,7 @@ import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageB; import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageC; +import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -27,6 +28,7 @@ /** * Tests for Proto2Coder. */ +@Ignore("Enable when we fix interal build process") @RunWith(JUnit4.class) public class Proto2CoderTest { From 783a5fa507460d9acce98d255365b536cfa7c5fb Mon Sep 17 00:00:00 2001 From: davor Date: Mon, 9 Mar 2015 15:42:19 -0700 Subject: [PATCH 0245/1541] String scrubbing. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88160076 --- .../cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java | 4 ++-- .../main/java/com/google/cloud/dataflow/sdk/util/Structs.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java index f01d5ffd62db6..cf2e87452308c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java @@ -55,8 +55,8 @@ public class RetryHttpRequestInitializer implements HttpRequestInitializer { * Http response codes that should be silently ignored. */ private static final Set DEFAULT_IGNORED_RESPONSE_CODES = new HashSet<>( - Arrays.asList(307 /* Redirect, handled by Apiary client */, - 308 /* Resume Incomplete, handled by Apiary client */)); + Arrays.asList(307 /* Redirect, handled by the client library */, + 308 /* Resume Incomplete, handled by the client library */)); /** * Http response timeout to use for hanging gets. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Structs.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Structs.java index eab606c8a214a..25d72cbbf73a4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Structs.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Structs.java @@ -59,7 +59,7 @@ public static byte[] getBytes(Map map, String name, @Nullable by return defaultValue; } // TODO: Need to agree on a format for encoding bytes in - // a string that can be sent over the Apiary wire, over the cloud + // a string that can be sent to the backend, over the cloud // map task work API. base64 encoding seems pretty common. Switch to it? return StringUtils.jsonStringToByteArray(jsonString); } From b14a07043797b3cadad499008aaf1f0100567eae Mon Sep 17 00:00:00 2001 From: Magnus Runesson Date: Tue, 10 Mar 2015 09:42:25 +0100 Subject: [PATCH 0246/1541] Fix spelling errors and adjusted copyright headers. --- contrib/join-library/README.md | 10 +++++----- contrib/join-library/pom.xml | 4 ++-- .../cloud/dataflow/contrib/joinlibrary/Join.java | 2 +- .../dataflow/contrib/joinlibrary/InnerJoinTest.java | 2 +- .../contrib/joinlibrary/OuterLeftJoinTest.java | 2 +- .../contrib/joinlibrary/OuterRightJoinTest.java | 2 +- 6 files changed, 11 insertions(+), 11 deletions(-) diff --git a/contrib/join-library/README.md b/contrib/join-library/README.md index b5766df50c6fd..47d85a18a4c1c 100644 --- a/contrib/join-library/README.md +++ b/contrib/join-library/README.md @@ -1,14 +1,14 @@ Join-library ============ -Join-library provide inner join, outer left and right join functions to -Google DataFlow. The aim is to simplify the most common cases of join to a +Join-library provides inner join, outer left and right join functions to +Google Cloud Dataflow. The aim is to simplify the most common cases of join to a simple function call. The functions are generic so it supports join of any types supported by -DataFlow. Input to the join functions are PCollections of Key/Values. Both the +Dataflow. Input to the join functions are PCollections of Key/Values. Both the left and right PCollections need the same type for the key. All the join -functions returns a Key/Value where Key is the join key and value is +functions return a Key/Value where Key is the join key and value is a Key/Value where the key is the left value and right is the value. In the cases of outer join, since null cannot be serialized the user have @@ -22,4 +22,4 @@ Example how to use join-library: PCollection>> joinedPcollection = Join.innerJoin(leftPcollection, rightPcollection); -Questions or comments: +Questions or comments: diff --git a/contrib/join-library/pom.xml b/contrib/join-library/pom.xml index 2ae8bad3a2301..263efbe58722d 100644 --- a/contrib/join-library/pom.xml +++ b/contrib/join-library/pom.xml @@ -6,9 +6,9 @@ 4.0.0 com.google.cloud.dataflow - join-library + google-cloud-dataflow-java-contrib-join-library Join library - Library with generic join functions for DataFlow. + Library with generic join functions for Dataflow. 0.0.2-SNAPSHOT jar diff --git a/contrib/join-library/src/main/java/com/google/cloud/dataflow/contrib/joinlibrary/Join.java b/contrib/join-library/src/main/java/com/google/cloud/dataflow/contrib/joinlibrary/Join.java index e13f9747c7942..9b550f82b6a31 100644 --- a/contrib/join-library/src/main/java/com/google/cloud/dataflow/contrib/joinlibrary/Join.java +++ b/contrib/join-library/src/main/java/com/google/cloud/dataflow/contrib/joinlibrary/Join.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2014 Google Inc. + * Copyright (C) 2015 The Google Cloud Dataflow Authors * * Licensed 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 diff --git a/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/InnerJoinTest.java b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/InnerJoinTest.java index c031cba7956a4..bdd3b46a5dff0 100644 --- a/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/InnerJoinTest.java +++ b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/InnerJoinTest.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2014 Google Inc. + * Copyright (C) 2015 The Google Cloud Dataflow Authors * * Licensed 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 diff --git a/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterLeftJoinTest.java b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterLeftJoinTest.java index c342ef706b0e1..e26f9b13e94a8 100644 --- a/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterLeftJoinTest.java +++ b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterLeftJoinTest.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2014 Google Inc. + * Copyright (C) 2015 The Google Cloud Dataflow Authors * * Licensed 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 diff --git a/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterRightJoinTest.java b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterRightJoinTest.java index 23bdf301dc90a..b0b4c81ecd58d 100644 --- a/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterRightJoinTest.java +++ b/contrib/join-library/src/test/java/com/google/cloud/dataflow/contrib/joinlibrary/OuterRightJoinTest.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2014 Google Inc. + * Copyright (C) 2015 The Google Cloud Dataflow Authors * * Licensed 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 From c8c305cc4cdeebbe4a4ba84071a4963eceaa121c Mon Sep 17 00:00:00 2001 From: Magnus Runesson Date: Tue, 10 Mar 2015 09:43:10 +0100 Subject: [PATCH 0247/1541] Modify checkstyle to accept new copyright header. As discussed in https://github.com/GoogleCloudPlatform/DataflowJavaSDK/pull/10 With external contributors the copyright header should be: Copyright (C) 2015 The Google Cloud Dataflow Authors Old copyright header is still accepted. --- checkstyle.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/checkstyle.xml b/checkstyle.xml index 3e79c3d48028c..d2a302c701bc1 100644 --- a/checkstyle.xml +++ b/checkstyle.xml @@ -36,7 +36,7 @@ page at http://checkstyle.sourceforge.net/config.html --> --> + value="^(//| \*) Copyright (\([cC]\) )?[\d]{4}(\-[\d]{4})? (Google Inc\.|The Google Cloud Dataflow Authors).*$" /> From 3d4bd412f0823fccc992ced6a0e126ce1307e334 Mon Sep 17 00:00:00 2001 From: Magnus Runesson Date: Tue, 10 Mar 2015 09:52:26 +0100 Subject: [PATCH 0248/1541] Adjusted comments to the new copyright header and copyright in checkstyle.xml --- checkstyle.xml | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/checkstyle.xml b/checkstyle.xml index d2a302c701bc1..224f37c1b4d9e 100644 --- a/checkstyle.xml +++ b/checkstyle.xml @@ -1,6 +1,6 @@ value="^(//| \*) Copyright (\([cC]\) )?[\d]{4}(\-[\d]{4})? (Google Inc\.|The Google Cloud Dataflow Authors).*$" /> - + From 14be24766a715faf42a115b99ee63e4f44a6f66a Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Tue, 10 Mar 2015 11:14:24 -0700 Subject: [PATCH 0249/1541] Revert "Adjusted comments to the new copyright header and copyright in checkstyle.xml" This reverts commit 3d4bd412f0823fccc992ced6a0e126ce1307e334. --- checkstyle.xml | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/checkstyle.xml b/checkstyle.xml index 224f37c1b4d9e..d2a302c701bc1 100644 --- a/checkstyle.xml +++ b/checkstyle.xml @@ -1,6 +1,6 @@ value="^(//| \*) Copyright (\([cC]\) )?[\d]{4}(\-[\d]{4})? (Google Inc\.|The Google Cloud Dataflow Authors).*$" /> - + From e9b3c96957a31c5d38cad87ce186e633eba50218 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Tue, 10 Mar 2015 11:46:50 -0700 Subject: [PATCH 0250/1541] Addition of AUTHORS.md. --- contrib/join-library/AUTHORS.md | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 contrib/join-library/AUTHORS.md diff --git a/contrib/join-library/AUTHORS.md b/contrib/join-library/AUTHORS.md new file mode 100644 index 0000000000000..331f927548c0b --- /dev/null +++ b/contrib/join-library/AUTHORS.md @@ -0,0 +1,6 @@ +# Authors of join-library + +The following is the official list of authors for copyright purposes of this community-contributed module. + + Google Inc. + Magnus Runesson, M.Runesson [at] gmail [dot] com From 6ba73cb47a54a72de6d1b4a556ef23fa98961b45 Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Tue, 10 Mar 2015 11:48:28 -0700 Subject: [PATCH 0251/1541] Minor fixes to README.md and pom.xml. --- contrib/join-library/README.md | 2 +- contrib/join-library/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/join-library/README.md b/contrib/join-library/README.md index 47d85a18a4c1c..a6e0b73d6c2e2 100644 --- a/contrib/join-library/README.md +++ b/contrib/join-library/README.md @@ -22,4 +22,4 @@ Example how to use join-library: PCollection>> joinedPcollection = Join.innerJoin(leftPcollection, rightPcollection); -Questions or comments: +Questions or comments: `M.Runesson [at] gmail [dot] com` diff --git a/contrib/join-library/pom.xml b/contrib/join-library/pom.xml index 263efbe58722d..5e843a20c9f74 100644 --- a/contrib/join-library/pom.xml +++ b/contrib/join-library/pom.xml @@ -6,7 +6,7 @@ 4.0.0 com.google.cloud.dataflow - google-cloud-dataflow-java-contrib-join-library + google-cloud-dataflow-java-contrib-joinlibrary Join library Library with generic join functions for Dataflow. 0.0.2-SNAPSHOT From ea56a5fe5c80281ef5e0d42d28671218d3d99794 Mon Sep 17 00:00:00 2001 From: davor Date: Mon, 9 Mar 2015 20:16:23 -0700 Subject: [PATCH 0252/1541] Fix parts of flakiness in RateLimitingTest. On a fast-enough machine, numFailures could be equal to DEFAULT_MAX_PARALLELISM. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88180618 --- .../google/cloud/dataflow/sdk/transforms/RateLimitingTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java index 377e533fc039b..14eeff96b937c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/RateLimitingTest.java @@ -21,6 +21,7 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.lessThan; +import static org.hamcrest.Matchers.lessThanOrEqualTo; import com.google.cloud.dataflow.sdk.TestUtils; import com.google.cloud.dataflow.sdk.runners.DirectPipeline; @@ -153,7 +154,7 @@ public void testExceptionHandling() { // The first failure should prevent the scheduling of any more elements. Assert.assertThat(fn.numFailures.get(), is(both(greaterThanOrEqualTo(1)) - .and(lessThan(RateLimiting.DEFAULT_MAX_PARALLELISM)))); + .and(lessThanOrEqualTo(RateLimiting.DEFAULT_MAX_PARALLELISM)))); } /** From ca3ffef4d93b1dacdca94898844c2a0e15542d12 Mon Sep 17 00:00:00 2001 From: robertwb Date: Tue, 10 Mar 2015 10:19:44 -0700 Subject: [PATCH 0253/1541] Allow arbitrarily large CoGroupByKey results by only caching the first 10K elements in memory. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88231835 --- .../sdk/transforms/join/CoGbkResult.java | 233 +++++++++++------- .../sdk/transforms/join/CoGbkResultTest.java | 109 ++++++++ 2 files changed, 258 insertions(+), 84 deletions(-) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java index 14121d649aa06..42a3e6e75d8d9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResult.java @@ -21,17 +21,21 @@ import com.google.api.client.util.Preconditions; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderException; -import com.google.cloud.dataflow.sdk.coders.ListCoder; +import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.coders.MapCoder; import com.google.cloud.dataflow.sdk.coders.StandardCoder; -import com.google.cloud.dataflow.sdk.coders.VarIntCoder; import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.cloud.dataflow.sdk.util.common.Reiterator; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.cloud.dataflow.sdk.values.TupleTagList; +import com.google.common.collect.Iterators; +import com.google.common.collect.PeekingIterator; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.io.InputStream; @@ -40,41 +44,60 @@ import java.util.Arrays; import java.util.Iterator; import java.util.List; -import java.util.Map; -import java.util.TreeMap; /** * A row result of a CoGroupByKey. This is a tuple of Iterables produced for * a given key, and these can be accessed in different ways. */ public class CoGbkResult { - // TODO: If we keep this representation for any amount of time, - // optimize it so that the union tag does not have to be repeated in the - // values stored under the union tag key. /** * A map of integer union tags to a list of union objects. * Note: the key and the embedded union tag are the same, so it is redundant * to store it multiple times, but for now it makes encoding easier. */ - private final Map> valueMap; + private final List> valueMap; private final CoGbkResultSchema schema; + private static final int DEFAULT_IN_MEMORY_ELEMENT_COUNT = 10_000; + + private static final Logger LOG = LoggerFactory.getLogger(CoGbkResult.class); + /** * A row in the PCollection resulting from a CoGroupByKey transform. * Currently, this row must fit into memory. * * @param schema the set of tuple tags used to refer to input tables and * result values - * @param values the raw results from a group-by-key + * @param taggedValues the raw results from a group-by-key */ + public CoGbkResult( + CoGbkResultSchema schema, + Iterable taggedValues) { + this(schema, taggedValues, DEFAULT_IN_MEMORY_ELEMENT_COUNT); + } + @SuppressWarnings("unchecked") public CoGbkResult( CoGbkResultSchema schema, - Iterable values) { + Iterable taggedValues, + int inMemoryElementCount) { this.schema = schema; - valueMap = new TreeMap<>(); - for (RawUnionValue value : values) { + valueMap = new ArrayList<>(); + for (int unionTag = 0; unionTag < schema.size(); unionTag++) { + valueMap.add(new ArrayList<>()); + } + + // Demultiplex the first imMemoryElementCount tagged union values + // according to their tag. + final Iterator taggedIter = taggedValues.iterator(); + int elementCount = 0; + while (taggedIter.hasNext()) { + if (elementCount++ >= inMemoryElementCount && taggedIter instanceof Reiterator) { + // Let the tails be lazy. + break; + } + RawUnionValue value = taggedIter.next(); // Make sure the given union tag has a corresponding tuple tag in the // schema. int unionTag = value.getUnionTag(); @@ -82,17 +105,48 @@ public CoGbkResult( throw new IllegalStateException("union tag " + unionTag + " has no corresponding tuple tag in the result schema"); } - List taggedValueList = valueMap.get(unionTag); - if (taggedValueList == null) { - taggedValueList = new ArrayList<>(); - valueMap.put(unionTag, taggedValueList); + List valueList = (List) valueMap.get(unionTag); + valueList.add(value.getValue()); + } + + if (taggedIter.hasNext()) { + // If we get here, there were more elements than we can afford to + // keep in memory, so we copy the re-iterable of remaining items + // and append filtered views to each of the sorted lists computed earlier. + LOG.info("CoGbkResult has more than " + inMemoryElementCount + " elements," + + "reiteration (which may be slow) is required."); + final Reiterator tail = (Reiterator) taggedIter; + for (int unionTag = 0; unionTag < schema.size(); unionTag++) { + final int unionTag0 = unionTag; + final Iterable head = valueMap.get(unionTag); + // This is a trinary-state array recording whether a given tag is + // present in the tail. The inital value is null (unknown) for all + // tags, and the first iteration through the entire list will set + // these values to true or false to avoid needlessly iterating if + // filtering against a given tag would not match anything. + final Boolean[] containsTag = new Boolean[schema.size()]; + valueMap.set( + unionTag, + new Iterable() { + Reiterator start = tail.copy(); + @Override + public Iterator iterator() { + return Iterators.concat( + head.iterator(), + new UnionValueIterator<>(unionTag0, tail.copy(), containsTag)); + } + }); } - taggedValueList.add(value); } } public boolean isEmpty() { - return valueMap == null || valueMap.isEmpty(); + for (Iterable tagValues : valueMap) { + if (tagValues.iterator().hasNext()) { + return false; + } + } + return true; } /** @@ -118,11 +172,9 @@ public Iterable getAll(TupleTag tag) { throw new IllegalArgumentException("TupleTag " + tag + " is not in the schema"); } - List unions = valueMap.get(index); - if (unions == null) { - return buildEmptyIterable(tag); - } - return new UnionValueIterable<>(unions); + @SuppressWarnings("unchecked") + Iterable unions = (Iterable) valueMap.get(index); + return unions; } /** @@ -149,7 +201,8 @@ public V getOnly(TupleTag tag, V defaultValue) { public static class CoGbkResultCoder extends StandardCoder { private final CoGbkResultSchema schema; - private final MapCoder> mapCoder; + private final UnionCoder unionCoder; + private MapCoder> mapCoder; /** * Returns a CoGbkResultCoder for the given schema and unionCoder. @@ -167,22 +220,14 @@ public static CoGbkResultCoder of( @JsonProperty(PropertyNames.CO_GBK_RESULT_SCHEMA) CoGbkResultSchema schema) { Preconditions.checkArgument(components.size() == 1, "Expecting 1 component, got " + components.size()); - return new CoGbkResultCoder(schema, (MapCoder) components.get(0)); + return new CoGbkResultCoder(schema, (UnionCoder) components.get(0)); } private CoGbkResultCoder( CoGbkResultSchema tupleTags, UnionCoder unionCoder) { this.schema = tupleTags; - this.mapCoder = MapCoder.of(VarIntCoder.of(), - ListCoder.of(unionCoder)); - } - - private CoGbkResultCoder( - CoGbkResultSchema tupleTags, - MapCoder mapCoder) { - this.schema = tupleTags; - this.mapCoder = mapCoder; + this.unionCoder = unionCoder; } @@ -193,7 +238,7 @@ public List> getCoderArguments() { @Override public List> getComponents() { - return Arrays.>asList(mapCoder); + return Arrays.>asList(unionCoder); } @Override @@ -204,6 +249,7 @@ public CloudObject asCloudObject() { } @Override + @SuppressWarnings("unchecked") public void encode( CoGbkResult value, OutputStream outStream, @@ -212,7 +258,9 @@ public void encode( if (!schema.equals(value.getSchema())) { throw new CoderException("input schema does not match coder schema"); } - mapCoder.encode(value.valueMap, outStream, context); + for (int unionTag = 0; unionTag < schema.size(); unionTag++) { + tagListCoder(unionTag).encode(value.valueMap.get(unionTag), outStream, Context.NESTED); + } } @Override @@ -220,9 +268,16 @@ public CoGbkResult decode( InputStream inStream, Context context) throws CoderException, IOException { - Map> map = mapCoder.decode( - inStream, context); - return new CoGbkResult(schema, map); + List> valueMap = new ArrayList<>(); + for (int unionTag = 0; unionTag < schema.size(); unionTag++) { + valueMap.add(tagListCoder(unionTag).decode(inStream, Context.NESTED)); + } + return new CoGbkResult(schema, valueMap); + } + + @SuppressWarnings("rawtypes") + private IterableCoder tagListCoder(int unionTag) { + return IterableCoder.of(unionCoder.getComponents().get(unionTag)); } @Override @@ -267,9 +322,8 @@ public CoGbkResult and(TupleTag tag, List data) { "Attempting to call and() on a CoGbkResult apparently not created by" + " of()."); } - Map> valueMap = new TreeMap<>(this.valueMap); - valueMap.put(nextTestUnionId, - convertValueListToUnionList(nextTestUnionId, data)); + List> valueMap = new ArrayList<>(this.valueMap); + valueMap.add(data); return new CoGbkResult( new CoGbkResultSchema(schema.getTupleTagList().and(tag)), valueMap, nextTestUnionId + 1); @@ -280,7 +334,7 @@ public CoGbkResult and(TupleTag tag, List data) { */ public static CoGbkResult empty() { return new CoGbkResult(new CoGbkResultSchema(TupleTagList.empty()), - new TreeMap>()); + new ArrayList>()); } ////////////////////////////////////////////////////////////////////////////// @@ -289,7 +343,7 @@ public static CoGbkResult empty() { private CoGbkResult( CoGbkResultSchema schema, - Map> valueMap, + List> valueMap, int nextTestUnionId) { this(schema, valueMap); this.nextTestUnionId = nextTestUnionId; @@ -297,24 +351,11 @@ private CoGbkResult( private CoGbkResult( CoGbkResultSchema schema, - Map> valueMap) { + List> valueMap) { this.schema = schema; this.valueMap = valueMap; } - private static List convertValueListToUnionList( - int unionTag, List data) { - List unionList = new ArrayList<>(); - for (V value : data) { - unionList.add(new RawUnionValue(unionTag, value)); - } - return unionList; - } - - private Iterable buildEmptyIterable(TupleTag tag) { - return new ArrayList<>(); - } - private V innerGetOnly( TupleTag tag, V defaultValue, @@ -324,8 +365,9 @@ private V innerGetOnly( throw new IllegalArgumentException("TupleTag " + tag + " is not in the schema"); } - List unions = valueMap.get(index); - if (unions == null || unions.isEmpty()) { + @SuppressWarnings("unchecked") + Iterator unions = (Iterator) valueMap.get(index).iterator(); + if (!unions.hasNext()) { if (useDefault) { return defaultValue; } else { @@ -333,44 +375,67 @@ private V innerGetOnly( + " corresponds to an empty result, and no default was provided"); } } - if (unions.size() != 1) { + V value = unions.next(); + if (unions.hasNext()) { throw new IllegalArgumentException("TupleTag " + tag - + " corresponds to a non-singleton result of size " + unions.size()); + + " corresponds to a non-singleton result"); } - return (V) unions.get(0).getValue(); + return value; } /** - * Lazily converts and recasts an {@code Iterable} into an - * {@code Iterable}, where V is the type of the raw union value's contents. + * Lazily filters and recasts an {@code Iterator} into an + * {@code Iterator}, where V is the type of the raw union value's contents. */ - private static class UnionValueIterable implements Iterable { + private static class UnionValueIterator implements Iterator { - private final Iterable unions; + private final int tag; + private final PeekingIterator unions; + private final Boolean[] containsTag; - private UnionValueIterable(Iterable unions) { - this.unions = unions; + private UnionValueIterator(int tag, Iterator unions, Boolean[] containsTag) { + this.tag = tag; + this.unions = Iterators.peekingIterator(unions); + this.containsTag = containsTag; } @Override - public Iterator iterator() { - final Iterator unionsIterator = unions.iterator(); - return new Iterator() { - @Override - public boolean hasNext() { - return unionsIterator.hasNext(); + public boolean hasNext() { + if (containsTag[tag] == Boolean.FALSE) { + return false; + } + advance(); + if (unions.hasNext()) { + return true; + } else { + // We can now resolve all the "unknown" null values. + for (int i = 0; i < containsTag.length; i++) { + if (containsTag[i] == null) { + containsTag[i] = false; + } } + return false; + } + } - @Override - public V next() { - return (V) unionsIterator.next().getValue(); - } + @Override + @SuppressWarnings("unchecked") + public V next() { + advance(); + return (V) unions.next().getValue(); + } - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }; + private void advance() { + int curTag; + while (unions.hasNext() && (curTag = unions.peek().getUnionTag()) != tag) { + containsTag[curTag] = true; + unions.next(); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); } } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultTest.java new file mode 100644 index 0000000000000..c71439aa351de --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/join/CoGbkResultTest.java @@ -0,0 +1,109 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.join; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.emptyIterable; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +import com.google.cloud.dataflow.sdk.util.common.Reiterable; +import com.google.cloud.dataflow.sdk.util.common.Reiterator; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TupleTagList; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.List; + +/** + * Tests the CoGbkResult. + */ +@RunWith(JUnit4.class) +public class CoGbkResultTest { + + @Test + public void testLazyResults() { + runLazyResult(0); + runLazyResult(1); + runLazyResult(3); + runLazyResult(10); + } + + public void runLazyResult(int cacheSize) { + int valueLen = 7; + TestUnionValues values = new TestUnionValues(0, 1, 0, 3, 0, 3, 3); + CoGbkResult result = new CoGbkResult(createSchema(5), values, cacheSize); + assertThat(values.maxPos(), equalTo(Math.min(cacheSize, valueLen))); + assertThat(result.getAll(new TupleTag("tag0")), contains(0, 2, 4)); + assertThat(values.maxPos(), equalTo(valueLen)); + assertThat(result.getAll(new TupleTag("tag3")), contains(3, 5, 6)); + assertThat(result.getAll(new TupleTag("tag2")), emptyIterable()); + assertThat(result.getOnly(new TupleTag("tag1")), equalTo(1)); + } + + private CoGbkResultSchema createSchema(int size) { + List> tags = new ArrayList<>(); + for (int i = 0; i < size; i++) { + tags.add(new TupleTag("tag" + i)); + } + return new CoGbkResultSchema(TupleTagList.of(tags)); + } + + private static class TestUnionValues implements Reiterable { + + final int[] tags; + int maxPos = 0; + + /** + * This will create a list of RawUnionValues whose tags are as given and + * values are increasing starting at 0 (i.e. the index in the constructor). + */ + public TestUnionValues(int... tags) { + this.tags = tags; + } + + /** + * Returns the highest position iterated to so far, useful for ensuring + * laziness. + */ + public int maxPos() { + return maxPos; + } + + @Override + public Reiterator iterator() { + return iterator(0); + } + + public Reiterator iterator(final int start) { + return new Reiterator() { + int pos = start; + public boolean hasNext() { return pos < tags.length; } + public RawUnionValue next() { + maxPos = Math.max(pos + 1, maxPos); + return new RawUnionValue(tags[pos], pos++); + } + public void remove() { throw new UnsupportedOperationException(); } + public Reiterator copy() { return iterator(pos); } + }; + } + } +} From 29d3312483540ac3ceb42b7e995d5caba5edc49c Mon Sep 17 00:00:00 2001 From: jlewi Date: Tue, 10 Mar 2015 11:46:33 -0700 Subject: [PATCH 0254/1541] Fix the javadoc for Count transforms. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88241545 --- .../cloud/dataflow/sdk/transforms/Count.java | 32 +++---------------- 1 file changed, 4 insertions(+), 28 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java index 87e6a1c5e10b8..2422483ac69aa 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java @@ -20,34 +20,10 @@ import com.google.cloud.dataflow.sdk.values.PCollection; /** - * {@code Count} takes a {@code PCollection} and returns a - * {@code PCollection>} representing a map from each - * distinct element of the input {@code PCollection} to the number of times - * that element occurs in the input. Each of the keys in the output - * {@code PCollection} is unique. - * - *

    Two values of type {@code T} are compared for equality not by - * regular Java {@link Object#equals}, but instead by first encoding - * each of the elements using the {@code PCollection}'s {@code Coder}, and then - * comparing the encoded bytes. This admits efficient parallel - * evaluation. - * - *

    By default, the {@code Coder} of the keys of the output - * {@code PCollection} is the same as the {@code Coder} of the - * elements of the input {@code PCollection}. - * - *

    Each output element is in the window by which its corresponding input - * was grouped, and has the timestamp of the end of that window. The output - * {@code PCollection} has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} - * as the input. - * - *

    Example of use: - *

     {@code
    - * PCollection words = ...;
    - * PCollection> wordCounts =
    - *     words.apply(Count.perElement());
    - * } 
    + * Count transforms can be used to count the number of elements in a PCollection. + * {@link PerElement Count.PerElement can be used to count the number of occurrences of each + * distinct element in the PCollection. {@link Globally Count.Globally} can + * be used to count the total number of elements in a PCollection. */ public class Count { From a60e1ac94cdd027b219f744dcd3e8ddad509dfab Mon Sep 17 00:00:00 2001 From: chernyak Date: Tue, 10 Mar 2015 12:09:12 -0700 Subject: [PATCH 0255/1541] Updates to Windmill API for streaming side inputs: break the GlobalDataId into an explicit tag and version field ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88243950 --- sdk/src/main/proto/windmill.proto | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/sdk/src/main/proto/windmill.proto b/sdk/src/main/proto/windmill.proto index a8d856766f83f..64d6d18a40044 100644 --- a/sdk/src/main/proto/windmill.proto +++ b/sdk/src/main/proto/windmill.proto @@ -81,8 +81,13 @@ message TagList { repeated Value values = 3; } +message GlobalDataId { + required string tag = 1; + required bytes version = 2; +} + message GlobalData { - required bytes data_id = 1; + required GlobalDataId data_id = 1; optional bool is_ready = 2; optional bytes data = 3; } @@ -93,7 +98,7 @@ message WorkItem { repeated InputMessageBundle message_bundles = 3; optional TimerBundle timers = 4; - repeated bytes global_data_ids_ready = 5; + repeated GlobalDataId global_data_id_notifications = 5; } message ComputationWorkItems { @@ -122,7 +127,7 @@ message KeyedGetDataRequest { required fixed64 work_token = 2; repeated TagValue values_to_fetch = 3; repeated TagList lists_to_fetch = 4; - repeated GlobalData global_data_to_fetch = 5; + repeated GlobalDataId global_data_to_fetch = 5; } message ComputationGetDataRequest { @@ -183,7 +188,7 @@ message WorkItemCommitRequest { repeated TagValue value_updates = 5; repeated TagList list_updates = 6; repeated Counter counter_updates = 8; - repeated bytes blocked_on_global_data_ids = 9; + repeated GlobalDataId global_data_id_requests = 9; } message ComputationCommitWorkRequest { From f1afe86d2c11edc1044275ae1996310861107cd6 Mon Sep 17 00:00:00 2001 From: bchambers Date: Tue, 10 Mar 2015 15:34:36 -0700 Subject: [PATCH 0256/1541] Fix Javadoc links. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88265230 --- .../com/google/cloud/dataflow/sdk/Pipeline.java | 6 +++--- .../cloud/dataflow/sdk/coders/DelegateCoder.java | 4 ++-- .../dataflow/sdk/coders/StringDelegateCoder.java | 2 +- .../google/cloud/dataflow/sdk/io/BigQueryIO.java | 3 ++- .../google/cloud/dataflow/sdk/io/DatastoreIO.java | 3 ++- .../dataflow/sdk/options/PipelineOptions.java | 9 +++++---- .../sdk/options/PipelineOptionsRegistrar.java | 8 ++++---- .../sdk/runners/DirectPipelineRunner.java | 5 +++-- .../sdk/runners/PipelineRunnerRegistrar.java | 12 ++++++------ .../dataflow/sdk/testing/DataflowAssert.java | 15 ++++++++------- .../sdk/transforms/ApproximateQuantiles.java | 4 ++-- .../cloud/dataflow/sdk/transforms/Combine.java | 7 ++++--- .../cloud/dataflow/sdk/transforms/DoFn.java | 8 ++++---- .../cloud/dataflow/sdk/transforms/Flatten.java | 8 ++++---- .../cloud/dataflow/sdk/transforms/GroupByKey.java | 2 +- .../cloud/dataflow/sdk/transforms/ParDo.java | 2 +- .../dataflow/sdk/transforms/RateLimiting.java | 3 +-- .../cloud/dataflow/sdk/transforms/View.java | 14 +++++++------- .../dataflow/sdk/values/PCollectionView.java | 10 ++++++---- .../google/cloud/dataflow/sdk/values/POutput.java | 5 ++--- 20 files changed, 68 insertions(+), 62 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java index 23405801827ee..dade39c8e9bc5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java @@ -50,8 +50,8 @@ * and concurrently. * *

    Each {@code Pipeline} is self-contained and isolated from any other - * {@code Pipeline}. The {@link PValues} that are inputs and outputs of each of a - * {@code Pipeline}'s {@link PTransform}s are also owned by that {@code Pipeline}. + * {@code Pipeline}. The {@link PValue PValues} that are inputs and outputs of each of a + * {@code Pipeline}'s {@link PTransform PTransforms} are also owned by that {@code Pipeline}. * A {@code PValue} owned by one {@code Pipeline} can be read only by {@code PTransform}s * also owned by that {@code Pipeline}. * @@ -279,7 +279,7 @@ Output applyInternal(Input input, } /** - * Returns all producing transforms for the {@link PValue}s contained + * Returns all producing transforms for the {@link PValue PValues} contained * in {@code output}. */ private List> getProducingTransforms(POutput output) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java index ae2c5e27e3b27..89e996c510a60 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java @@ -22,10 +22,10 @@ import java.io.Serializable; /** - * A {@code DelegateCoder} wraps a {@link Coder Coder

    } and + * A {@code DelegateCoder} wraps a {@link Coder Coder<DT>} and * encodes/decodes values of type {@code T}s by converting * to/from {@code DT} and then encoding/decoding using the underlying - * {@link Coder Coder
    }. + * {@link Coder Coder<DT>}. * *

    The conversions from {@code T} to {@code DT} and vice versa * must be supplied as {@link CodingFunction}, a serializable diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoder.java index 567c94a3100b6..139dfba83a6d2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/StringDelegateCoder.java @@ -19,7 +19,7 @@ import java.lang.reflect.InvocationTargetException; /** - * A {@code StringDelegateCoder} wraps a {@link Coder} + * A {@code StringDelegateCoder} wraps a {@code Coder} * and encodes/decodes values of type {@code T} via string representations. * *

    To decode, the input byte stream is decoded to diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java index 061bf68cd5049..cf89d23c5134e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java @@ -69,7 +69,8 @@ * *

    * BigQuery table references are stored as a {@link TableReference}, which comes - * from the BigQuery Java Client API. + * from the + * BigQuery Java Client API. * Tables can be referred to as Strings, with or without the {@code projectId}. * A helper function is provided ({@link BigQueryIO#parseTableSpec(String)}), * which parses the following string forms into a {@link TableReference}: diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java index 8a85429287c08..64ca240d66823 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java @@ -367,7 +367,8 @@ public static Sink writeTo(String datasetId) { * entities to a Datastore kind. * *

    Current version only supports Write operation running on - * {@link DirectPipelineRunner}. If Write is used on {@link DataflowPipelineRunner}, + * {@link DirectPipelineRunner}. If Write is used on + * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}, * it throws {@link UnsupportedOperationException} and won't continue on the * operation. * diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java index 3f9515f4cee28..1fb5d9aabb63c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java @@ -43,11 +43,12 @@ @JsonDeserialize(using = Deserializer.class) public interface PipelineOptions { /** - * Transforms this object into an object of type . must extend {@link PipelineOptions}. + * Transforms this object into an object of type {@code }. {@code } must extend + * {@link PipelineOptions}. *

    - * If is not registered with the {@link PipelineOptionsFactory}, then we attempt to - * verify that is composable with every interface that this instance of the PipelineOptions - * has seen. + * If {@code } is not registered with the {@link PipelineOptionsFactory}, then we + * attempt to verify that {@code } is composable with every interface that this + * instance of the {@code PipelineOptions} has seen. * * @param kls The class of the type to transform to. * @return An object of type kls. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsRegistrar.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsRegistrar.java index 6e0b60e58f397..d23fe682ddb14 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsRegistrar.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsRegistrar.java @@ -16,8 +16,6 @@ package com.google.cloud.dataflow.sdk.options; -import com.google.auto.service.AutoService; - import java.util.ServiceLoader; /** @@ -25,11 +23,13 @@ * {@link PipelineOptions} registered with this SDK by creating a {@link ServiceLoader} entry * and a concrete implementation of this interface. *

    - * Note that automatic registration of any {@PipelineOptions} requires users + * Note that automatic registration of any + * {@link com.google.cloud.dataflow.sdk.options.PipelineOptions} requires users * conform to the limitations discussed on {@link PipelineOptionsFactory#register(Class)}. *

    * 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. + * {@link com.google.auto.service.AutoService} to generate the necessary META-INF + * files automatically. */ public interface PipelineOptionsRegistrar { Iterable> getPipelineOptions(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java index 6c481d624701d..27e15304fadb7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DirectPipelineRunner.java @@ -372,8 +372,9 @@ public interface EvaluationResults extends PipelineResult { /** * Retrieves the values indicated by the given {@link PCollectionView}. - * Note that within the {@link DoFnContext} a {@link PCollectionView} - * converts from this representation to a suitable side input value. + * Note that within the {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context} + * implementation a {@link PCollectionView} should convert from this representation to a + * suitable side input value. */ Iterable> getPCollectionView(PCollectionView view); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerRegistrar.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerRegistrar.java index d63de01948aac..954bace372ce9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerRegistrar.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/PipelineRunnerRegistrar.java @@ -16,8 +16,6 @@ package com.google.cloud.dataflow.sdk.runners; -import com.google.auto.service.AutoService; - import java.util.ServiceLoader; /** @@ -25,12 +23,14 @@ * {@link PipelineRunner} registered with this SDK by creating a {@link ServiceLoader} entry * and a concrete implementation of this interface. *

    - * Note that automatic registration of any {@PipelineOptions} requires users - * conform to the limit that each {@link PipelineRunner}'s {@link Class#getSimpleName() simple name} - * must be unique. + * Note that automatic registration of any + * {@link com.google.cloud.dataflow.sdk.options.PipelineOptions} requires users + * conform to the limit that each {@link PipelineRunner}'s + * {@link Class#getSimpleName() simple name} must be unique. *

    * 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. + * {@link com.google.auto.service.AutoService} to generate the necessary + * META-INF files automatically. */ public interface PipelineRunnerRegistrar { public Iterable>> getPipelineRunners(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java index ee6152c04b04d..5d75312e9eb7a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/DataflowAssert.java @@ -77,7 +77,7 @@ private DataflowAssert() {} /** * Constructs an {@link IterableAssert} for the elements of the provided - * {@link PCollection PCollection}. + * {@link PCollection PCollection<T>}. */ public static IterableAssert that(PCollection actual) { return new IterableAssert<>(actual.apply(View.asIterable())) @@ -86,7 +86,8 @@ public static IterableAssert that(PCollection actual) { /** * Constructs an {@link IterableAssert} for the value of the provided - * {@link PCollection PCollection>}, which must be a singleton. + * {@link PCollection PCollection<Iterable<T>>}, which must be a + * singleton. */ public static IterableAssert thatSingletonIterable(PCollection> actual) { List> maybeElementCoder = actual.getCoder().getCoderArguments(); @@ -125,7 +126,7 @@ public static SingletonAssert thatSingleton(PCollection actual) { /** * An assertion about the contents of a - * {@link PCollectionView PCollectionView<, ?>}. + * {@link PCollectionView PCollectionView<Iterable<T>, ?>}. */ @SuppressWarnings("serial") public static class IterableAssert implements Serializable { @@ -247,7 +248,7 @@ public IterableAssert containsInOrder(Iterable expectedElements) { /** * An assertion about the single value of type {@code T} - * associated with a {@link PCollectionView PCollectionView}. + * associated with a {@link PCollectionView PCollectionView<T, ?>}. */ @SuppressWarnings("serial") public static class SingletonAssert implements Serializable { @@ -344,7 +345,7 @@ public SingletonAssert is(T expectedValue) { //////////////////////////////////////////////////////////////////////// /** - * An assertion checker that takes a single {@link PCollectionView PCollectionView} + * An assertion checker that takes a single {@link PCollectionView PCollectionView<A, ?>} * and an assertion over {@code A}, and checks it within a dataflow pipeline. * *

    Note that the entire assertion must be serializable. If @@ -382,8 +383,8 @@ public void processElement(ProcessContext c) { } /** - * An assertion checker that takes a {@link PCollectionView PCollectionView}, - * a {@link PCollectionView PCollectionView}, a relation + * An assertion checker that takes a {@link PCollectionView PCollectionView<A, ?>}, + * a {@link PCollectionView PCollectionView<B, ?>}, a relation * over {@code A} and {@code B}, and checks that the relation holds * within a dataflow pipeline. * diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java index bc9a7d99df340..80c80b539b34c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java @@ -192,10 +192,10 @@ PTransform, PCollection>> globally(int numQuantiles) { * *

    To evaluate the quantiles we use the "New Algorithm" described here: *

    -   *   [MRL98] Manku, Rajagopalan & Lindsay, "Approximate Medians and other
    +   *   [MRL98] Manku, Rajagopalan & Lindsay, "Approximate Medians and other
        *   Quantiles in One Pass and with Limited Memory", Proc. 1998 ACM
        *   SIGMOD, Vol 27, No 2, p 426-435, June 1998.
    -   *   http://citeseerx.ist.psu.edu/viewdoc/download?doi=10.1.1.6.6513&rep=rep1&type=pdf
    +   *   http://citeseerx.ist.psu.edu/viewdoc/download?doi=10.1.1.6.6513&rep=rep1&type=pdf
        * 
    * *

    The default error bound is {@code 1 / N}, though in practice diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java index 3e7f2895631df..5d51a3c17303d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.VarIntCoder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -728,8 +729,8 @@ public Coder getDefaultOutputCoder( * {@code Combine.Globally} takes a {@code PCollection} * and returns a {@code PCollection} whose single element is the result of * combining all the elements of the input {@code PCollection}, - * using a specified - * {@link CombineFn CombineFn}. It is common + * using a specified} + * {@link CombineFn CombineFn<VI, VA, VO>}. It is common * for {@code VI == VO}, but not required. Common combining * functions include sums, mins, maxes, and averages of numbers, * conjunctions and disjunctions of booleans, statistical @@ -1124,7 +1125,7 @@ protected String getKindString() { * {@code GroupedValues} takes a * {@code PCollection>>}, such as the result of * {@link GroupByKey}, applies a specified - * {@link KeyedCombineFn KeyedCombineFn} + * {@link KeyedCombineFn KeyedCombineFn<K, VI, VA, VO>} * to each of the input {@code KV>} elements to * produce a combined output {@code KV} element, and returns a * {@code PCollection>} containing all the combined output diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java index 3b71738ad7970..fdfa2672620e2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java @@ -81,7 +81,7 @@ public abstract class Context { * element will have the same timestamp and be in the same windows * as the input element passed to {@link DoFn#processElement}). * - *

    If invoked from {@link #startBundle} or {@link #finishValue}, + *

    If invoked from {@link #startBundle} or {@link #finishBundle}, * this will attempt to use the * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * of the input {@code PCollection} to determine what windows the element @@ -100,7 +100,7 @@ public abstract class Context { * {@link DoFn#getAllowedTimestampSkew}. The output element will * be in the same windows as the input element. * - *

    If invoked from {@link #startBundle} or {@link #finishValue}, + *

    If invoked from {@link #startBundle} or {@link #finishBundle}, * this will attempt to use the * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * of the input {@code PCollection} to determine what windows the element @@ -122,7 +122,7 @@ public abstract class Context { *

    The output element will have the same timestamp and be in the same * windows as the input element passed to {@link DoFn#processElement}). * - *

    If invoked from {@link #startBundle} or {@link #finishValue}, + *

    If invoked from {@link #startBundle} or {@link #finishBundle}, * this will attempt to use the * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * of the input {@code PCollection} to determine what windows the element @@ -145,7 +145,7 @@ public abstract class Context { * {@link DoFn#getAllowedTimestampSkew}. The output element will * be in the same windows as the input element. * - *

    If invoked from {@link #startBundle} or {@link #finishValue}, + *

    If invoked from {@link #startBundle} or {@link #finishBundle}, * this will attempt to use the * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * of the input {@code PCollection} to determine what windows the element diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java index c2a10d375b0d3..bce444d955158 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java @@ -52,8 +52,8 @@ public class Flatten { /** - * Returns a {@link PTransform} that flattens a {@link CollectionList} - * into a {@link PCollection} containing all the elements of all + * Returns a {@link PTransform} that flattens a {@link PCollectionList} + * into a {@link PCollection} containing all the elements of all * the {@link PCollection}s in its input. * *

    If any of the inputs to {@code Flatten} require window merging, @@ -98,8 +98,8 @@ public static FlattenIterables iterables() { } /** - * A {@link PTransform} that flattens a {@link PCollectionList} - * into a {@link PCollection} containing all the elements of all + * A {@link PTransform} that flattens a {@link PCollectionList} + * into a {@link PCollection} containing all the elements of all * the {@link PCollection}s in its input. * * @param the type of the elements in the input and output diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index 1f7024d4314b9..1483ceb7ec989 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -108,7 +108,7 @@ * have the same {@link WindowFn} as the input. * *

    If the input {@code PCollection} contains late data (see - * {@link com.google.cloud.dataflow.sdk.PubsubIO.Read.Bound#timestampLabel} + * {@link com.google.cloud.dataflow.sdk.io.PubsubIO.Read.Bound#timestampLabel} * for an example of how this can occur), then there may be multiple elements * output by a {@code GroupByKey} that correspond to the same key and window. * diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java index 234c5fb825d77..0b23080f0fcd0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java @@ -52,7 +52,7 @@ *

    {@code DoFn}s

    * *

    The function to use to process each element is specified by a - * {@link DoFn DoFn}. + * {@link DoFn DoFn<I, O>}. * *

    Conceptually, when a {@code ParDo} transform is executed, the * elements of the input {@code PCollection} are first divided up diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java index 76d6dbf3d0fd7..2c5a1ba500bbd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RateLimiting.java @@ -27,7 +27,6 @@ import com.google.common.util.concurrent.RateLimiter; import org.joda.time.Instant; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -110,7 +109,7 @@ public RateLimitingTransform(DoFn doFn) { *

    This rate limit may not be reachable unless there is sufficient * parallelism. * - *

    A rate of <= 0.0 disables rate limiting. + *

    A rate of ≤ 0.0 disables rate limiting. */ public RateLimitingTransform withRateLimit( double maxElementsPerSecond) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java index 4f7fcd543628e..f7ec7a6d40895 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/View.java @@ -73,7 +73,7 @@ public static AsIterable asIterable() { } /** - * Returns an {@link AsMultimap} that takes a {@link PCollection} as input + * Returns an {@link AsMultimap} that takes a {@link PCollection} as input * and produces a {@link PCollectionView} of the values to be consumed * as a {@code Map>} side input. * @@ -84,8 +84,8 @@ public static AsMultimap asMap() { } /** - * A {@PTransform} that produces a {@link PCollectionView} of a singleton {@link PCollection} - * yielding the single element it contains. + * A {@link PTransform} that produces a {@link PCollectionView} of a singleton + * {@link PCollection} yielding the single element it contains. * *

    Instantiate via {@link View#asIterable}. */ @@ -106,8 +106,8 @@ public PCollectionView, Iterable>> apply( } /** - * A {@PTransform} that produces a {@link PCollectionView} of a singleton {@link PCollection} - * yielding the single element it contains. + * A {@link PTransform} that produces a {@link PCollectionView} of a singleton + * {@link PCollection} yielding the single element it contains. * *

    Instantiate via {@link View#asIterable}. */ @@ -127,7 +127,7 @@ public PCollectionView> apply(PCollection input) { } /** - * A {@PTransform} that produces a {@link PCollectionView} of a keyed {@link PCollection} + * A {@link PTransform} that produces a {@link PCollectionView} of a keyed {@link PCollection} * yielding a map of keys to all associated values. * *

    Instantiate via {@link View#asMap}. @@ -166,7 +166,7 @@ public AsSingletonMap withCombiner(CombineFn combineFn) /** - * A {@PTransform} that produces a {@link PCollectionView} of a keyed {@link PCollection} + * A {@link PTransform} that produces a {@link PCollectionView} of a keyed {@link PCollection} * yielding a map of keys to a single associated values. * *

    Instantiate via {@link View#asMap}. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionView.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionView.java index 7eee0f4a94540..35d1e6a433cce 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionView.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionView.java @@ -23,11 +23,13 @@ /** * A {@code PCollectionView} is an immutable view of a * {@link PCollection} that can be accessed e.g. as a - * side input to a {@link DoFn}. + * side input to a {@link com.google.cloud.dataflow.sdk.transforms.DoFn}. * - *

    A {@PCollectionView} should always be the output of a {@link PTransform}. It is - * the joint responsibility of this transform and each {@link PipelineRunner} to - * implement the view in a runner-specific manner. + *

    A {@link PCollectionView} should always be the output of a + * {@link com.google.cloud.dataflow.sdk.transforms.PTransform}. It is the joint + * responsibility of this transform and each + * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} to implement the + * view in a runner-specific manner. * * @param the type of the value(s) accessible via this {@code PCollectionView} * @param the type of the windowed value(s) accessible via this {@code PCollectionView} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutput.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutput.java index 35842827facea..521a1a1ec34f5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutput.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutput.java @@ -45,9 +45,8 @@ public interface POutput { * {@code PTransform} in the given {@code Pipeline}. * *

    Should expand this {@code POutput} and invoke - * {@link PValue#recordAsOutput(Pipeline, - * com.google.cloud.dataflow.sdk.transforms.PTransform, - * String)} on each component output {@code PValue}. + * {@link PValue#recordAsOutput(Pipeline, com.google.cloud.dataflow.sdk.transforms.PTransform)} + * on each component output {@code PValue}. * *

    Automatically invoked as part of applying a * {@code PTransform}. Not to be invoked directly by user code. From e89f47f756d0729c1a4567064a58970a74b4ea98 Mon Sep 17 00:00:00 2001 From: malo Date: Tue, 10 Mar 2015 18:37:58 -0700 Subject: [PATCH 0257/1541] Change DataflowWorkerHarness to repeatedly request work. Switches to having 1 thread per core. Remove deprecated completion time logging. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88283022 --- .../runners/worker/DataflowWorkerHarness.java | 106 ++++++++++++------ .../worker/DataflowWorkerHarnessTest.java | 22 ++-- 2 files changed, 79 insertions(+), 49 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java index a078d754abb1f..106b4ec151224 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarness.java @@ -19,7 +19,9 @@ import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudTime; -import com.google.api.client.util.Lists; +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.Sleeper; import com.google.api.services.dataflow.Dataflow; import com.google.api.services.dataflow.model.LeaseWorkItemRequest; import com.google.api.services.dataflow.model.LeaseWorkItemResponse; @@ -32,6 +34,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingFormatter; import com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingInitializer; +import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff; import com.google.cloud.dataflow.sdk.util.GcsIOChannelFactory; import com.google.cloud.dataflow.sdk.util.IOChannelUtils; import com.google.cloud.dataflow.sdk.util.PropertyNames; @@ -39,19 +42,17 @@ import com.google.common.collect.ImmutableList; import org.joda.time.DateTime; -import org.joda.time.DateTimeUtils; import org.joda.time.Duration; -import org.joda.time.format.ISODateTimeFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.lang.Thread.UncaughtExceptionHandler; import java.util.Collections; +import java.util.LinkedList; import java.util.List; import java.util.concurrent.Callable; -import java.util.concurrent.CompletionService; -import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; import javax.annotation.concurrent.ThreadSafe; @@ -72,6 +73,10 @@ public class DataflowWorkerHarness { private static final String APPLICATION_NAME = "DataflowWorkerHarness"; + // ExponentialBackOff parameters for the task retry strategy. Visible for testing. + static final int BACKOFF_INITIAL_INTERVAL_MILLIS = 5000; // 5 second + static final int BACKOFF_MAX_ATTEMPTS = 10; // 10 attempts will take approx. 15 min. + /** * This uncaught exception handler logs the {@link Throwable} to the logger, {@link System#err} * and exits the application with status code 1. @@ -88,6 +93,14 @@ public void uncaughtException(Thread t, Throwable e) { } } + /** + * Helper for initializing the BackOff used for retries. + */ + private static BackOff createBackOff() { + return new AttemptBoundedExponentialBackOff( + BACKOFF_MAX_ATTEMPTS, BACKOFF_INITIAL_INTERVAL_MILLIS); + } + /** * Fetches and processes work units from the Dataflow service. */ @@ -99,47 +112,68 @@ public static void main(String[] args) throws Exception { PipelineOptionsFactory.createFromSystemProperties(); DataflowWorkerLoggingInitializer.configure(pipelineOptions); + final Sleeper sleeper = Sleeper.DEFAULT; final DataflowWorker worker = create(pipelineOptions); - processWork(pipelineOptions, worker); + processWork(pipelineOptions, worker, sleeper); } - // Visible for testing. - static void processWork(DataflowWorkerHarnessOptions pipelineOptions, - final DataflowWorker worker) { - - long startTime = DateTimeUtils.currentTimeMillis(); - int numThreads = Math.max(Runtime.getRuntime().availableProcessors() - 1, 1); - CompletionService completionService = - new ExecutorCompletionService<>(pipelineOptions.getExecutorService()); - for (int i = 0; i < numThreads; ++i) { - completionService.submit(new Callable() { - @Override - public Boolean call() throws Exception { - return worker.getAndPerformWork(); - } - }); + /** + * A thread which repeatedly fetches and processes work units from the Dataflow service. + */ + private static class WorkerThread implements Callable { + // sleeper is used to sleep the appropriate amount of time + WorkerThread(final DataflowWorker worker, final Sleeper sleeper) { + this.worker = worker; + this.sleeper = sleeper; + this.backOff = createBackOff(); } - List completionTimes = Lists.newArrayList(); - for (int i = 0; i < numThreads; ++i) { + @Override + public Boolean call() { + boolean success = true; try { - // CompletionService returns the tasks in the order in which the completed at. - completionService.take().get(); - } catch (Exception e) { - LOG.error("Failed waiting on thread to process work.", e); + do { // We loop getting and processing work. + try { + LOG.debug("Thread starting getAndPerformWork."); + success = worker.getAndPerformWork(); + LOG.debug("{} processing one WorkItem.", success ? "Finished" : "Failed"); + } catch (IOException e) { // If there is a problem getting work. + success = false; + } + 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) { + LOG.error("Interrupted during thread execution or sleep.", e); } - completionTimes.add(DateTimeUtils.currentTimeMillis()); + return false; } - long endTime = DateTimeUtils.currentTimeMillis(); - LOG.debug("Parallel worker thread processing start time: {}, end time: {}", - ISODateTimeFormat.dateTime().print(startTime), - ISODateTimeFormat.dateTime().print(endTime)); - for (long completionTime : completionTimes) { - LOG.debug("Worker thread execution time {}ms, idle time waiting for other work threads: {}ms", - completionTime - startTime, - endTime - completionTime); + private final DataflowWorker worker; + private final Sleeper sleeper; + private final BackOff backOff; + } + + // Visible for testing. + static void processWork(DataflowWorkerHarnessOptions pipelineOptions, + final DataflowWorker worker, Sleeper sleeper) throws InterruptedException { + int numThreads = Math.max(Runtime.getRuntime().availableProcessors(), 1); + ExecutorService executor = pipelineOptions.getExecutorService(); + final List> tasks = new LinkedList<>(); + + 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); } static DataflowWorker create(DataflowWorkerHarnessOptions options) { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java index 50eb5a72f8a5f..70895f0f8cc6e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerHarnessTest.java @@ -36,6 +36,7 @@ import com.google.cloud.dataflow.sdk.options.DataflowWorkerHarnessOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.runners.worker.logging.DataflowWorkerLoggingFormatter; +import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper; import com.google.cloud.dataflow.sdk.testing.RestoreDataflowLoggingFormatter; import com.google.cloud.dataflow.sdk.testing.RestoreSystemProperties; import com.google.cloud.dataflow.sdk.util.TestCredential; @@ -62,6 +63,7 @@ public class DataflowWorkerHarnessTest { @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); @Rule public TestRule restoreLogging = new RestoreDataflowLoggingFormatter(); @Rule public ExpectedException expectedException = ExpectedException.none(); + @Rule public FastNanoClockAndSleeper fastNanoClockAndSleeper = new FastNanoClockAndSleeper(); @Mock private MockHttpTransport transport; @Mock private MockLowLevelHttpRequest request; @Mock private DataflowWorker mockDataflowWorker; @@ -80,20 +82,14 @@ public void setUp() throws Exception { } @Test - public void testThatWeOnlyProcessWorkOncePerAvailableProcessor() throws Exception { - int numWorkers = Math.max(Runtime.getRuntime().availableProcessors() - 1, 1); - when(mockDataflowWorker.getAndPerformWork()).thenReturn(true); - DataflowWorkerHarness.processWork(pipelineOptions, mockDataflowWorker); - verify(mockDataflowWorker, times(numWorkers)).getAndPerformWork(); - verifyNoMoreInteractions(mockDataflowWorker); - } - - @Test - public void testThatWeOnlyProcessWorkOncePerAvailableProcessorEvenWhenFailing() throws Exception { - int numWorkers = Math.max(Runtime.getRuntime().availableProcessors() - 1, 1); + public void testThatWeRetryIfTaskExecutionFailAgainAndAgain() throws Exception { + int numWorkers = Math.max(Runtime.getRuntime().availableProcessors(), 1); when(mockDataflowWorker.getAndPerformWork()).thenReturn(false); - DataflowWorkerHarness.processWork(pipelineOptions, mockDataflowWorker); - verify(mockDataflowWorker, times(numWorkers)).getAndPerformWork(); + DataflowWorkerHarness.processWork( + pipelineOptions, mockDataflowWorker, fastNanoClockAndSleeper); + // Test that the backoff mechanism will retry the BACKOFF_MAX_ATTEMPTS number of times. + verify(mockDataflowWorker, times(numWorkers * DataflowWorkerHarness.BACKOFF_MAX_ATTEMPTS)) + .getAndPerformWork(); verifyNoMoreInteractions(mockDataflowWorker); } From 8ca93fb64589c46bf594567e2193bfdb93616401 Mon Sep 17 00:00:00 2001 From: ckuhn Date: Wed, 11 Mar 2015 09:39:04 -0700 Subject: [PATCH 0258/1541] To create unique job names for tests, we need to use method name rather than class name, and prioritize the unique part of the name. We no longer need to limit these names to 40 characters. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88342400 --- .../sdk/options/DataflowPipelineOptions.java | 10 --------- .../dataflow/sdk/testing/TestPipeline.java | 21 ++++++------------- .../options/DataflowPipelineOptionsTest.java | 20 ++++++++---------- .../sdk/testing/TestPipelineTest.java | 11 ++++++---- 4 files changed, 22 insertions(+), 40 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java index b4b1bccfb3973..5dec12031b2f5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java @@ -86,8 +86,6 @@ public interface DataflowPipelineOptions extends public static class JobNameFactory implements DefaultValueFactory { private static final DateTimeFormatter FORMATTER = DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC); - private static final int MAX_APP_NAME = 19; - private static final int MAX_USER_NAME = 9; @Override public String create(PipelineOptions options) { @@ -100,14 +98,6 @@ public String create(PipelineOptions options) { String normalizedUserName = userName.toLowerCase() .replaceAll("[^a-z0-9]", "0"); String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis()); - - // Maximize the amount of the app name and user name we can use. - normalizedAppName = normalizedAppName.substring(0, - Math.min(normalizedAppName.length(), - MAX_APP_NAME + Math.max(0, MAX_USER_NAME - normalizedUserName.length()))); - normalizedUserName = normalizedUserName.substring(0, - Math.min(userName.length(), - MAX_USER_NAME + Math.max(0, MAX_APP_NAME - normalizedAppName.length()))); return normalizedAppName + "-" + normalizedUserName + "-" + datePart; } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java index 7545e7ee10521..45645888e71a0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java @@ -112,7 +112,6 @@ static TestDataflowPipelineOptions getPipelineOptions() { System.getProperty(PROPERTY_DATAFLOW_OPTIONS), PipelineOptions.class) .as(TestDataflowPipelineOptions.class); options.setAppName(getAppName()); - options.setJobName(getJobName()); return options; } catch (IOException e) { throw new RuntimeException("Unable to instantiate test options from system property " @@ -120,28 +119,20 @@ static TestDataflowPipelineOptions getPipelineOptions() { } } - /** Returns the class name of the test, or a default name. */ + /** Returns the class + method name of the test, or a default name. */ private static String getAppName() { Optional stackTraceElement = findCallersStackTrace(); if (stackTraceElement.isPresent()) { + String methodName = stackTraceElement.get().getMethodName(); String className = stackTraceElement.get().getClassName(); - return className.contains(".") - ? className.substring(className.lastIndexOf(".") + 1) - : className; + if (className.contains(".")) { + className = className.substring(className.lastIndexOf(".") + 1); + } + return className + "-" + methodName; } return "UnitTest"; } - /** Returns the method name of the test, or a default name. */ - private static String getJobName() { - Optional stackTraceElement = findCallersStackTrace(); - if (stackTraceElement.isPresent()) { - String name = stackTraceElement.get().getMethodName(); - return name.substring(0, Math.min(40, name.length())); - } - return "unittestjob"; - } - /** Returns the {@link StackTraceElement} of the calling class. */ private static Optional findCallersStackTrace() { Iterator elements = diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java index 725288a1a779d..03edfacd0ab89 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptionsTest.java @@ -52,33 +52,32 @@ public void testUserNameIsNotSet() { } @Test - public void testAppNameAndUserNameIsTooLong() { + public void testAppNameAndUserNameAreLong() { resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z"); System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde"); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("1234567890123456789012345678901234567890"); - assertEquals("a234567890123456789-abcdeabcd-1208190706", options.getJobName()); - assertTrue(options.getJobName().length() <= 40); + assertEquals( + "a234567890123456789012345678901234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706", + options.getJobName()); } @Test - public void testAppNameIsTooLong() { + public void testAppNameIsLong() { resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z"); System.getProperties().put("user.name", "abcde"); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("1234567890123456789012345678901234567890"); - assertEquals("a2345678901234567890123-abcde-1208190706", options.getJobName()); - assertTrue(options.getJobName().length() <= 40); + assertEquals("a234567890123456789012345678901234567890-abcde-1208190706", options.getJobName()); } @Test - public void testUserNameIsTooLong() { + public void testUserNameIsLong() { resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z"); System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde"); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("1234567890"); - assertEquals("a234567890-abcdeabcdeabcdeabc-1208190706", options.getJobName()); - assertTrue(options.getJobName().length() <= 40); + assertEquals("a234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706", options.getJobName()); } @@ -88,7 +87,6 @@ public void testUtf8UserNameAndApplicationNameIsNormalized() { System.getProperties().put("user.name", "ði ıntəˈnæʃənəl "); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("fəˈnɛtık əsoʊsiˈeıʃn"); - assertEquals("f00n0t0k00so0si0e00-0i00nt00n-1208190706", options.getJobName()); - assertTrue(options.getJobName().length() <= 40); + assertEquals("f00n0t0k00so0si0e00n-0i00nt00n000n0l0-1208190706", options.getJobName()); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java index e01b8b5c2b54d..5e476f2450f12 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java @@ -16,8 +16,10 @@ package com.google.cloud.dataflow.sdk.testing; +import static org.hamcrest.CoreMatchers.startsWith; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner; import com.google.common.collect.ImmutableMap; @@ -60,8 +62,7 @@ public void testCreationOfPipelineOptions() throws Exception { System.getProperties().put("dataflowOptions", stringOptions); TestDataflowPipelineOptions options = TestPipeline.getPipelineOptions(); assertEquals(DataflowPipelineRunner.class, options.getRunner()); - assertEquals("TestPipelineTest", options.getAppName()); - assertEquals("testCreationOfPipelineOptions", options.getJobName()); + assertThat(options.getJobName(), startsWith("testpipelinetest0testcreationofpipelineoptions-")); assertEquals("testProject", options.getProject()); assertEquals("testApiRootUrl", options.getApiRootUrl()); assertEquals("testDataflowEndpoint", options.getDataflowEndpoint()); @@ -81,7 +82,9 @@ public void testCreationOfPipelineOptionsFromReallyVerboselyNamedTestCase() thro .build())); System.getProperties().put("dataflowOptions", stringOptions); TestDataflowPipelineOptions options = TestPipeline.getPipelineOptions(); - assertEquals("TestPipelineTest", options.getAppName()); - assertEquals("testCreationOfPipelineOptionsFromReallyV", options.getJobName()); + assertThat(options.getAppName(), startsWith( + "TestPipelineTest-testCreationOfPipelineOptionsFromReallyVerboselyNamedTestCase")); + assertThat(options.getJobName(), startsWith( + "testpipelinetest0testcreationofpipelineoptionsfrom")); } } From 84742de2b34777fc143e65da3c935c36614871e5 Mon Sep 17 00:00:00 2001 From: robertwb Date: Sat, 7 Feb 2015 10:26:41 -0800 Subject: [PATCH 0259/1541] Some fixes for combining in streaming mode. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88347787 --- .../worker/GroupAlsoByWindowsParDoFn.java | 32 ++++--- .../dataflow/sdk/util/CombiningWindowSet.java | 84 +++++++++++------ .../sdk/util/GroupAlsoByWindowsDoFn.java | 34 +++---- .../util/StreamingGroupAlsoByWindowsDoFn.java | 65 ++++++++----- .../StreamingGroupAlsoByWindowsDoFnTest.java | 92 +++++++++++-------- 5 files changed, 189 insertions(+), 118 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java index 62c185a06f286..1c573701d9c77 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java @@ -64,23 +64,26 @@ public static GroupAlsoByWindowsParDoFn create( CounterSet.AddCounterMutator addCounterMutator, StateSampler sampler /* unused */) throws Exception { - final Object windowFn = + final Object windowFnObj = SerializableUtils.deserializeFromByteArray( getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), "serialized window fn"); - if (!(windowFn instanceof WindowFn)) { + if (!(windowFnObj instanceof WindowFn)) { throw new Exception( - "unexpected kind of WindowFn: " + windowFn.getClass().getName()); + "unexpected kind of WindowFn: " + windowFnObj.getClass().getName()); } + final WindowFn windowFn = (WindowFn) windowFnObj; byte[] serializedCombineFn = getBytes(cloudUserFn, PropertyNames.COMBINE_FN, null); - final Object combineFn; + final KeyedCombineFn combineFn; if (serializedCombineFn != null) { - combineFn = + Object combineFnObj = SerializableUtils.deserializeFromByteArray(serializedCombineFn, "serialized combine fn"); - if (!(combineFn instanceof KeyedCombineFn)) { - throw new Exception("unexpected kind of KeyedCombineFn: " + combineFn.getClass().getName()); + if (!(combineFnObj instanceof KeyedCombineFn)) { + throw new Exception( + "unexpected kind of KeyedCombineFn: " + combineFnObj.getClass().getName()); } + combineFn = (KeyedCombineFn) combineFnObj; } else { combineFn = null; } @@ -98,6 +101,7 @@ public static GroupAlsoByWindowsParDoFn create( throw new Exception( "Expected KvCoder for inputCoder, got: " + elemCoder.getClass().getName()); } + final KvCoder kvCoder = (KvCoder) elemCoder; boolean isStreamingPipeline = false; if (options instanceof StreamingOptions) { @@ -111,9 +115,10 @@ public static GroupAlsoByWindowsParDoFn create( public DoFnInfo createDoFnInfo() { return new DoFnInfo( StreamingGroupAlsoByWindowsDoFn.create( - (WindowFn) windowFn, - (KeyedCombineFn) combineFn, - ((KvCoder) elemCoder).getValueCoder()), + windowFn, + combineFn, + kvCoder.getKeyCoder(), + kvCoder.getValueCoder()), null); } }; @@ -127,9 +132,10 @@ public DoFnInfo createDoFnInfo() { public DoFnInfo createDoFnInfo() { return new DoFnInfo( GroupAlsoByWindowsDoFn.create( - (WindowFn) windowFn, - (KeyedCombineFn) combineFn, - elemCoder), + windowFn, + combineFn, + kvCoder.getKeyCoder(), + kvCoder.getValueCoder()), null); } }; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombiningWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombiningWindowSet.java index fe754bcb7cd96..d726551f80f16 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombiningWindowSet.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombiningWindowSet.java @@ -20,6 +20,7 @@ import com.google.api.client.util.Lists; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; @@ -28,7 +29,6 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.common.collect.Iterators; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashSet; @@ -39,33 +39,54 @@ * A WindowSet for combine accumulators. * It merges accumulators when windows are added or merged. * - * @param key tyep + * @param key type + * @param value input type * @param accumulator type + * @param value output type * @param window type */ -public class CombiningWindowSet - extends AbstractWindowSet { +public class CombiningWindowSet + extends AbstractWindowSet { private final CodedTupleTag> windowListTag = CodedTupleTag.of("liveWindowsList", IterableCoder.of(windowFn.windowCoder())); - private final KeyedCombineFn combineFn; + private final KeyedCombineFn combineFn; private final Set liveWindows; + private final Coder accumulatorCoder; private boolean liveWindowsModified; protected CombiningWindowSet( K key, WindowFn windowFn, - KeyedCombineFn combineFn, - Coder inputCoder, - DoFnProcessContext> context, + KeyedCombineFn combineFn, + Coder keyCoder, + Coder inputValueCoder, + DoFnProcessContext> context, ActiveWindowManager activeWindowManager) throws Exception { - super(key, windowFn, inputCoder, context, activeWindowManager); + super(key, windowFn, inputValueCoder, context, activeWindowManager); this.combineFn = combineFn; liveWindows = new HashSet(); Iterators.addAll(liveWindows, emptyIfNull(context.keyedState().lookup(windowListTag)).iterator()); liveWindowsModified = false; + // TODO: Use the pipeline's registry once the TODO in GroupByKey is resolved. + CoderRegistry coderRegistry = new CoderRegistry(); + coderRegistry.registerStandardCoders(); + accumulatorCoder = combineFn.getAccumulatorCoder(coderRegistry, keyCoder, inputValueCoder); + } + + protected static CombiningWindowSet + create( + K key, + WindowFn windowFn, + KeyedCombineFn combineFn, + Coder keyCoder, + Coder inputValueCoder, + DoFnProcessContext> context, + ActiveWindowManager activeWindowManager) throws Exception { + return new CombiningWindowSet( + key, windowFn, combineFn, keyCoder, inputValueCoder, context, activeWindowManager); } @Override @@ -74,29 +95,25 @@ protected Collection windows() { } @Override - protected VA finalValue(W window) throws Exception { - return context.keyedState().lookup( - bufferTag(window, windowFn.windowCoder(), inputCoder)); + protected VO finalValue(W window) throws Exception { + return combineFn.extractOutput( + key, + context.keyedState().lookup(bufferTag(window, windowFn.windowCoder(), accumulatorCoder))); } @Override - protected void put(W window, VA value) throws Exception { - CodedTupleTag tag = bufferTag(window, windowFn.windowCoder(), inputCoder); - VA va = context.keyedState().lookup(tag); - VA newValue; + protected void put(W window, VI value) throws Exception { + VA va = context.keyedState().lookup(accumulatorTag(window)); if (va == null) { - newValue = value; - } else { - newValue = combineFn.mergeAccumulators(key, Arrays.asList(value, va)); + va = combineFn.createAccumulator(key); } - context.keyedState().store(tag, newValue); - activeWindowManager.addWindow(window); - liveWindowsModified = liveWindows.add(window); + combineFn.addInput(key, va, value); + store(window, va); } @Override protected void remove(W window) throws Exception { - context.keyedState().remove(bufferTag(window, windowFn.windowCoder(), inputCoder)); + context.keyedState().remove(accumulatorTag(window)); activeWindowManager.addWindow(window); liveWindowsModified = liveWindows.remove(window); } @@ -104,17 +121,28 @@ protected void remove(W window) throws Exception { @Override protected void merge(Collection toBeMerged, W mergeResult) throws Exception { List accumulators = Lists.newArrayList(); - for (W w : toBeMerged) { - VA va = context.keyedState().lookup( - bufferTag(w, windowFn.windowCoder(), inputCoder)); + for (W window : toBeMerged) { + VA va = context.keyedState().lookup(accumulatorTag(window)); // TODO: determine whether null means no value associated with the tag, b/19201776. if (va != null) { accumulators.add(va); } - remove(w); + remove(window); } VA mergedVa = combineFn.mergeAccumulators(key, accumulators); - put(mergeResult, mergedVa); + store(mergeResult, mergedVa); + } + + private CodedTupleTag accumulatorTag(W window) throws Exception { + // TODO: Cache this. + return bufferTag(window, windowFn.windowCoder(), accumulatorCoder); + } + + private void store(W window, VA va) throws Exception { + CodedTupleTag tag = accumulatorTag(window); + context.keyedState().store(tag, va); + activeWindowManager.addWindow(window); + liveWindowsModified = liveWindows.add(window); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java index 896dc98b5d033..f109e619fe4e9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java @@ -80,17 +80,21 @@ AbstractWindowSet, W> createWindowSet(K key, /** * Create a {@link GroupAlsoByWindowsDoFn} using the specified combineFn. */ - private static GroupAlsoByWindowsDoFn - createCombine(final WindowFn windowFn, - final KeyedCombineFn combineFn, - final Coder inputCoder) { - return new GABWViaWindowSetDoFn(windowFn) { - @Override - AbstractWindowSet createWindowSet(K key, - DoFnProcessContext>>, KV> context, - BatchActiveWindowManager activeWindowManager) throws Exception { - return new CombiningWindowSet( - key, windowFn, combineFn, inputCoder, context, activeWindowManager); + private static GroupAlsoByWindowsDoFn + createCombine( + final WindowFn windowFn, + final KeyedCombineFn combineFn, + final Coder keyCoder, + final Coder inputCoder) { + return new GABWViaWindowSetDoFn(windowFn) { + @Override + AbstractWindowSet createWindowSet( + K key, + DoFnProcessContext>>, KV> context, + BatchActiveWindowManager activeWindowManager) throws Exception { + return CombiningWindowSet.create( + key, windowFn, combineFn, keyCoder, inputCoder, + (DoFnProcessContext>) (DoFnProcessContext) context, activeWindowManager); } }; } @@ -99,8 +103,8 @@ AbstractWindowSet createWindowSet(K key, * Construct a {@link GroupAlsoByWindowsDoFn} using the {@code combineFn} if available. */ public static GroupAlsoByWindowsDoFn - create(WindowFn windowFn, KeyedCombineFn combineFn, - Coder inputCoder) { + create(WindowFn windowFn, KeyedCombineFn combineFn, + Coder keyCoder, Coder inputCoder) { if (combineFn == null) { // Without combineFn, it should be the case that VO = Iterable, so this is safe @SuppressWarnings("unchecked") @@ -108,10 +112,8 @@ AbstractWindowSet createWindowSet(K key, (GroupAlsoByWindowsDoFn) create(windowFn, inputCoder); return fn; } else { - // With a combineFn, then VI = VO, and we'll use those as the type of the accumulator - @SuppressWarnings("unchecked") GroupAlsoByWindowsDoFn fn = - (GroupAlsoByWindowsDoFn) createCombine(windowFn, combineFn, inputCoder); + createCombine(windowFn, combineFn, keyCoder, inputCoder); return fn; } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java index 741cd7130bfaf..fa7a62f102862 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFn.java @@ -39,25 +39,29 @@ public class StreamingGroupAlsoByWindowsDoFn extends DoFn>, KV> implements DoFn.RequiresKeyedState { - protected WindowFn windowFn; - protected KeyedCombineFn combineFn; - protected Coder inputCoder; + protected final WindowFn windowFn; + protected final KeyedCombineFn combineFn; + protected final Coder keyCoder; + protected final Coder inputValueCoder; protected StreamingGroupAlsoByWindowsDoFn( WindowFn windowFn, - KeyedCombineFn combineFn, - Coder inputCoder) { + KeyedCombineFn combineFn, + Coder keyCoder, + Coder inputValueCoder) { this.windowFn = windowFn; this.combineFn = combineFn; - this.inputCoder = inputCoder; + this.keyCoder = keyCoder; + this.inputValueCoder = inputValueCoder; } public static StreamingGroupAlsoByWindowsDoFn create( WindowFn windowFn, - KeyedCombineFn combineFn, - Coder inputCoder) { - return new StreamingGroupAlsoByWindowsDoFn<>(windowFn, combineFn, inputCoder); + KeyedCombineFn combineFn, + Coder keyCoder, + Coder inputValueCoder) { + return new StreamingGroupAlsoByWindowsDoFn<>(windowFn, combineFn, keyCoder, inputValueCoder); } private AbstractWindowSet createWindowSet( @@ -65,27 +69,44 @@ private AbstractWindowSet createWindowSet( DoFnProcessContext> context, AbstractWindowSet.ActiveWindowManager activeWindowManager) throws Exception { if (combineFn != null) { - return new CombiningWindowSet( - key, windowFn, combineFn, inputCoder, context, activeWindowManager); - } else if (windowFn instanceof PartitioningWindowFn) { - return new PartitionBufferingWindowSet( - key, windowFn, inputCoder, context, activeWindowManager); + return CombiningWindowSet.create( + key, windowFn, combineFn, keyCoder, inputValueCoder, context, activeWindowManager); } else { - return new BufferingWindowSet( - key, windowFn, inputCoder, context, activeWindowManager); + // VO == Iterable + @SuppressWarnings({"unchecked", "rawtypes"}) + DoFnProcessContext>> iterableContext = (DoFnProcessContext) context; + AbstractWindowSet, W> iterableWindowSet = + createNonCombiningWindowSet(key, iterableContext, activeWindowManager); + @SuppressWarnings({"unchecked", "rawtypes"}) + AbstractWindowSet windowSet = (AbstractWindowSet) iterableWindowSet; + return windowSet; + } + } + + private AbstractWindowSet, W> createNonCombiningWindowSet( + K key, + DoFnProcessContext>> context, + AbstractWindowSet.ActiveWindowManager activeWindowManager) throws Exception { + if (windowFn instanceof PartitioningWindowFn) { + return new PartitionBufferingWindowSet( + key, windowFn, inputValueCoder, context, activeWindowManager); + } else { + return new BufferingWindowSet( + key, windowFn, inputValueCoder, context, activeWindowManager); } } @Override - public void processElement(ProcessContext processContext) throws Exception { - DoFnProcessContext>, KV> context = - (DoFnProcessContext>, KV>) processContext; + public void processElement(ProcessContext context) throws Exception { + @SuppressWarnings("unchecked") + DoFnProcessContext>, KV> doFnContext = + (DoFnProcessContext>, KV>) context; if (!context.element().isTimer()) { KV element = context.element().element(); K key = element.getKey(); VI value = element.getValue(); AbstractWindowSet windowSet = createWindowSet( - key, context, new StreamingActiveWindowManager<>(windowFn, context)); + key, doFnContext, new StreamingActiveWindowManager<>(windowFn, doFnContext)); for (BoundedWindow window : context.windows()) { windowSet.put((W) window, value); @@ -93,9 +114,9 @@ public void processElement(ProcessContext processContext) throws Exception { windowSet.flush(); } else { - TimerOrElement timer = context.element(); + TimerOrElement> timer = context.element(); AbstractWindowSet windowSet = createWindowSet( - (K) timer.key(), context, new StreamingActiveWindowManager<>(windowFn, context)); + (K) timer.key(), doFnContext, new StreamingActiveWindowManager<>(windowFn, doFnContext)); // Attempt to merge windows before emitting; that may remove the current window under // consideration. diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java index 5d5c03406f29c..94cf19599ab2a 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; @@ -29,7 +30,6 @@ import com.google.cloud.dataflow.sdk.transforms.Sum; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; -import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; @@ -52,7 +52,7 @@ /** Unit tests for {@link StreamingGroupAlsoByWindowsDoFn}. */ @RunWith(JUnit4.class) -@SuppressWarnings({"rawtypes", "unchecked"}) +@SuppressWarnings("rawtypes") public class StreamingGroupAlsoByWindowsDoFnTest { ExecutionContext execContext; CounterSet counters; @@ -65,15 +65,14 @@ public class StreamingGroupAlsoByWindowsDoFnTest { } @Test public void testEmpty() throws Exception { - DoFnRunner>, - KV>, List> runner = - makeRunner(FixedWindows.of(Duration.millis(10)), null); + DoFnRunner>, KV>, List> runner = + makeRunner(FixedWindows.of(Duration.millis(10))); runner.startBundle(); runner.finishBundle(); - List>> result = runner.getReceiver(outputTag); + List result = runner.getReceiver(outputTag); assertEquals(0, result.size()); } @@ -81,9 +80,9 @@ public class StreamingGroupAlsoByWindowsDoFnTest { @Test public void testFixedWindows() throws Exception { DoFnRunner>, KV>, List> runner = - makeRunner(FixedWindows.of(Duration.millis(10)), null); + makeRunner(FixedWindows.of(Duration.millis(10))); - Coder windowCoder = FixedWindows.of(Duration.millis(10)).windowCoder(); + Coder windowCoder = FixedWindows.of(Duration.millis(10)).windowCoder(); runner.startBundle(); @@ -119,6 +118,7 @@ public class StreamingGroupAlsoByWindowsDoFnTest { runner.finishBundle(); + @SuppressWarnings("unchecked") List>>> result = runner.getReceiver(outputTag); assertEquals(2, result.size()); @@ -139,10 +139,10 @@ public class StreamingGroupAlsoByWindowsDoFnTest { @Test public void testSlidingWindows() throws Exception { DoFnRunner>, KV>, List> runner = - makeRunner(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10)), null); + makeRunner(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10))); Coder windowCoder = - SlidingWindows.of(Duration.millis(10)).every(Duration.millis(10)).windowCoder(); + SlidingWindows.of(Duration.millis(10)).every(Duration.millis(10)).windowCoder(); runner.startBundle(); @@ -178,6 +178,7 @@ public class StreamingGroupAlsoByWindowsDoFnTest { runner.finishBundle(); + @SuppressWarnings("unchecked") List>>> result = runner.getReceiver(outputTag); assertEquals(3, result.size()); @@ -204,10 +205,10 @@ public class StreamingGroupAlsoByWindowsDoFnTest { @Test public void testSessions() throws Exception { DoFnRunner>, KV>, List> runner = - makeRunner(Sessions.withGapDuration(Duration.millis(10)), null); + makeRunner(Sessions.withGapDuration(Duration.millis(10))); Coder windowCoder = - Sessions.withGapDuration(Duration.millis(10)).windowCoder(); + Sessions.withGapDuration(Duration.millis(10)).windowCoder(); runner.startBundle(); @@ -243,6 +244,7 @@ public class StreamingGroupAlsoByWindowsDoFnTest { runner.finishBundle(); + @SuppressWarnings("unchecked") List>>> result = runner.getReceiver(outputTag); assertEquals(2, result.size()); @@ -261,84 +263,96 @@ public class StreamingGroupAlsoByWindowsDoFnTest { } @Test public void testSessionsCombine() throws Exception { - CombineFn combineFn = Combine.SimpleCombineFn.of(new Sum.SumLongFn()); - DoFnRunner>>, - KV>, List> runner = - makeRunner(Sessions.withGapDuration(Duration.millis(10)), - combineFn.asKeyedFn()); + CombineFn combineFn = Combine.SimpleCombineFn.of(new Sum.SumLongFn()); + DoFnRunner>, + KV, List> runner = + makeRunner(Sessions.withGapDuration(Duration.millis(10)), + combineFn.asKeyedFn()); Coder windowCoder = - Sessions.withGapDuration(Duration.millis(10)).windowCoder(); + Sessions.withGapDuration(Duration.millis(10)).windowCoder(); runner.startBundle(); runner.processElement(WindowedValue.of( - TimerOrElement.element(KV.of("k", (Iterable) Arrays.asList(1L))), + TimerOrElement.element(KV.of("k", 1L)), new Instant(0), Arrays.asList(window(0, 10)))); runner.processElement(WindowedValue.of( - TimerOrElement.element(KV.of("k", (Iterable) Arrays.asList(2L))), + TimerOrElement.element(KV.of("k", 2L)), new Instant(5), Arrays.asList(window(5, 15)))); runner.processElement(WindowedValue.of( - TimerOrElement.element(KV.of("k", (Iterable) Arrays.asList(3L))), + TimerOrElement.element(KV.of("k", 3L)), new Instant(15), Arrays.asList(window(15, 25)))); runner.processElement(WindowedValue.of( - TimerOrElement.element(KV.of("k", (Iterable) Arrays.asList(4L))), + TimerOrElement.element(KV.of("k", 4L)), new Instant(3), Arrays.asList(window(3, 13)))); runner.processElement(WindowedValue.valueInEmptyWindows( - TimerOrElement.>>timer( + TimerOrElement.>timer( windowToString((IntervalWindow) window(0, 15), windowCoder), new Instant(14), "k"))); runner.processElement(WindowedValue.valueInEmptyWindows( - TimerOrElement.>>timer( + TimerOrElement.>timer( windowToString((IntervalWindow) window(15, 25), windowCoder), new Instant(24), "k"))); runner.finishBundle(); - List>>> result = runner.getReceiver(outputTag); + @SuppressWarnings("unchecked") + List>> result = runner.getReceiver(outputTag); assertEquals(2, result.size()); - WindowedValue>> item0 = result.get(0); + WindowedValue> item0 = result.get(0); assertEquals("k", item0.getValue().getKey()); - assertThat(item0.getValue().getValue(), Matchers.containsInAnyOrder(7L)); + assertEquals((Long) 7L, item0.getValue().getValue()); assertEquals(new Instant(14), item0.getTimestamp()); assertThat(item0.getWindows(), Matchers.contains(window(0, 15))); - WindowedValue>> item1 = result.get(1); + WindowedValue> item1 = result.get(1); assertEquals("k", item1.getValue().getKey()); - assertThat(item1.getValue().getValue(), Matchers.containsInAnyOrder(3L)); + assertEquals((Long) 3L, item1.getValue().getValue()); assertEquals(new Instant(24), item1.getTimestamp()); assertThat(item1.getWindows(), Matchers.contains(window(15, 25))); } - private DoFnRunner makeRunner( - WindowFn windowingStrategy, - KeyedCombineFn combineFn) { - StreamingGroupAlsoByWindowsDoFn fn = - StreamingGroupAlsoByWindowsDoFn.create(windowingStrategy, combineFn, StringUtf8Coder.of()); + private DoFnRunner>, KV>, List> + makeRunner(WindowFn windowFn) { + return makeRunner(windowFn, null, StringUtf8Coder.of()); + } - DoFnRunner runner = + private DoFnRunner>, KV, List> makeRunner( + WindowFn windowFn, + KeyedCombineFn combineFn) { + return makeRunner(windowFn, combineFn, BigEndianLongCoder.of()); + } + + private DoFnRunner>, KV, List> makeRunner( + WindowFn windowFn, + KeyedCombineFn combineFn, + Coder inputValueCoder) { + StreamingGroupAlsoByWindowsDoFn fn = + StreamingGroupAlsoByWindowsDoFn.create( + windowFn, combineFn, StringUtf8Coder.of(), inputValueCoder); + + return DoFnRunner.createWithListOutputs( PipelineOptionsFactory.create(), fn, PTuple.empty(), - outputTag, + (TupleTag>) (TupleTag) outputTag, new ArrayList>(), execContext.createStepContext("merge"), counters.getAddCounterMutator(), - new GlobalWindows()); - - return runner; + windowFn); } private BoundedWindow window(long start, long end) { From dc3057b550a326a57a9c78155bd1c992823c2585 Mon Sep 17 00:00:00 2001 From: robertwb Date: Wed, 11 Mar 2015 17:44:55 -0700 Subject: [PATCH 0260/1541] Add Combine.BinaryCombineFn for efficiently implementing aggregations that most easily expressed as binary operations. Also added primitive int, long, and double versions of BinaryCombineFn. Used the aforementioned binary combine classes to improve the efficiency of sum, min, max. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88393409 --- .../dataflow/sdk/transforms/Combine.java | 326 ++++++++++++++++++ .../cloud/dataflow/sdk/transforms/Max.java | 37 +- .../cloud/dataflow/sdk/transforms/Min.java | 35 +- .../cloud/dataflow/sdk/transforms/Sum.java | 48 +-- .../dataflow/sdk/transforms/CombineTest.java | 44 +++ .../dataflow/sdk/transforms/ViewTest.java | 3 +- .../dataflow/sdk/util/AggregatorImplTest.java | 25 +- .../StreamingGroupAlsoByWindowsDoFnTest.java | 3 +- 8 files changed, 452 insertions(+), 69 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java index 5d51a3c17303d..729ad0ff33211 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -17,7 +17,12 @@ package com.google.cloud.dataflow.sdk.transforms; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.Coder.Context; +import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException; +import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.coders.CustomCoder; +import com.google.cloud.dataflow.sdk.coders.DelegateCoder; import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.VarIntCoder; @@ -35,6 +40,9 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; @@ -442,6 +450,324 @@ public Coder getDefaultOutputCoder( } + ///////////////////////////////////////////////////////////////////////////// + + /** + * An abstract subclass of CombineFn for implementing combiners that are more + * easily expressed as binary operations. + */ + public abstract static class BinaryCombineFn + extends CombineFn, V> { + + /** + * Applies the binary operation to the two operands, returning the result. + */ + public abstract V apply(V left, V right); + + /** + * Returns the value that should be used for the combine of the empty set. + */ + public V identity() { + return null; + } + + @Override + public Holder createAccumulator() { + return new Holder<>(); + } + + @Override + public void addInput(Holder accumulator, V input) { + if (accumulator.present) { + accumulator.set(apply(accumulator.value, input)); + } else { + accumulator.set(input); + } + } + + @Override + public Holder mergeAccumulators(Iterable> accumulators) { + Holder running = new Holder<>(); + for (Holder accumulator : accumulators) { + if (accumulator.present) { + if (running.present) { + running.set(apply(running.value, accumulator.value)); + } else { + running.set(accumulator.value); + } + } + } + return running; + } + + @Override + public V extractOutput(Holder accumulator) { + if (accumulator.present) { + return accumulator.value; + } else { + return identity(); + } + } + + @Override + public Coder> getAccumulatorCoder(CoderRegistry registry, final Coder inputCoder) { + return new CustomCoder>() { + @Override + public void encode(Holder accumulator, OutputStream outStream, Context context) + throws CoderException, IOException { + if (accumulator.present) { + outStream.write(1); + inputCoder.encode(accumulator.value, outStream, context); + } else { + outStream.write(0); + } + } + + @Override + public Holder decode(InputStream inStream, Context context) + throws CoderException, IOException { + if (inStream.read() == 1) { + return new Holder(inputCoder.decode(inStream, context)); + } else { + return new Holder<>(); + } + } + + @Override + @Deprecated + public boolean isDeterministic() { + return inputCoder.isDeterministic(); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + inputCoder.verifyDeterministic(); + } + }; + } + + @Override + public Coder getDefaultOutputCoder(CoderRegistry registry, Coder inputCoder) { + return inputCoder; + } + + private static class Holder { + public V value; + public boolean present; + public Holder() { } + public Holder(V value) { set(value); } + public void set(V value) { + this.present = true; + this.value = value; + } + } + } + + /** + * An abstract subclass of CombineFn for implementing combiners that are more + * easily expressed as binary operations on ints. + */ + public abstract static class BinaryCombineIntegerFn extends CombineFn { + + /** + * Applies the binary operation to the two operands, returning the result. + */ + public abstract int apply(int left, int right); + + /** + * Returns the identity element of this operation, i.e. an element {@code e} + * such that {@code apply(e, x) == apply(x, e) == x} for all values of {@code x}. + */ + public abstract int identity(); + + @Override + public int[] createAccumulator() { + return wrap(identity()); + } + + @Override + public void addInput(int[] accumulator, Integer input) { + accumulator[0] = apply(accumulator[0], input); + } + + @Override + public int[] mergeAccumulators(Iterable accumulators) { + Iterator iter = accumulators.iterator(); + if (!iter.hasNext()) { + return createAccumulator(); + } else { + int running = iter.next()[0]; + while (iter.hasNext()) { + running = apply(running, iter.next()[0]); + } + return wrap(running); + } + } + + @Override + public Integer extractOutput(int[] accumulator) { + return accumulator[0]; + } + + @Override + public Coder getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) { + return DelegateCoder.of( + inputCoder, + new DelegateCoder.CodingFunction() { + @Override public Integer apply(int[] accumulator) { return accumulator[0]; } + }, + new DelegateCoder.CodingFunction() { + @Override public int[] apply(Integer value) { return wrap(value); } + }); + } + + @Override + public Coder getDefaultOutputCoder(CoderRegistry registry, + Coder inputCoder) { + return inputCoder; + } + + private int[] wrap(int value) { + return new int[] { value }; + } + } + + /** + * An abstract subclass of CombineFn for implementing combiners that are more + * easily expressed as binary operations on longs. + */ + public abstract static class BinaryCombineLongFn extends CombineFn { + + /** + * Applies the binary operation to the two operands, returning the result. + */ + public abstract long apply(long left, long right); + + /** + * Returns the identity element of this operation, i.e. an element {@code e} + * such that {@code apply(e, x) == apply(x, e) == x} for all values of {@code x}. + */ + public abstract long identity(); + + @Override + public long[] createAccumulator() { + return wrap(identity()); + } + + @Override + public void addInput(long[] accumulator, Long input) { + accumulator[0] = apply(accumulator[0], input); + } + + @Override + public long[] mergeAccumulators(Iterable accumulators) { + Iterator iter = accumulators.iterator(); + if (!iter.hasNext()) { + return createAccumulator(); + } else { + long running = iter.next()[0]; + while (iter.hasNext()) { + running = apply(running, iter.next()[0]); + } + return wrap(running); + } + } + + @Override + public Long extractOutput(long[] accumulator) { + return accumulator[0]; + } + + @Override + public Coder getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) { + return DelegateCoder.of( + inputCoder, + new DelegateCoder.CodingFunction() { + @Override public Long apply(long[] accumulator) { return accumulator[0]; } + }, + new DelegateCoder.CodingFunction() { + @Override public long[] apply(Long value) { return wrap(value); } + }); + } + + @Override + public Coder getDefaultOutputCoder(CoderRegistry registry, Coder inputCoder) { + return inputCoder; + } + + private long[] wrap(long value) { + return new long[] { value }; + } + } + + /** + * An abstract subclass of CombineFn for implementing combiners that are more + * easily expressed as binary operations on doubles. + */ + public abstract static class BinaryCombineDoubleFn extends CombineFn { + + /** + * Applies the binary operation to the two operands, returning the result. + */ + public abstract double apply(double left, double right); + + /** + * Returns the identity element of this operation, i.e. an element {@code e} + * such that {@code apply(e, x) == apply(x, e) == x} for all values of {@code x}. + */ + public abstract double identity(); + + @Override + public double[] createAccumulator() { + return wrap(identity()); + } + + @Override + public void addInput(double[] accumulator, Double input) { + accumulator[0] = apply(accumulator[0], input); + } + + @Override + public double[] mergeAccumulators(Iterable accumulators) { + Iterator iter = accumulators.iterator(); + if (!iter.hasNext()) { + return createAccumulator(); + } else { + double running = iter.next()[0]; + while (iter.hasNext()) { + running = apply(running, iter.next()[0]); + } + return wrap(running); + } + } + + @Override + public Double extractOutput(double[] accumulator) { + return accumulator[0]; + } + + @Override + public Coder getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) { + return DelegateCoder.of( + inputCoder, + new DelegateCoder.CodingFunction() { + @Override public Double apply(double[] accumulator) { return accumulator[0]; } + }, + new DelegateCoder.CodingFunction() { + @Override public double[] apply(Double value) { return wrap(value); } + }); + } + + @Override + public Coder getDefaultOutputCoder(CoderRegistry registry, Coder inputCoder) { + return inputCoder; + } + + private double[] wrap(double value) { + return new double[] { value }; + } + } + ///////////////////////////////////////////////////////////////////////////// /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java index 70d595da982eb..7c707140a7a7b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java @@ -133,15 +133,15 @@ public static Combine.PerKey doublesPerKey() { ///////////////////////////////////////////////////////////////////////////// /** - * A {@code SerializableFunction} that computes the maximum of an - * {@code Iterable} of numbers of type {@code N}, useful as an - * argument to {@link Combine#globally} or {@link Combine#perKey}. + * A {@code CombineFn} that computes the maximum of a set of elements + * of type {@code N}, useful as an argument to {@link Combine#globally} + * or {@link Combine#perKey}. * * @param the type of the {@code Number}s being compared */ @SuppressWarnings("serial") - public static class MaxFn> - implements SerializableFunction, N> { + public static class MaxFn> + extends Combine.BinaryCombineFn { /** The smallest value of type N. */ private final N initialValue; @@ -157,20 +157,19 @@ public MaxFn(N initialValue) { } @Override - public N apply(Iterable input) { - N max = initialValue; - for (N value : input) { - if (value.compareTo(max) > 0) { - max = value; - } - } - return max; + public N apply(N a, N b) { + return a.compareTo(b) >= 0 ? a : b; + } + + @Override + public N identity() { + return initialValue; } } /** - * A {@code SerializableFunction} that computes the maximum of an - * {@code Iterable} of {@code Integer}s, useful as an argument to + * A {@code CombineFn} that computes the maximum of a collection + * of {@code Integer}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ @SuppressWarnings("serial") @@ -179,8 +178,8 @@ public static class MaxIntegerFn extends MaxFn { } /** - * A {@code SerializableFunction} that computes the maximum of an - * {@code Iterable} of {@code Long}s, useful as an argument to + * A {@code CombineFn} that computes the maximum of a collection + * of {@code Long}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ @SuppressWarnings("serial") @@ -189,8 +188,8 @@ public static class MaxLongFn extends MaxFn { } /** - * A {@code SerializableFunction} that computes the maximum of an - * {@code Iterable} of {@code Double}s, useful as an argument to + * A {@code CombineFn} that computes the maximum of a collection + * of {@code Double}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ @SuppressWarnings("serial") diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java index 0eb58e41563cd..aaf7d3aaf61e4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java @@ -133,14 +133,14 @@ public static Combine.PerKey doublesPerKey() { ///////////////////////////////////////////////////////////////////////////// /** - * A {@code SerializableFunction} that computes the minimum of an - * {@code Iterable} of numbers of type {@code N}, useful as an + * A {@code CombineFn} that computes the minimum of a collection + * of elements of type {@code N}, useful as an * argument to {@link Combine#globally} or {@link Combine#perKey}. * * @param the type of the {@code Number}s being compared */ - public static class MinFn> - implements SerializableFunction, N> { + public static class MinFn> + extends Combine.BinaryCombineFn { private static final long serialVersionUID = 0; /** The largest value of type N. */ @@ -157,20 +157,19 @@ public MinFn(N initialValue) { } @Override - public N apply(Iterable input) { - N min = initialValue; - for (N value : input) { - if (value.compareTo(min) < 0) { - min = value; - } - } - return min; + public N apply(N a, N b) { + return a.compareTo(b) <= 0 ? a : b; + } + + @Override + public N identity() { + return initialValue; } } /** - * A {@code SerializableFunction} that computes the minimum of an - * {@code Iterable} of {@code Integer}s, useful as an argument to + * A {@code CombineFn} that computes the minimum of a collection + * of {@code Integer}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ public static class MinIntegerFn extends MinFn { @@ -180,8 +179,8 @@ public static class MinIntegerFn extends MinFn { } /** - * A {@code SerializableFunction} that computes the minimum of an - * {@code Iterable} of {@code Long}s, useful as an argument to + * A {@code CombineFn} that computes the minimum of a collection + * of {@code Long}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ public static class MinLongFn extends MinFn { @@ -191,8 +190,8 @@ public static class MinLongFn extends MinFn { } /** - * A {@code SerializableFunction} that computes the minimum of an - * {@code Iterable} of {@code Double}s, useful as an argument to + * A {@code CombineFn} that computes the minimum of a collection + * of {@code Double}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ public static class MinDoubleFn extends MinFn { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java index b61def31d3aad..98f2adb2b20a1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java @@ -132,15 +132,15 @@ public static Combine.PerKey doublesPerKey() { * {@code Iterable} of {@code Integer}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ - public static class SumIntegerFn - implements SerializableFunction, Integer> { + public static class SumIntegerFn extends Combine.BinaryCombineIntegerFn { @Override - public Integer apply(Iterable input) { - int sum = 0; - for (int value : input) { - sum += value; - } - return sum; + public int apply(int a, int b) { + return a + b; + } + + @Override + public int identity() { + return 0; } } @@ -149,15 +149,15 @@ public Integer apply(Iterable input) { * {@code Iterable} of {@code Long}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ - public static class SumLongFn - implements SerializableFunction, Long> { + public static class SumLongFn extends Combine.BinaryCombineLongFn { @Override - public Long apply(Iterable input) { - long sum = 0; - for (long value : input) { - sum += value; - } - return sum; + public long apply(long a, long b) { + return a + b; + } + + @Override + public long identity() { + return 0; } } @@ -166,15 +166,15 @@ public Long apply(Iterable input) { * {@code Iterable} of {@code Double}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ - public static class SumDoubleFn - implements SerializableFunction, Double> { + public static class SumDoubleFn extends Combine.BinaryCombineDoubleFn { + @Override + public double apply(double a, double b) { + return a + b; + } + @Override - public Double apply(Iterable input) { - double sum = 0; - for (double value : input) { - sum += value; - } - return sum; + public double identity() { + return 0; } } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java index 38497c3028fd4..0a3dcdce719e6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java @@ -16,6 +16,7 @@ package com.google.cloud.dataflow.sdk.transforms; +import static com.google.cloud.dataflow.sdk.TestUtils.checkCombineFn; import static org.junit.Assert.assertThat; import com.google.api.client.util.Preconditions; @@ -343,6 +344,49 @@ public void testHotKeyCombining() { p.run(); } + @Test + public void testBinaryCombineFn() { + Pipeline p = TestPipeline.create(); + PCollection> input = copy(createInput(p, TABLE), 2); + PCollection> intProduct = input + .apply(Combine.perKey(new TestProdInt())); + PCollection> objProduct = input + .apply(Combine.perKey(new TestProdObj())); + + List> expected = Arrays.asList(KV.of("a", 16), KV.of("b", 169)); + DataflowAssert.that(intProduct).containsInAnyOrder(expected); + DataflowAssert.that(objProduct).containsInAnyOrder(expected); + + p.run(); + } + + @Test + public void testBinaryCombineFnWithNulls() { + checkCombineFn(new NullCombiner(), Arrays.asList(3, 3, 5), 45); + checkCombineFn(new NullCombiner(), Arrays.asList(null, 3, 5), 30); + checkCombineFn(new NullCombiner(), Arrays.asList(3, 3, null), 18); + checkCombineFn(new NullCombiner(), Arrays.asList(null, 3, null), 12); + checkCombineFn(new NullCombiner(), Arrays.asList(null, null, null), 8); + } + + private static final class TestProdInt extends Combine.BinaryCombineIntegerFn { + public int apply(int left, int right) { return left * right; } + public int identity() { return 1; } + } + + private static final class TestProdObj extends Combine.BinaryCombineFn { + public Integer apply(Integer left, Integer right) { return left * right; } + } + + /** + * Computes the product, considering null values to be 2. + */ + private static final class NullCombiner extends Combine.BinaryCombineFn { + public Integer apply(Integer left, Integer right) { + return (left == null ? 2 : left) * (right == null ? 2 : right); + } + } + //////////////////////////////////////////////////////////////////////////// // Test classes, for different kinds of combining fns. diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ViewTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ViewTest.java index f4843c4154278..852c73cbe4948 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ViewTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ViewTest.java @@ -219,8 +219,7 @@ public void testCombinedMapSideInput() { final PCollectionView, ?> view = pipeline .apply(Create.of(KV.of("a", 1), KV.of("a", 20), KV.of("b", 3))) - .apply(View.asMap().withCombiner( - Combine.SimpleCombineFn.of(new Sum.SumIntegerFn()))); + .apply(View.asMap().withCombiner(new Sum.SumIntegerFn())); PCollection> output = pipeline .apply(Create.of("apple", "banana", "blackberry")) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java index 85ddbef41e334..dc9c8e5c7c858 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AggregatorImplTest.java @@ -70,6 +70,23 @@ private void testAggregator(List items, CounterTestUtils.extractCounterUpdate(expectedCounter, false)); } + @SuppressWarnings("rawtypes") + private void testAggregator(List items, + Combine.CombineFn combiner, + Counter expectedCounter) { + CounterSet counters = new CounterSet(); + Aggregator aggregator = new AggregatorImpl( + AGGREGATOR_NAME, combiner, counters.getAddCounterMutator()); + for (V item : items) { + aggregator.addValue(item); + } + + List cloudCounterSet = CounterTestUtils.extractCounterUpdates(counters, false); + Assert.assertEquals(cloudCounterSet.size(), 1); + Assert.assertEquals(cloudCounterSet.get(0), + CounterTestUtils.extractCounterUpdate(expectedCounter, false)); + } + @Test public void testSumInteger() throws Exception { testAggregator(Arrays.asList(2, 4, 1, 3), new Sum.SumIntegerFn(), @@ -128,12 +145,12 @@ public void testMaxDouble() throws Exception { public void testCompatibleDuplicateNames() throws Exception { CounterSet counters = new CounterSet(); Aggregator aggregator1 = - new AggregatorImpl, Integer>( + new AggregatorImpl( AGGREGATOR_NAME, new Sum.SumIntegerFn(), counters.getAddCounterMutator()); Aggregator aggregator2 = - new AggregatorImpl, Integer>( + new AggregatorImpl( AGGREGATOR_NAME, new Sum.SumIntegerFn(), counters.getAddCounterMutator()); @@ -148,7 +165,7 @@ public void testCompatibleDuplicateNames() throws Exception { @Test public void testIncompatibleDuplicateNames() throws Exception { CounterSet counters = new CounterSet(); - new AggregatorImpl, Integer>( + new AggregatorImpl( AGGREGATOR_NAME, new Sum.SumIntegerFn(), counters.getAddCounterMutator()); @@ -156,7 +173,7 @@ public void testIncompatibleDuplicateNames() throws Exception { expectedEx.expectMessage(Matchers.containsString( "aggregator's name collides with an existing aggregator or " + "system-provided counter of an incompatible type")); - new AggregatorImpl, Long>( + new AggregatorImpl( AGGREGATOR_NAME, new Sum.SumLongFn(), counters.getAddCounterMutator()); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java index 94cf19599ab2a..cef060822f060 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/StreamingGroupAlsoByWindowsDoFnTest.java @@ -24,7 +24,6 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; import com.google.cloud.dataflow.sdk.transforms.Sum; @@ -263,7 +262,7 @@ public class StreamingGroupAlsoByWindowsDoFnTest { } @Test public void testSessionsCombine() throws Exception { - CombineFn combineFn = Combine.SimpleCombineFn.of(new Sum.SumLongFn()); + CombineFn combineFn = new Sum.SumLongFn(); DoFnRunner>, KV, List> runner = makeRunner(Sessions.withGapDuration(Duration.millis(10)), From 48ae86accd33821658d96be39c9b55d6497e972a Mon Sep 17 00:00:00 2001 From: vanya Date: Wed, 11 Mar 2015 23:55:48 -0700 Subject: [PATCH 0261/1541] Retry GCS file read on transient problems that cause RuntimeExceptions instead of IOExceptions. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88415354 --- .../gcsio/GoogleCloudStorageReadChannel.java | 34 +++++++++++++------ 1 file changed, 24 insertions(+), 10 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java index 689a57495a253..a67d18205873d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/gcsio/GoogleCloudStorageReadChannel.java @@ -255,16 +255,17 @@ public int read(ByteBuffer buffer) int remainingBeforeRead = buffer.remaining(); try { int numBytesRead = readChannel.read(buffer); - Preconditions.checkState(numBytesRead != 0, "Read 0 bytes without blocking!"); + checkIOPrecondition(numBytesRead != 0, "Read 0 bytes without blocking"); if (numBytesRead < 0) { // Check that we didn't get a premature End of Stream signal by checking the number of // bytes read against the stream size. Unfortunately we don't have information about the // actual size of the data stream when stream compression is used, so we can only ignore // this case here. - Preconditions.checkState(isCompressedStream || currentPosition == size, - "Received end of stream result before all the file data has been received; " - + "totalBytesRead: %s, currentPosition: %s, size: %s", - totalBytesRead, currentPosition, size); + checkIOPrecondition(isCompressedStream || currentPosition == size, + String.format( + "Received end of stream result before all the file data has been received; " + + "totalBytesRead: %s, currentPosition: %s, size: %s", + totalBytesRead, currentPosition, size)); break; } totalBytesRead += numBytesRead; @@ -341,7 +342,7 @@ public int read(ByteBuffer buffer) readChannel.close(); readChannel = null; } catch (SSLException ssle) { - LOG.warn("Got SSLException on readChannel.close() before retry; ignoring it.", ssle); + LOG.debug("Got SSLException on readChannel.close() before retry; ignoring it.", ssle); readChannel = null; } // For "other" exceptions, we'll let it propagate out without setting readChannel to @@ -361,9 +362,9 @@ public int read(ByteBuffer buffer) // Check that we didn't get a premature End of Stream signal by checking the number of bytes // read against the stream size. Unfortunately we don't have information about the actual size // of the data stream when stream compression is used, so we can only ignore this case here. - Preconditions.checkState(isCompressedStream || currentPosition == size, - "Failed to read any data before all the file data has been received; " - + "currentPosition: %s, size: %s", currentPosition, size); + checkIOPrecondition(isCompressedStream || currentPosition == size, + String.format("Failed to read any data before all the file data has been received; " + + "currentPosition: %s, size: %s", currentPosition, size)); return -1; } else { return totalBytesRead; @@ -490,7 +491,7 @@ protected void validatePosition(long newPosition) { /** * Seeks to the given position in the underlying stream. * - * Note: Seek is an expensive operation because a new stream is opened each time. + *

    Note: Seek is an expensive operation because a new stream is opened each time. * * @throws java.io.FileNotFoundException if the underlying object does not exist. * @throws IOException on IO error @@ -583,4 +584,17 @@ private void throwIfNotOpen() throw new ClosedChannelException(); } } + + /** + * Throws an IOException if precondition is false. + * + *

    This method should be used in place of Preconditions.checkState in cases where the + * precondition is derived from the status of the IO operation. That makes it possible to retry + * the operation by catching IOException. + */ + private void checkIOPrecondition(boolean precondition, String errorMessage) throws IOException { + if (!precondition) { + throw new IOException(errorMessage); + } + } } From ca2e7dafb8cb021ffb9211f84d0ab86220b9af66 Mon Sep 17 00:00:00 2001 From: kirpichov Date: Thu, 12 Mar 2015 13:31:44 -0700 Subject: [PATCH 0262/1541] Improves error messages in AvroReader and FileBasedReader when filepattern matches zero files or multiple files AND an offset range is specified. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88472006 --- .../sdk/runners/worker/AvroReader.java | 11 +++++-- .../sdk/runners/worker/FileBasedReader.java | 9 ++++-- .../sdk/runners/worker/AvroReaderTest.java | 23 ++++++++++++++ .../sdk/runners/worker/TextReaderTest.java | 30 +++++++++++++++++++ 4 files changed, 67 insertions(+), 6 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java index cf51cc7344034..92c3492e218d7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java @@ -28,6 +28,7 @@ import org.apache.avro.file.SeekableInput; import org.apache.avro.io.DatumReader; +import java.io.FileNotFoundException; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; @@ -70,16 +71,20 @@ public AvroReader(String filename, @Nullable Long startPosition, @Nullable Long public ReaderIterator> iterator(DatumReader datumReader) throws IOException { IOChannelFactory factory = IOChannelUtils.getFactory(filename); Collection inputs = factory.match(filename); + if (inputs.isEmpty()) { + throw new FileNotFoundException("No match for file pattern '" + filename + "'"); + } if (inputs.size() == 1) { String input = inputs.iterator().next(); ReadableByteChannel reader = factory.open(input); return new AvroFileIterator(datumReader, input, reader, startPosition, endPosition); - } else { if (startPosition != null || endPosition != null) { - throw new UnsupportedOperationException( - "Unable to apply range limits to multiple-input stream: " + filename); + throw new IllegalArgumentException( + "Offset range specified: [" + startPosition + ", " + endPosition + "), so " + + "an exact filename was expected, but more than 1 file matched \"" + filename + + "\" (total " + inputs.size() + "): apparently a filepattern was given."); } return new AvroFileMultiIterator(datumReader, factory, inputs.iterator()); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java index d9e8bd2ab09fb..f55165c4e5ba7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java @@ -35,6 +35,7 @@ import java.io.BufferedInputStream; import java.io.ByteArrayOutputStream; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.io.PushbackInputStream; @@ -98,13 +99,15 @@ public ReaderIterator iterator() throws IOException { IOChannelFactory factory = IOChannelUtils.getFactory(filename); Collection inputs = factory.match(filename); if (inputs.isEmpty()) { - throw new IOException("No match for file pattern '" + filename + "'"); + throw new FileNotFoundException("No match for file pattern '" + filename + "'"); } if (startPosition != null || endPosition != null) { if (inputs.size() != 1) { - throw new UnsupportedOperationException( - "Unable to apply range limits to multiple-input stream: " + filename); + throw new IllegalArgumentException( + "Offset range specified: [" + startPosition + ", " + endPosition + "), so " + + "an exact filename was expected, but more than 1 file matched \"" + filename + + "\" (total " + inputs.size() + "): apparently a filepattern was given."); } return newReaderIteratorForRangeInFile(factory, inputs.iterator().next(), diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderTest.java index d5ea3b82664ab..957da90ab5807 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderTest.java @@ -31,11 +31,13 @@ import org.junit.Assert; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import java.io.File; +import java.io.FileNotFoundException; import java.io.OutputStream; import java.nio.channels.Channels; import java.util.ArrayList; @@ -52,6 +54,8 @@ public class AvroReaderTest { @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); + @Rule + public ExpectedException expectedException = ExpectedException.none(); private void runTestRead( List> elemsList, AvroCoder coder, boolean requireExactMatch) throws Exception { @@ -179,6 +183,25 @@ public void testReadBigRanges() throws Exception { false/* don't require exact match */); } + @Test + public void testErrorOnFileNotFound() throws Exception { + expectedException.expect(FileNotFoundException.class); + readElems("file-not-found", 0L, 100L, AvroCoder.of(String.class), new ArrayList()); + } + + @Test + public void testErrorOnMultipleFiles() throws Exception { + File file1 = tmpFolder.newFile("foo1.avro"); + File file2 = tmpFolder.newFile("foo2.avro"); + Channels.newOutputStream(IOChannelUtils.create(file1.getPath(), MimeTypes.BINARY)).close(); + Channels.newOutputStream(IOChannelUtils.create(file2.getPath(), MimeTypes.BINARY)).close(); + + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("more than 1 file matched"); + readElems(new File(tmpFolder.getRoot(), "*").getPath(), 0L, 100L, + AvroCoder.of(String.class), new ArrayList()); + } + // TODO: sharded filenames // TODO: reading from GCS } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java index ea7a3e5c37882..857f0b66bde82 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java @@ -38,6 +38,8 @@ import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.io.TextIO.CompressionType; import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.MimeTypes; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; import com.google.cloud.dataflow.sdk.util.common.worker.Reader; @@ -45,15 +47,18 @@ import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import java.io.File; +import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; import java.io.PrintStream; +import java.nio.channels.Channels; import java.util.ArrayList; import java.util.Arrays; import java.util.LinkedList; @@ -79,6 +84,8 @@ public class TextReaderTest { @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); + @Rule + public ExpectedException expectedException = ExpectedException.none(); private File initTestFile() throws IOException { File tmpFile = tmpFolder.newFile(); @@ -651,6 +658,29 @@ public void testCompressionTypeFileGlob() throws IOException { } } + @Test + public void testErrorOnFileNotFound() throws Exception { + expectedException.expect(FileNotFoundException.class); + TextReader textReader = new TextReader<>( + "file-not-found", true, 0L, 100L, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); + textReader.iterator(); + } + + @Test + public void testErrorOnMultipleFiles() throws Exception { + File file1 = tmpFolder.newFile("foo1.avro"); + File file2 = tmpFolder.newFile("foo2.avro"); + Channels.newOutputStream(IOChannelUtils.create(file1.getPath(), MimeTypes.BINARY)).close(); + Channels.newOutputStream(IOChannelUtils.create(file2.getPath(), MimeTypes.BINARY)).close(); + TextReader textReader = new TextReader<>( + new File(tmpFolder.getRoot(), "*").getPath(), true, 0L, 100L, + StringUtf8Coder.of(), TextIO.CompressionType.UNCOMPRESSED); + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("more than 1 file matched"); + textReader.iterator(); + } + // TODO: sharded filenames // TODO: reading from GCS } From f0d71517219cb416140db5d4a6a36e4f0fb3834c Mon Sep 17 00:00:00 2001 From: kirpichov Date: Thu, 12 Mar 2015 15:26:35 -0700 Subject: [PATCH 0263/1541] Makes a few ReaderIterator classes treat the condition "requestFork at a position below the current stop position" as normal (but of course not fork-inducing), rather than throw an exception. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88484651 --- .../sdk/runners/worker/FileBasedReader.java | 7 ++++--- .../runners/worker/GroupingShuffleReader.java | 7 ++++--- .../sdk/runners/worker/InMemoryReader.java | 7 ++++--- .../worker/GroupingShuffleReaderTest.java | 21 ++++--------------- .../runners/worker/InMemoryReaderTest.java | 11 +--------- .../sdk/runners/worker/TextReaderTest.java | 10 +-------- 6 files changed, 18 insertions(+), 45 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java index f55165c4e5ba7..b5f1e46bf3edf 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/FileBasedReader.java @@ -218,9 +218,10 @@ public ForkResult requestFork(ForkRequest forkRequest) { } if (endOffset != null && forkOffset >= endOffset) { - throw new IllegalArgumentException( - "Fork requested at an offset beyond the end of the current range: " + forkOffset - + " >= " + endOffset); + LOG.info( + "Fork requested at an offset beyond the end of the current range: {} >= {}", + forkOffset, endOffset); + return null; } this.endOffset = forkOffset; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java index 780949d935958..9e39250007b13 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java @@ -250,10 +250,11 @@ public ForkResult requestFork(ForkRequest forkRequest) { } if (this.stopPosition != null && newStopPosition.compareTo(this.stopPosition) >= 0) { - throw new IllegalArgumentException( + LOG.info( "Fork requested at a shuffle position beyond the end of the current range: " - + forkShufflePosition - + " >= current stop position: " + this.stopPosition.encodeBase64()); + + "{} >= current stop position: {}", + forkShufflePosition, this.stopPosition.encodeBase64()); + return null; } this.stopPosition = newStopPosition; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java index 1bb1841466571..da6b209940a33 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java @@ -146,9 +146,10 @@ public ForkResult requestFork(ForkRequest forkRequest) { return null; } if (forkIndex >= endPosition) { - throw new IllegalArgumentException( - "Fork requested at an index beyond the end of the current range: " + forkIndex - + " >= " + endPosition); + LOG.info( + "Fork requested at an index beyond the end of the current range: {} >= {}", + forkIndex, endPosition); + return null; } this.endPosition = forkIndex.intValue(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java index 2bc08717ed992..c20626e70e59c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java @@ -26,7 +26,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -50,7 +49,6 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.common.collect.Lists; -import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; @@ -295,13 +293,7 @@ public void testReadFromEmptyShuffleDataAndRequestFork() throws Exception { stop = encodeBase64URLSafeString(fabricatePosition(1, null)); proposedForkPosition.setShufflePosition(stop); - try { - iter.requestFork(toForkRequest(createApproximateProgress(proposedForkPosition))); - fail("IllegalArgumentException expected"); - } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), Matchers.containsString( - "Fork requested at a shuffle position beyond the end of the current range")); - } + assertNull(iter.requestFork(toForkRequest(createApproximateProgress(proposedForkPosition)))); } } @@ -330,14 +322,9 @@ public void testReadFromShuffleDataAndFailToFork() throws Exception { try (Reader.ReaderIterator>>> iter = groupingShuffleReader.iterator(shuffleReader)) { - // Cannot fork since the value provided is past the current stop position. - try { - iter.requestFork(forkRequestAtPosition(makeShufflePosition(kNumRecords + 1, null))); - fail("IllegalArgumentException expected"); - } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), Matchers.containsString( - "Fork requested at a shuffle position beyond the end of the current range")); - } + // Cannot fork since the value provided is past the current stop position. + assertNull( + iter.requestFork(forkRequestAtPosition(makeShufflePosition(kNumRecords + 1, null)))); int i = 0; for (; iter.hasNext(); ++i) { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java index 2b957e46affaa..7d7ac3dbc3a3e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java @@ -27,9 +27,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import com.google.api.services.dataflow.model.ApproximateProgress; import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; @@ -37,7 +35,6 @@ import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; import com.google.cloud.dataflow.sdk.util.common.worker.Reader; -import org.hamcrest.Matchers; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -176,13 +173,7 @@ public void testFork() throws Exception { // Proposed fork position is after the current stop (end) position, no update. try (InMemoryReader.InMemoryReaderIterator iterator = (InMemoryReader.InMemoryReaderIterator) inMemoryReader.iterator()) { - try { - iterator.requestFork(forkRequestAtIndex(end + 1)); - fail("IllegalArgumentException expected"); - } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), Matchers.containsString( - "Fork requested at an index beyond the end of the current range")); - } + assertNull(iterator.requestFork(forkRequestAtIndex(end + 1))); assertEquals((int) end, iterator.endPosition); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java index 857f0b66bde82..1c245973badbf 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java @@ -44,7 +44,6 @@ import com.google.cloud.dataflow.sdk.util.common.worker.Reader; import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; -import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -472,14 +471,7 @@ public void testUpdateStopPosition() throws Exception { try (TextReader.TextFileIterator iterator = (TextReader.TextFileIterator) textReader.iterator()) { assertEquals(fileContent[0], iterator.next()); - try { - iterator.requestFork(forkRequestAtByteOffset(stop)); - fail("IllegalArgumentException expected"); - } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), Matchers.containsString( - "Fork requested at an offset beyond the end of the current range")); - } - + assertNull(iterator.requestFork(forkRequestAtByteOffset(stop))); assertEquals(end, iterator.getEndOffset().longValue()); assertFalse(iterator.hasNext()); assertEquals(Arrays.asList(fileContent[0].length()), observer.getActualSizes()); From 05c1ca13089d2be1cafa736ff4f8aebdcbb219ca Mon Sep 17 00:00:00 2001 From: millsd Date: Thu, 12 Mar 2015 17:26:22 -0700 Subject: [PATCH 0264/1541] Move GlobalData requests to the top level of GetData ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88495958 --- sdk/src/main/proto/windmill.proto | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/proto/windmill.proto b/sdk/src/main/proto/windmill.proto index 64d6d18a40044..09cd2e1ddc63f 100644 --- a/sdk/src/main/proto/windmill.proto +++ b/sdk/src/main/proto/windmill.proto @@ -127,7 +127,6 @@ message KeyedGetDataRequest { required fixed64 work_token = 2; repeated TagValue values_to_fetch = 3; repeated TagList lists_to_fetch = 4; - repeated GlobalDataId global_data_to_fetch = 5; } message ComputationGetDataRequest { @@ -137,6 +136,7 @@ message ComputationGetDataRequest { message GetDataRequest { repeated ComputationGetDataRequest requests = 1; + repeated GlobalDataId global_data_to_fetch = 2; } message KeyedGetDataResponse { @@ -145,7 +145,6 @@ message KeyedGetDataResponse { optional bool failed = 2; repeated TagValue values = 3; repeated TagList lists = 4; - repeated GlobalData global_data = 5; } message ComputationGetDataResponse { @@ -155,6 +154,7 @@ message ComputationGetDataResponse { message GetDataResponse { repeated ComputationGetDataResponse data = 1; + repeated GlobalData global_data = 2; } // CommitWork From 1167ab9df8cbb14251df5d78b9f6f11d0f22768b Mon Sep 17 00:00:00 2001 From: lcwik Date: Fri, 13 Mar 2015 09:44:26 -0700 Subject: [PATCH 0265/1541] Swap to use varargs instead of array based setters within PipelineOptions. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88554068 --- .../dataflow/sdk/options/DataflowWorkerLoggingOptions.java | 2 +- .../cloud/dataflow/sdk/options/GoogleApiDebugOptions.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java index a38cd44ab7e02..3f59c8365495c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java @@ -52,7 +52,7 @@ public enum Level { * {@link WorkerLogLevelOverride#create(String)}. */ WorkerLogLevelOverride[] getWorkerLogLevelOverrides(); - void setWorkerLogLevelOverrides(WorkerLogLevelOverride[] workerLogLevelOverrides); + void setWorkerLogLevelOverrides(WorkerLogLevelOverride... workerLogLevelOverrides); /** * Defines a log level override for a specific class, package, or name. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java index ad1ca996075d2..4b3d69c87a9e8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java @@ -38,7 +38,7 @@ public interface GoogleApiDebugOptions extends PipelineOptions { * An invalid tracing token will result in 400 errors from Google when the API is invoked. */ GoogleApiTracer[] getGoogleApiTrace(); - void setGoogleApiTrace(GoogleApiTracer[] commands); + void setGoogleApiTrace(GoogleApiTracer... commands); /** * A {@link GoogleClientRequestInitializer} which adds the 'trace' token to Google API calls. From d805b83041ce0859c0a9f34668970e828c48c593 Mon Sep 17 00:00:00 2001 From: millsd Date: Fri, 13 Mar 2015 11:05:28 -0700 Subject: [PATCH 0266/1541] Add GlobalData updates to the Windmill API. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88561610 --- sdk/src/main/proto/windmill.proto | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/proto/windmill.proto b/sdk/src/main/proto/windmill.proto index 09cd2e1ddc63f..76b17a51dd6b3 100644 --- a/sdk/src/main/proto/windmill.proto +++ b/sdk/src/main/proto/windmill.proto @@ -178,7 +178,7 @@ message Counter { optional int64 mean_count = 6; } -// next id: 10 +// next id: 11 message WorkItemCommitRequest { required bytes key = 1; required fixed64 work_token = 2; @@ -189,6 +189,7 @@ message WorkItemCommitRequest { repeated TagList list_updates = 6; repeated Counter counter_updates = 8; repeated GlobalDataId global_data_id_requests = 9; + repeated GlobalData global_data_updates = 10; } message ComputationCommitWorkRequest { From b9f4384544149d7ed9a5f9d8ce649fd918fedca1 Mon Sep 17 00:00:00 2001 From: chamikara Date: Fri, 13 Mar 2015 11:07:14 -0700 Subject: [PATCH 0267/1541] Introduces FileBasedSource, a custom source that implements functionality common to all file-based custom sources. Also introduces ByteOffsetBasedSource as the parent of FileBasedSource. FileBasedSource supports both single files and file patterns. Additionally introduces readers ByteOffsetBasedReader and FileBasedReader to encapsulate code common to readers of ByteOffsetBasedSources and FileBasedSources respectively. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88561798 --- .../sdk/io/ByteOffsetBasedSource.java | 161 ++++ .../dataflow/sdk/io/FileBasedSource.java | 514 +++++++++++++ .../sdk/io/ByteOffsetBasedSourceTest.java | 126 ++++ .../dataflow/sdk/io/FileBasedSourceTest.java | 689 ++++++++++++++++++ 4 files changed, 1490 insertions(+) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ByteOffsetBasedSource.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ByteOffsetBasedSourceTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/io/FileBasedSourceTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ByteOffsetBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ByteOffsetBasedSource.java new file mode 100644 index 0000000000000..4d48d210a5464 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ByteOffsetBasedSource.java @@ -0,0 +1,161 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.common.base.Preconditions; + +import java.util.ArrayList; +import java.util.List; + +/** + * A source that uses byte offsets to define starting and ending positions. Extend this class to + * implement your own byte offset based custom source. {@link FileBasedSource} which is a subclass + * of this adds additional functionality useful for custom sources that are based on files. If + * possible implementors should start from {@code FileBasedSource} instead of + * {@code ByteOffsetBasedSource}. + * + *

    This is a common base class for all sources that use a byte offset range. It stores the range + * and implements splitting into shards. This should be used for sources which can be cheaply read + * starting at any given byte offset. + * + *

    The byte offset range of the source is between {@code startOffset} (inclusive) and endOffset + * (exclusive), i.e. [{@code startOffset}, {@code endOffset}). The source may include a record if + * its offset is at the range [{@code startOffset}, {@code endOffset}) even if the record extend + * past the range. The source does not include any record at offsets before this range even if it + * extend into this range because the previous range will include this record. A source may choose + * to include records at offsets after this range. For example, a source may choose to set offset + * boundaries based on blocks of records in which case certain records may start after + * {@code endOffset}. But for any given source type the combined set of data read by two sources for + * ranges [A, B) and [B, C) must be the same as the records read by a single source of the same type + * for the range [A, C). + * + * @param Type of records represented by the source. + */ +public abstract class ByteOffsetBasedSource extends Source { + private final long startOffset; + private final long endOffset; + private final long minShardSize; + + /** + * @param startOffset starting byte offset (inclusive) of the source. Must be non-negative. + * + * @param endOffset ending byte offset (exclusive) of the source. Any + * {@code offset >= getMaxEndOffset()}, e.g., {@code Long.MAX_VALUE}, means the same as + * {@code getMaxEndOffset()}. Must be {@code >= startOffset}. + * + * @param minShardSize minimum shard size in bytes that should be used when splitting the source + * into sub-sources. This will not be respected if the total range of the source is smaller + * than the specified {@code minShardSize}. Must be non-negative. + */ + public ByteOffsetBasedSource(long startOffset, long endOffset, long minShardSize) { + this.startOffset = startOffset; + this.endOffset = endOffset; + this.minShardSize = minShardSize; + Preconditions.checkArgument(startOffset >= 0, + "Start offset has value " + startOffset + ", must be non-negative"); + Preconditions.checkArgument(endOffset >= 0, + "End offset has value " + endOffset + ", must be non-negative"); + Preconditions.checkArgument(minShardSize >= 0, + "minShardSize has value " + minShardSize + ", must be non-negative"); + } + + /** + * Returns the starting offset of the source. + */ + public long getStartOffset() { + return startOffset; + } + + /** + * Returns the specified ending offset of the source. If this is {@code >= getMaxEndOffset()}, + * e.g. Long.MAX_VALUE, this implies {@code getMaxEndOffset()}. + */ + public long getEndOffset() { + return endOffset; + } + + /** + * Returns the minimum shard size that should be used when splitting the source into sub-sources. + * This will not be respected if the total range of the source is smaller than the specified + * {@code minShardSize}. + */ + public long getMinShardSize() { + return minShardSize; + } + + @Override + public List> splitIntoShards(long desiredShardSizeBytes, + PipelineOptions options) throws Exception { + // Split the range into shards based on the desiredShardSizeBytes. Final shard is adjusted to + // make sure that we do not end up with a too small shard at the end. If desiredShardSizeBytes + // is smaller than the minShardSize of the source then minShardSize will be used instead. + + desiredShardSizeBytes = Math.max(desiredShardSizeBytes, minShardSize); + + List> subSources = new ArrayList<>(); + long start = startOffset; + long maxEnd = Math.min(endOffset, getMaxEndOffset(options)); + + while (start < maxEnd) { + long end = start + desiredShardSizeBytes; + end = Math.min(end, maxEnd); + // Avoid having a too small shard at the end and ensure that we respect minShardSize. + long remainingBytes = maxEnd - end; + if ((remainingBytes < desiredShardSizeBytes / 4) || (remainingBytes < minShardSize)) { + end = maxEnd; + } + subSources.add(createSourceForSubrange(start, end)); + + start = end; + } + return subSources; + } + + /** + * Returns the exact ending offset of the current source. This will be used if the source was + * constructed with an endOffset value {@code Long.MAX_VALUE}. + */ + public abstract long getMaxEndOffset(PipelineOptions options) throws Exception; + + /** + * Returns a {@code ByteOffsetBasedSource} for a subrange of the current source. [start, end) will + * be within the range [startOffset, endOffset] of the current source. + */ + public abstract ByteOffsetBasedSource createSourceForSubrange(long start, long end); + + /** + * A reader that implements code common to readers of all {@link ByteOffsetBasedSource}s. + */ + public abstract static class ByteOffsetBasedReader implements Reader { + + /** + * @param source the {@code ByteOffsetBasedSource} to be read by the current reader. + */ + public ByteOffsetBasedReader(ByteOffsetBasedSource source) {} + + /** + * Returns the current offset of the reader. The value returned by this method is undefined + * until the method {@link Source.Reader#start} is called. After {@link Source.Reader#start} is + * called the value returned by this method should represent the offset of the value that will + * be returned by the {@link Source.Reader#getCurrent} call. Values returned for two consecutive + * records should be non-strictly increasing. If the reader has reached the end of the stream + * this should return {@code Long.MAX_VALUE}. The value returned may be outside the range + * defined by the {@code ByteOffsetBasedSource} corresponding to this reader, for reasons + * described in the comment to {@code ByteOffsetBasedSource}. + */ + protected abstract long getCurrentOffset(); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java new file mode 100644 index 0000000000000..865d923713de5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java @@ -0,0 +1,514 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.IOChannelFactory; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.common.collect.ImmutableList; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.SeekableByteChannel; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * A common base class for all file-based {@link Source}s. Extend this class to implement your own + * file-based custom source. + * + *

    A file-based {@code Source} is a {@code Source} backed by a file pattern defined as a Java + * glob, a single file, or a offset range for a single file. See {@link ByteOffsetBasedSource} for + * semantics of offset ranges. + * + *

    This source stores a {@code String} that is an {@link IOChannelFactory} specification for a + * file or file pattern. There should be an {@code IOChannelFactory} defined for the file + * specification provided. Please refer to {@link IOChannelUtils} and {@link IOChannelFactory} for + * more information on this. + * + *

    In addition to the methods left abstract from {@code Source}, subclasses must implement + * methods to create a sub-source and a reader for a range of a single file - + * {@link #createForSubrangeOfFile} and {@link #createSingleFileReader}. + * + * @param Type of records represented by the source. + */ +public abstract class FileBasedSource extends ByteOffsetBasedSource { + private static final Logger LOG = LoggerFactory.getLogger(FileBasedSource.class); + + private final String fileOrPatternSpec; + private final Mode mode; + + /** + * A given {@code FileBasedSource} represents a file resource of one of these types. + */ + public enum Mode { + FILEPATTERN, FULL_SINGLE_FILE, SUBRANGE_OF_SINGLE_FILE + } + + /** + * Create a {@code FileBasedSource} based on a file or a file pattern specification. + * + *

    See {@link ByteOffsetBasedSource} for detailed descriptions of {@code minShardSize}, + * {@code startOffset}, and {@code endOffset}. + * + * @param isFilePattern if {@code true} provided {@code fileOrPatternSpec} may be a file pattern + * and {@code FileBasedSource} will try to expand the file pattern, if {@code false} + * provided {@code fileOrPatternSpec} will be considered a single file and will be used + * verbatim. + * @param fileOrPatternSpec {@link IOChannelFactory} specification of file or file pattern + * represented by the {@link FileBasedSource}. + * @param minShardSize minimum shard size in bytes. + * @param startOffset starting byte offset. + * @param endOffset ending byte offset. If the specified value {@code >= #getMaxEndOffset()} it + * implies {@code #getMaxEndOffSet()}. + */ + public FileBasedSource(boolean isFilePattern, String fileOrPatternSpec, long minShardSize, + long startOffset, long endOffset) { + super(startOffset, endOffset, minShardSize); + if (isFilePattern) { + mode = Mode.FILEPATTERN; + } else if (startOffset == 0 && endOffset == Long.MAX_VALUE) { + mode = Mode.FULL_SINGLE_FILE; + } else { + mode = Mode.SUBRANGE_OF_SINGLE_FILE; + } + if (mode == Mode.FILEPATTERN || mode == Mode.FULL_SINGLE_FILE) { + Preconditions.checkArgument(startOffset == 0, + "FileBasedSource is based on a file pattern or a full single file but the starting offset" + + " proposed " + startOffset + " is not zero"); + Preconditions.checkArgument(startOffset == 0, + "FileBasedSource is based on a file pattern or a full single file but the starting offset" + + " proposed " + endOffset + " is not Long.MAX_VALUE"); + } + this.fileOrPatternSpec = fileOrPatternSpec; + } + + public final String getFileOrPatternSpec() { + return fileOrPatternSpec; + } + + public final Mode getMode() { + return mode; + } + + @Override + public final FileBasedSource createSourceForSubrange(long start, long end) { + Preconditions.checkArgument(mode != Mode.FILEPATTERN, + "Cannot split a file pattern based source based on positions"); + Preconditions.checkArgument(start >= getStartOffset(), "Start offset value " + start + + " of the subrange cannot be smaller than the start offset value " + getStartOffset() + + " of the parent source"); + Preconditions.checkArgument(end <= getEndOffset(), "End offset value " + end + + " of the subrange cannot be larger than the end offset value " + getEndOffset() + + " of the parent source"); + + FileBasedSource source = createForSubrangeOfFile(fileOrPatternSpec, start, end); + if (start > 0 || end != Long.MAX_VALUE) { + Preconditions.checkArgument(source.getMode() == Mode.SUBRANGE_OF_SINGLE_FILE, + "Source created for the range [" + start + "," + end + ")" + + " must be a subrange source"); + } + return source; + } + + /** + * Creates and returns a new {@code FileBasedSource} of the same type as the current + * {@code FileBasedSource} backed by a given file and an offset range. When current source is + * being split, this method is used to generate new sub-sources. When creating the source + * subclasses must call the constructor of {@code FileBasedSource} with exactly the same + * {@code start} and {@code end} values passed here. + * + * @param fileName file backing the new {@code FileBasedSource}. + * @param start starting byte offset of the new {@code FileBasedSource}. + * @param end ending byte offset of the new {@code FileBasedSource}. May be Long.MAX_VALUE, in + * which case it will be inferred using {@link #getMaxEndOffset}. + */ + public abstract FileBasedSource createForSubrangeOfFile(String fileName, long start, long end); + + /** + * Creates and returns an instance of a {@code FileBasedReader} implementation for the current + * source assuming the source represents a single file. File patterns will be handled by + * {@code FileBasedSource} implementation automatically. + */ + public abstract FileBasedReader createSingleFileReader(PipelineOptions options, Coder coder, + ExecutionContext executionContext); + + @Override + public final long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + // This implementation of method getEstimatedSizeBytes is provided to simplify subclasses. Here + // we perform the size estimation of files and file patterns using the interface provided by + // IOChannelFactory. + + IOChannelFactory factory = IOChannelUtils.getFactory(fileOrPatternSpec); + if (mode == Mode.FILEPATTERN) { + // TODO Implement a more efficient parallel/batch size estimation mechanism for file patterns. + long startTime = System.currentTimeMillis(); + long totalSize = 0; + Collection inputs = factory.match(fileOrPatternSpec); + for (String input : inputs) { + totalSize += factory.getSizeBytes(input); + } + LOG.debug("Size estimation of file pattern " + fileOrPatternSpec + " took " + + (System.currentTimeMillis() - startTime) + " ms"); + return totalSize; + } else { + long start = getStartOffset(); + long end = Math.min(getEndOffset(), getMaxEndOffset(options)); + return end - start; + } + } + + @Override + public final List> splitIntoShards(long desiredShardSizeBytes, + PipelineOptions options) throws Exception { + // This implementation of method splitIntoShards is provided to simplify subclasses. Here we + // split a FileBasedSource based on a file pattern to FileBasedSources based on full single + // files. For files that can be efficiently seeked, we further split FileBasedSources based on + // those files to FileBasedSources based on sub ranges of single files. + + if (mode == Mode.FILEPATTERN) { + long startTime = System.currentTimeMillis(); + List> splitResults = new ArrayList<>(); + for (String file : expandFilePattern()) { + splitResults.addAll(createForSubrangeOfFile(file, 0, Long.MAX_VALUE).splitIntoShards( + desiredShardSizeBytes, options)); + } + LOG.debug("Splitting the source based on file pattern " + fileOrPatternSpec + " took " + + (System.currentTimeMillis() - startTime) + " ms"); + return splitResults; + } else { + // We split a file-based source into subranges only if the file is seekable. If a file is not + // seekable it will be highly inefficient to create and read a source based on a subrange of + // that file. + IOChannelFactory factory = IOChannelUtils.getFactory(fileOrPatternSpec); + if (factory.isReadSeekEfficient(fileOrPatternSpec)) { + List> splitResults = new ArrayList<>(); + for (ByteOffsetBasedSource split : + super.splitIntoShards(desiredShardSizeBytes, options)) { + splitResults.add((FileBasedSource) split); + } + return splitResults; + } else { + LOG.debug("The source for file " + fileOrPatternSpec + + " is not split into sub-range based sources since the file is not seekable"); + return ImmutableList.of(this); + } + } + } + + @Override + protected final Reader createBasicReader(PipelineOptions options, Coder coder, + ExecutionContext executionContext) throws IOException { + if (mode == Mode.FILEPATTERN) { + long startTime = System.currentTimeMillis(); + Collection files = expandFilePattern(); + List> fileReaders = new ArrayList<>(); + for (String fileName : files) { + fileReaders.add(createForSubrangeOfFile(fileName, 0, Long.MAX_VALUE).createSingleFileReader( + options, coder, executionContext)); + } + LOG.debug("Creating a reader for file pattern " + fileOrPatternSpec + " took " + + (System.currentTimeMillis() - startTime) + " ms"); + return new FilePatternReader(fileReaders.iterator()); + } else { + return createSingleFileReader(options, coder, executionContext); + } + } + + @Override + public final long getMaxEndOffset(PipelineOptions options) throws Exception { + if (mode == Mode.FILEPATTERN) { + throw new IllegalArgumentException("Cannot determine the exact end offset of a file pattern"); + } + if (getEndOffset() == Long.MAX_VALUE) { + IOChannelFactory factory = IOChannelUtils.getFactory(fileOrPatternSpec); + return factory.getSizeBytes(fileOrPatternSpec); + } else { + return getEndOffset(); + } + } + + private Collection expandFilePattern() throws IOException { + if (mode != Mode.FILEPATTERN) { + throw new IllegalArgumentException("Not a file pattern"); + } + IOChannelFactory factory = IOChannelUtils.getFactory(fileOrPatternSpec); + return factory.match(fileOrPatternSpec); + } + + /** + * A {@link Source.Reader reader} that implements code common to readers of + * {@code FileBasedSource}s. + * + *

    Seekability

    + * + *

    This reader uses a {@link ReadableByteChannel} created for the file represented by the + * corresponding source to efficiently move to the correct starting position defined in the + * source. Subclasses of this reader should implement {@link #startReading} to get access to this + * channel. If the source corresponding to the reader is for a subrange of a file the + * {@code ReadableByteChannel} provided is guaranteed to be an instance of the type + * {@link SeekableByteChannel} which may be used by subclass to traverse back in the channel to + * determine the correct starting position. + * + *

    Split Points

    + * + *

    Simple record-based formats (such as reading lines, reading CSV etc.), where each record can + * be identified by a unique offset, should interpret a range [A, B) as "read from the first + * record starting at or after offset A, up to but not including the first record starting at or + * after offset B". + * + *

    More complex formats, such as some block-based formats, may have records which are not + * directly addressable: i.e. for some records, there is no way to describe the location of a + * record using a single offset number. For example, imagine a file format consisting of a + * sequence of blocks, where each block is compressed using some block compression algorithm. Then + * blocks have offsets, but individual records don't. More complex cases are also possible. + * + *

    Many such formats still admit reading a range of offsets in a way consistent with the + * semantics of {@code ByteOffsetBasedReader}, i.e. reading [A, B) and [B, C) is equivalent to + * reading [A, C). E.g., for the compressed block-based format discussed above, reading [A, B) + * would mean "read all the records in all blocks whose starting offset is in [A, B)". + * + *

    To support such complex formats in {@code FileBasedReader}, we introduce the notion of + * split points. We say that a record is a split point if there exists an offset A such + * that the record is the first one to be read for a range [A, {@code Long.MAX_VALUE}). E.g. for + * the block-based format above, the only split points would be the first records in each block. + * + *

    With the above definition of split points an extended definition of the offset of a record + * can be specified. For a record which is at a split point, its offset is defined to be the + * largest A such that reading a source with the range [A, Long.MAX_VALUE) includes this record; + * offsets of other records are only required to be non-strictly increasing. Offsets of records of + * a {@code FileBasedReader} should be set based on this definition. + * + *

    Reading Records

    + * + *

    Sequential reading is implemented using {@link #readNextRecord}. + * + *

    Then {@code FileBasedReader} implements "reading a range [A, B)" in the following way. + *

      + *
    1. {@code start()} opens the file + *
    2. {@code start()} seeks the {@code SeekableByteChannel} to A (reading offset ranges for + * non-seekable files is not supported) and calls {@code startReading()} + *
    3. the subclass must do whatever is needed to move to the first split point at or after this + * position in the channel + *
    4. {@code start()} calls {@code advance()} once + *
    5. if the previous advance call returned {@code true} sequential reading starts and + * {@code advance()} will be called repeatedly + *
    + * {@code advance()} calls {@code readNextRecord()} on the subclass, and stops (returns false) if + * the new record is at a split point AND the offset of the new record is at or after B. + * + *

    Thread Safety

    + * + * Since this class implements {@link Source.Reader} it guarantees thread safety. Abstract methods + * defined here will not be accessed by more than one thread concurrently. + */ + public abstract static class FileBasedReader extends ByteOffsetBasedReader { + private ReadableByteChannel channel = null; + private boolean finished = false; // Reader has finished advancing. + private boolean endPositionReached = false; // If true, records have been read up to the ending + // offset but the last split point may not have been + // reached. + private boolean startCalled = false; + private FileBasedSource source = null; + + /** + * Subclasses should not perform IO operations at the constructor. All IO operations should be + * delayed until the {@link #startReading} method is invoked. + */ + public FileBasedReader(FileBasedSource source) { + super(source); + Preconditions.checkArgument(source.getMode() != Mode.FILEPATTERN, + "FileBasedReader does not support reading file patterns"); + this.source = source; + } + + protected final FileBasedSource getSource() { + return source; + } + + @Override + public final boolean start() throws IOException { + Preconditions.checkState(!startCalled, "start() should only be called once"); + IOChannelFactory factory = IOChannelUtils.getFactory(source.getFileOrPatternSpec()); + this.channel = factory.open(source.getFileOrPatternSpec()); + + if (channel instanceof SeekableByteChannel) { + SeekableByteChannel seekChannel = (SeekableByteChannel) channel; + seekChannel.position(source.getStartOffset()); + } else { + // Channel is not seekable. Must not be a subrange. + Preconditions.checkArgument(source.mode != Mode.SUBRANGE_OF_SINGLE_FILE, + "Subrange-based sources must only be defined for file types that support seekable " + + " read channels"); + Preconditions.checkArgument(source.getStartOffset() == 0, "Start offset " + + source.getStartOffset() + + " is not zero but channel for reading the file is not seekable."); + } + + startReading(channel); + startCalled = true; + + // Advance once to load the first record. + return advance(); + } + + @Override + public final boolean advance() throws IOException { + Preconditions.checkState(startCalled, "advance() called before calling start()"); + if (finished) { + return false; + } + + if (!readNextRecord()) { + // End of the stream reached. + finished = true; + return false; + } + if (getCurrentOffset() >= source.getEndOffset()) { + // Current record is at or after the end position defined by the source. The reader should + // continue reading until the next split point is reached. + endPositionReached = true; + } + + // If the current record is at or after the end position defined by the source and if the + // current record is at a split point, then the current record, and any record after that + // does not belong to the offset range of the source. + if (endPositionReached && isAtSplitPoint()) { + finished = true; + return false; + } + + return true; + } + + /** + * Closes any {@link ReadableByteChannel} created for the current reader. This implementation is + * idempotent. Any {@code close()} method introduced by a subclass must be idempotent and must + * call the {@code close()} method in the {@code FileBasedReader}. + */ + @Override + public void close() throws IOException { + if (channel != null) { + channel.close(); + } + } + + /** + * Specifies if the current record of the reader is at a split point. + * + *

    This returns {@code true} if {@link #readNextRecord} was invoked at least once and the + * last record returned by {@link #readNextRecord} is at a split point, {@code false} otherwise. + * Please refer to {@link FileBasedSource.FileBasedReader FileBasedReader} for the definition of + * split points. + */ + protected abstract boolean isAtSplitPoint(); + + /** + * Performs any initialization of the subclass of {@code FileBasedReader} that involves IO + * operations. Will only be invoked once and before that invocation the base class will seek the + * channel to the source's starting offset. + * + *

    Provided {@link ReadableByteChannel} is for the file represented by the source of this + * reader. Subclass may use the {@code channel} to build a higher level IO abstraction, e.g., a + * BufferedReader or an XML parser. + * + *

    A subclass may additionally use this to adjust the starting position prior to reading + * records. For example, the channel of a reader that reads text lines may point to the middle + * of a line after the position adjustment done at {@code FileBasedReader}. In this case the + * subclass could adjust the position of the channel to the beginning of the next line. If the + * corresponding source is for a subrange of a file, {@code channel} is guaranteed to be an + * instance of the type {@link SeekableByteChannel} in which case the subclass may traverse back + * in the channel to determine if the channel is already at the correct starting position (e.g., + * to check if the previous character was a newline). + * + *

    After this method is invoked the base class will not be reading data from the channel or + * adjusting the position of the channel. But the base class is responsible for properly closing + * the channel. + * + * @param channel a byte channel representing the file backing the reader. + */ + protected abstract void startReading(ReadableByteChannel channel) throws IOException; + + /** + * Reads the next record from the channel provided by {@link #startReading}. Methods + * {@link #getCurrent}, {@link #getCurrentOffset}, and {@link #isSplitPoint} should return the + * corresponding information about the record read by the last invocation of this method. + * + * @return {@code true} if a record was successfully read, {@code false} if the end of the + * channel was reached before successfully reading a new record. + */ + protected abstract boolean readNextRecord() throws IOException; + } + + // An internal Reader implementation that concatenates a sequence of FileBasedReaders. + private class FilePatternReader implements Reader { + final Iterator> fileReaders; + FileBasedReader currentReader = null; + + public FilePatternReader(Iterator> fileReaders) { + this.fileReaders = fileReaders; + } + + @Override + public boolean start() throws IOException { + return startNextNonemptyReader(); + } + + @Override + public boolean advance() throws IOException { + Preconditions.checkState(currentReader != null, "Call start() before advance()"); + if (currentReader.advance()) { + return true; + } + return startNextNonemptyReader(); + } + + private boolean startNextNonemptyReader() throws IOException { + while (fileReaders.hasNext()) { + currentReader = fileReaders.next(); + if (currentReader.start()) { + return true; + } + currentReader.close(); + } + return false; + } + + @Override + public T getCurrent() throws NoSuchElementException { + // A NoSuchElement will be thrown by the last FileBasedReader if getCurrent() is called after + // advance() returns false. + return currentReader.getCurrent(); + } + + @Override + public void close() throws IOException { + // Close all readers that may have not yet been closed. + currentReader.close(); + while (fileReaders.hasNext()) { + fileReaders.next().close(); + } + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ByteOffsetBasedSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ByteOffsetBasedSourceTest.java new file mode 100644 index 0000000000000..d46530a081d45 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ByteOffsetBasedSourceTest.java @@ -0,0 +1,126 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import static org.junit.Assert.assertEquals; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.List; + +/** + * Tests code common to all offset-based sources. + */ +@RunWith(JUnit4.class) +public class ByteOffsetBasedSourceTest { + + class TestByteOffsetBasedSource extends ByteOffsetBasedSource { + + private static final long serialVersionUID = 85539250; + + public TestByteOffsetBasedSource(long startOffset, long endOffset, long minShardSize) { + super(startOffset, endOffset, minShardSize); + } + + @Override + public ByteOffsetBasedSource createSourceForSubrange(long start, long end) { + return new TestByteOffsetBasedSource(start, end, 1024); + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + return 0; + } + + @Override + public boolean producesSortedKeys(PipelineOptions options) throws Exception { + return false; + } + + @Override + public void validate() {} + + @Override + public Coder getDefaultOutputCoder() { + return null; + } + + @Override + public long getMaxEndOffset(PipelineOptions options) { + return getEndOffset(); + } + } + + public static void assertSplitsAre(List> splits, + long[] expectedBoundaries) { + assertEquals(splits.size(), expectedBoundaries.length - 1); + int i = 0; + for (ByteOffsetBasedSource split : splits) { + assertEquals(split.getStartOffset(), expectedBoundaries[i]); + assertEquals(split.getEndOffset(), expectedBoundaries[i + 1]); + i++; + } + } + + @Test + public void testSplitPositionsZeroStart() throws Exception { + long start = 0; + long end = 1000; + long minShardSize = 50; + long desiredShardSize = 150; + TestByteOffsetBasedSource testSource = new TestByteOffsetBasedSource(start, end, minShardSize); + long[] boundaries = {0, 150, 300, 450, 600, 750, 900, 1000}; + assertSplitsAre(testSource.splitIntoShards(desiredShardSize, null), boundaries); + } + + @Test + public void testSplitPositionsNonZeroStart() throws Exception { + long start = 300; + long end = 1000; + long minShardSize = 50; + long desiredShardSize = 150; + TestByteOffsetBasedSource testSource = new TestByteOffsetBasedSource(start, end, minShardSize); + long[] boundaries = {300, 450, 600, 750, 900, 1000}; + assertSplitsAre(testSource.splitIntoShards(desiredShardSize, null), boundaries); + } + + @Test + public void testMinShardSize() throws Exception { + long start = 300; + long end = 1000; + long minShardSize = 150; + long desiredShardSize = 100; + TestByteOffsetBasedSource testSource = new TestByteOffsetBasedSource(start, end, minShardSize); + long[] boundaries = {300, 450, 600, 750, 1000}; + assertSplitsAre(testSource.splitIntoShards(desiredShardSize, null), boundaries); + } + + @Test + public void testSplitPositionsCollapseEndShard() throws Exception { + long start = 0; + long end = 1000; + long minShardSize = 50; + long desiredShardSize = 110; + TestByteOffsetBasedSource testSource = new TestByteOffsetBasedSource(start, end, minShardSize); + // Last 10 bytes should collapse to the previous shard. + long[] boundaries = {0, 110, 220, 330, 440, 550, 660, 770, 880, 1000}; + assertSplitsAre(testSource.splitIntoShards(desiredShardSize, null), boundaries); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/FileBasedSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/FileBasedSourceTest.java new file mode 100644 index 0000000000000..d995f9ba5ce60 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/FileBasedSourceTest.java @@ -0,0 +1,689 @@ +/* + * Copyright (C) 2014 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.when; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.io.FileBasedSource.FileBasedReader; +import com.google.cloud.dataflow.sdk.io.FileBasedSource.Mode; +import com.google.cloud.dataflow.sdk.io.Source.Reader; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.DirectPipeline; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.EvaluationResults; +import com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; +import com.google.cloud.dataflow.sdk.util.IOChannelFactory; +import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.TestCredential; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.collect.ImmutableList; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mockito; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.SeekableByteChannel; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Random; + +/** + * Tests code common to all file-based sources. + */ +@RunWith(JUnit4.class) +public class FileBasedSourceTest { + + Random random = new Random(System.currentTimeMillis()); + + @Rule + public TemporaryFolder tempFolder = new TemporaryFolder(); + + /** + * If {@code splitHeader} is null, this is just a simple line-based reader. Otherwise, the file is + * considered to consist of blocks beginning with {@code splitHeader}. The header itself is not + * returned as a record. The first record after the header is considered to be a split point. + * + *

    E.g., if {@code splitHeader} is "h" and the lines of the file are: h, a, b, h, h, c, then + * the records in this source are a,b,c, and records a and c are split points. + */ + class TestFileBasedSource extends FileBasedSource { + + private static final long serialVersionUID = 85539251; + + ReadableByteChannel channel = null; + final String splitHeader; + + public TestFileBasedSource(boolean isFilePattern, String fileOrPattern, long minShardSize, + String splitHeader) { + super(isFilePattern, fileOrPattern, minShardSize, 0L, Long.MAX_VALUE); + this.splitHeader = splitHeader; + } + + public TestFileBasedSource(String fileOrPattern, long minShardSize, long startOffset, + long endOffset, String splitHeader) { + super(false, fileOrPattern, minShardSize, startOffset, endOffset); + this.splitHeader = splitHeader; + } + + @Override + public boolean producesSortedKeys(PipelineOptions options) throws Exception { + return false; + } + + @Override + public void validate() {} + + @Override + public Coder getDefaultOutputCoder() { + return StringUtf8Coder.of(); + } + + @Override + public FileBasedSource createForSubrangeOfFile(String fileName, long start, long end) { + return new TestFileBasedSource(fileName, getMinShardSize(), start, end, splitHeader); + } + + @Override + public FileBasedReader createSingleFileReader(PipelineOptions options, + Coder coder, ExecutionContext executionContext) { + if (splitHeader == null) { + return new TestReader(this); + } else { + return new TestReaderWithSplits(this); + } + } + } + + /** + * A reader that can read lines of text from a {@link TestFileBasedSource}. This reader does not + * consider {@code splitHeader} defined by {@code TestFileBasedSource} hence every line can be the + * first line of a split. + */ + class TestReader extends FileBasedReader { + private ReadableByteChannel channel = null; + private final byte boundary; + private long nextOffset = 0; + private long currentOffset = 0; + private boolean isAtSplitPoint = false; + private final ByteBuffer buf; + private static final int BUF_SIZE = 1024; + private String currentValue = null; + + public TestReader(TestFileBasedSource source) { + super(source); + boundary = '\n'; + buf = ByteBuffer.allocate(BUF_SIZE); + buf.flip(); + } + + private int readNextLine(ByteArrayOutputStream out) throws IOException { + int byteCount = 0; + while (true) { + if (!buf.hasRemaining()) { + buf.clear(); + int read = channel.read(buf); + if (read < 0) { + break; + } + buf.flip(); + } + byte b = buf.get(); + byteCount++; + if (b == boundary) { + break; + } + out.write(b); + } + return byteCount; + } + + @Override + protected void startReading(ReadableByteChannel channel) throws IOException { + boolean removeLine = false; + if (getSource().getMode() == Mode.SUBRANGE_OF_SINGLE_FILE) { + SeekableByteChannel seekChannel = (SeekableByteChannel) channel; + // If we are not at the beginning of a line, we should ignore the current line. + if (seekChannel.position() > 0) { + // Start from one character back and read till we find a new line. + seekChannel.position(seekChannel.position() - 1); + removeLine = true; + } + nextOffset = seekChannel.position(); + } + this.channel = channel; + if (removeLine) { + nextOffset += readNextLine(new ByteArrayOutputStream()); + } + } + + @Override + protected boolean readNextRecord() throws IOException { + currentOffset = nextOffset; + + ByteArrayOutputStream buf = new ByteArrayOutputStream(); + int offsetAdjustment = readNextLine(buf); + if (offsetAdjustment == 0) { + // EOF + return false; + } + nextOffset += offsetAdjustment; + isAtSplitPoint = true; + currentValue = CoderUtils.decodeFromByteArray(StringUtf8Coder.of(), buf.toByteArray()); + return true; + } + + @Override + protected boolean isAtSplitPoint() { + return isAtSplitPoint; + } + + @Override + protected long getCurrentOffset() { + return currentOffset; + } + + @Override + public String getCurrent() throws NoSuchElementException { + return currentValue; + } + } + + /** + * A reader that can read lines of text from a {@link TestFileBasedSource}. This reader considers + * {@code splitHeader} defined by {@code TestFileBasedSource} hence only lines that immediately + * follow a {@code splitHeader} are split points. + */ + class TestReaderWithSplits extends TestReader { + private final String splitHeader; + private boolean isAtSplitPoint = false; + + public TestReaderWithSplits(TestFileBasedSource source) { + super(source); + this.splitHeader = source.splitHeader; + } + + @Override + protected void startReading(ReadableByteChannel channel) throws IOException { + super.startReading(channel); + + // Ignore all lines until next header. + if (!super.readNextRecord()) { + return; + } + String current = super.getCurrent(); + while (current == null || !current.equals(splitHeader)) { + if (!super.readNextRecord()) { + return; + } + current = super.getCurrent(); + } + } + + @Override + protected boolean readNextRecord() throws IOException { + // Get next record. If next record is a header read up to the next non-header record (ignoring + // any empty splits that does not have any records). + + isAtSplitPoint = false; + while (true) { + if (!super.readNextRecord()) { + return false; + } + String current = super.getCurrent(); + if (current == null || !current.equals(splitHeader)) { + return true; + } + isAtSplitPoint = true; + } + } + + @Override + protected boolean isAtSplitPoint() { + return isAtSplitPoint; + } + } + + public File createFileWithData(String fileName, List data) throws IOException { + File file = tempFolder.newFile(fileName); + Files.write(file.toPath(), data, StandardCharsets.UTF_8); + return file; + } + + private String createRandomString(int length) { + char[] chars = "abcdefghijklmnopqrstuvwxyz".toCharArray(); + StringBuilder builder = new StringBuilder(); + for (int i = 0; i < length; i++) { + builder.append(chars[random.nextInt(chars.length)]); + } + return builder.toString(); + } + + public List createStringDataset(int dataItemLength, int numItems) { + List list = new ArrayList(); + for (int i = 0; i < numItems; i++) { + list.add(createRandomString(dataItemLength)); + } + return list; + } + + private List readEverythingFromReader(Reader reader) throws IOException { + List results = new ArrayList(); + for (boolean available = reader.start(); available; available = reader.advance()) { + results.add(reader.getCurrent()); + } + return results; + } + + @Test + public void testFullyReadSingleFile() throws IOException { + List data = createStringDataset(3, 5000); + + String fileName = "file"; + File file = createFileWithData(fileName, data); + + TestFileBasedSource source = new TestFileBasedSource(false, file.getPath(), 1024, null); + assertEquals(data, readEverythingFromReader(source.createBasicReader(null, null, null))); + } + + @Test + public void testFullyReadFilePattern() throws IOException { + List data1 = createStringDataset(3, 1000); + File file1 = createFileWithData("file1", data1); + + List data2 = createStringDataset(3, 1000); + createFileWithData("file2", data2); + + List data3 = createStringDataset(3, 1000); + createFileWithData("file3", data3); + + List data4 = createStringDataset(3, 1000); + createFileWithData("otherfile", data4); + + TestFileBasedSource source = + new TestFileBasedSource(true, file1.getParent() + "/" + "file*", 1024, null); + List expectedResults = new ArrayList(); + expectedResults.addAll(data1); + expectedResults.addAll(data2); + expectedResults.addAll(data3); + assertThat(expectedResults, containsInAnyOrder( + readEverythingFromReader(source.createBasicReader(null, null, null)).toArray())); + } + + @Test + public void testFullyReadFilePatternFirstRecordEmpty() throws IOException { + File file1 = createFileWithData("file1", new ArrayList()); + + IOChannelFactory mockIOFactory = Mockito.mock(IOChannelFactory.class); + String parent = file1.getParent(); + String pattern = "mocked://test"; + when(mockIOFactory.match(pattern)).thenReturn( + ImmutableList.of(parent + "/" + "file1", parent + "/" + "file2", parent + "/" + "file3")); + IOChannelUtils.setIOFactory("mocked", mockIOFactory); + + List data2 = createStringDataset(3, 1000); + createFileWithData("file2", data2); + + List data3 = createStringDataset(3, 1000); + createFileWithData("file3", data3); + + List data4 = createStringDataset(3, 1000); + createFileWithData("otherfile", data4); + + TestFileBasedSource source = new TestFileBasedSource(true, pattern, 1024, null); + + List expectedResults = new ArrayList(); + expectedResults.addAll(data2); + expectedResults.addAll(data3); + assertThat(expectedResults, containsInAnyOrder( + readEverythingFromReader(source.createBasicReader(null, null, null)).toArray())); + } + + @Test + public void testReadRangeAtStart() throws IOException { + List data = createStringDataset(3, 1000); + + String fileName = "file"; + File file = createFileWithData(fileName, data); + + TestFileBasedSource source = new TestFileBasedSource(file.getPath(), 1024, 0, 102, null); + + // Each line represents 4 bytes (3 random characters + new line + // character). + // So offset range 0-102 include 26 lines. + assertEquals(data.subList(0, 26), + readEverythingFromReader(source.createBasicReader(null, null, null))); + } + + @Test + public void testReadEverythingFromFileWithSplits() throws IOException { + String header = ""; + List data = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + data.add(header); + data.addAll(createStringDataset(3, 9)); + } + String fileName = "file"; + File file = createFileWithData(fileName, data); + + TestFileBasedSource source = + new TestFileBasedSource(file.getPath(), 1024, 0, Long.MAX_VALUE, header); + + List expectedResults = new ArrayList(); + expectedResults.addAll(data); + // Remove all occurrences of header from expected results. + expectedResults.removeAll(Arrays.asList(header)); + + assertEquals(expectedResults, + readEverythingFromReader(source.createBasicReader(null, null, null))); + } + + @Test + public void testReadRangeFromFileWithSplitsFromStart() throws IOException { + String header = ""; + List data = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + data.add(header); + data.addAll(createStringDataset(3, 9)); + } + String fileName = "file"; + File file = createFileWithData(fileName, data); + + TestFileBasedSource source = new TestFileBasedSource(file.getPath(), 1024, 0, 60, header); + + List expectedResults = new ArrayList(); + expectedResults.addAll(data.subList(0, 20)); + // Remove all occurrences of header from expected results. + expectedResults.removeAll(Arrays.asList(header)); + + assertEquals(expectedResults, + readEverythingFromReader(source.createBasicReader(null, null, null))); + } + + @Test + public void testReadRangeFromFileWithSplitsFromMiddle() throws IOException { + String header = ""; + List data = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + data.add(header); + data.addAll(createStringDataset(3, 9)); + } + String fileName = "file"; + File file = createFileWithData(fileName, data); + + TestFileBasedSource source = new TestFileBasedSource(file.getPath(), 1024, 502, 702, header); + + List expectedResults = new ArrayList(); + + // Each line represents 4 bytes (3 random characters + new line + // character). + // First 126 lines take 504 bytes of space. So record starting at next split point (130) + // should be the first line that belongs to the split. + // Similarly, record at index 179 should be the last record in the split. + expectedResults.addAll(data.subList(130, 180)); + // Remove all occurrences of header from expected results. + expectedResults.removeAll(Arrays.asList(header)); + + assertEquals(expectedResults, + readEverythingFromReader(source.createBasicReader(null, null, null))); + } + + @Test + public void testReadRangeFromFileWithSplitsFromMiddleOfHeader() throws IOException { + String header = ""; + List data = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + data.add(header); + data.addAll(createStringDataset(3, 9)); + } + String fileName = "file"; + File file = createFileWithData(fileName, data); + + List expectedResults = new ArrayList(); + expectedResults.addAll(data.subList(10, 20)); + // Remove all occurrences of header from expected results. + expectedResults.removeAll(Arrays.asList(header)); + + // Split starts after "<" of the header + TestFileBasedSource source = new TestFileBasedSource(file.getPath(), 1024, 1, 60, header); + assertEquals(expectedResults, + readEverythingFromReader(source.createBasicReader(null, null, null))); + + // Split starts after "" of the header + source = new TestFileBasedSource(file.getPath(), 1024, 3, 60, header); + assertEquals(expectedResults, + readEverythingFromReader(source.createBasicReader(null, null, null))); + } + + @Test + public void testReadRangeAtMiddle() throws IOException { + List data = createStringDataset(3, 1000); + String fileName = "file"; + File file = createFileWithData(fileName, data); + + TestFileBasedSource source = new TestFileBasedSource(file.getPath(), 1024, 502, 702, null); + + // Each line represents 4 bytes (3 random characters + new line + // character). + // First 126 lines take 504 bytes of space. So 127th line (index 126) + // should be the first line that belongs to the split. + // Similarly, 176th line (index 175) should be the last line of the + // split. (Note that end index of data.subList() is exclusive). + assertEquals(data.subList(126, 176), + readEverythingFromReader(source.createBasicReader(null, null, null))); + } + + @Test + public void testReadRangeAtEnd() throws IOException { + List data = createStringDataset(3, 1000); + + String fileName = "file"; + File file = createFileWithData(fileName, data); + + TestFileBasedSource source = + new TestFileBasedSource(file.getPath(), 1024, 802, Long.MAX_VALUE, null); + + // Each line represents 4 bytes (3 random characters + new line + // character). + // First 201 lines take 804 bytes so line 202 (index 201) should be the + // first line of the split. + assertEquals(data.subList(201, data.size()), + readEverythingFromReader(source.createBasicReader(null, null, null))); + } + + @Test + public void testReadAllSplitsOfSingleFile() throws Exception { + List data = createStringDataset(3, 10000); + + String fileName = "file"; + File file = createFileWithData(fileName, data); + + TestFileBasedSource source = new TestFileBasedSource(false, file.getPath(), 1024, null); + + List> sources = source.splitIntoShards(4096, null); + // Each line is 4 bytes (3 random characters + new line character) we write + // 10,000 lines so the total size of the file is 40,000 bytes. Because of + // this above call produces 10 (40000/4096) splits. + assertEquals(sources.size(), 10); + + List results = new ArrayList(); + for (Source split : sources) { + results.addAll(readEverythingFromReader(split.createBasicReader(null, null, null))); + } + + assertEquals(data, results); + } + + @Test + public void testDataflowFile() throws IOException { + TestDataflowPipelineOptions options = + PipelineOptionsFactory.as(TestDataflowPipelineOptions.class); + options.setGcpCredential(new TestCredential()); + + DirectPipeline p = DirectPipeline.createForTest(); + List data = createStringDataset(3, 10000); + + String fileName = "file"; + File file = createFileWithData(fileName, data); + + TestFileBasedSource source = new TestFileBasedSource(false, file.getPath(), 1024, null); + + PCollection output = p.apply(ReadSource.from(source).named("ReadFileData")); + + EvaluationResults results = p.run(); + List readData = results.getPCollection(output); + + // Need to sort here since we have no control over the order of files returned from a file + // pattern expansion. + Collections.sort(data); + Collections.sort(readData); + + assertEquals(data, readData); + } + + @Test + public void testDataflowFilePattern() throws IOException { + TestDataflowPipelineOptions options = + PipelineOptionsFactory.as(TestDataflowPipelineOptions.class); + options.setGcpCredential(new TestCredential()); + + DirectPipeline p = DirectPipeline.createForTest(); + + List data1 = createStringDataset(3, 1000); + File file1 = createFileWithData("file1", data1); + + List data2 = createStringDataset(3, 1000); + createFileWithData("file2", data2); + + List data3 = createStringDataset(3, 1000); + createFileWithData("file3", data3); + + List data4 = createStringDataset(3, 1000); + createFileWithData("otherfile", data4); + + TestFileBasedSource source = + new TestFileBasedSource(true, file1.getParent() + "/" + "file*", 1024, null); + + PCollection output = p.apply(ReadSource.from(source).named("ReadFileData")); + + EvaluationResults pipelineResults = p.run(); + List results = pipelineResults.getPCollection(output); + + List expectedResults = new ArrayList(); + expectedResults.addAll(data1); + expectedResults.addAll(data2); + expectedResults.addAll(data3); + + // Need to sort here since we have no control over the order of files returned from a file + // pattern expansion. + Collections.sort(expectedResults); + Collections.sort(results); + + assertEquals(expectedResults, results); + } + + @Test + public void testEstimatedSizeOfFile() throws Exception { + List data = createStringDataset(3, 1000); + String fileName = "file"; + File file = createFileWithData(fileName, data); + + TestFileBasedSource source = new TestFileBasedSource(false, file.getPath(), 1024, null); + + // Size of the file should be 4*1000 + assertEquals(4000, source.getEstimatedSizeBytes(null)); + + } + + @Test + public void testEstimatedSizeOfFilePattern() throws Exception { + List data1 = createStringDataset(3, 500); + File file1 = createFileWithData("file1", data1); + + List data2 = createStringDataset(3, 1000); + createFileWithData("file2", data2); + + List data3 = createStringDataset(3, 1500); + createFileWithData("file3", data3); + + List data4 = createStringDataset(3, 600); + createFileWithData("otherfile", data4); + + List data5 = createStringDataset(3, 700); + createFileWithData("anotherfile", data5); + + TestFileBasedSource source = + new TestFileBasedSource(true, file1.getParent() + "/" + "file*", 1024, null); + + // Size of the pattern should be 4*(500+1000+1500) + assertEquals(12000, source.getEstimatedSizeBytes(null)); + } + + @Test + public void testReadAllSplitsOfFilePattern() throws Exception { + List data1 = createStringDataset(3, 10000); + File file1 = createFileWithData("file1", data1); + + List data2 = createStringDataset(3, 10000); + createFileWithData("file2", data2); + + List data3 = createStringDataset(3, 10000); + createFileWithData("file3", data3); + + List data4 = createStringDataset(3, 10000); + createFileWithData("otherfile", data4); + + TestFileBasedSource source = + new TestFileBasedSource(true, file1.getParent() + "/" + "file*", 1024, null); + List> sources = source.splitIntoShards(4096, null); + assertEquals(sources.size(), 30); + + List results = new ArrayList(); + for (Source split : sources) { + results.addAll(readEverythingFromReader(split.createBasicReader(null, null, null))); + } + + List expectedResults = new ArrayList(); + expectedResults.addAll(data1); + expectedResults.addAll(data2); + expectedResults.addAll(data3); + + assertThat(expectedResults, containsInAnyOrder(results.toArray())); + } +} From a791bed4845bbfe675d41608b05c5d39d2e2b590 Mon Sep 17 00:00:00 2001 From: davor Date: Fri, 13 Mar 2015 14:00:44 -0700 Subject: [PATCH 0268/1541] Make pom.xml compatible with Eclipse and m2e plugin. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88577082 --- pom.xml | 57 +++++++++++++++++++++++++++++++++++++++++++++++++++++ sdk/pom.xml | 24 ++++++++++++++++++++++ 2 files changed, 81 insertions(+) diff --git a/pom.xml b/pom.xml index e5da472cd1257..b51b7f27d4e3f 100644 --- a/pom.xml +++ b/pom.xml @@ -196,6 +196,63 @@ + + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + org.apache.avro + avro-maven-plugin + [1.7.7,) + + schema + + + + + false + + + + + + org.apache.maven.plugins + maven-jar-plugin + [2.5,) + + jar + test-jar + + + + + + + + + org.jacoco + jacoco-maven-plugin + [0.7.1,) + + report + prepare-agent + + + + + + + + + + diff --git a/sdk/pom.xml b/sdk/pom.xml index a1babd81f7fee..5cd5c0ab046ba 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -182,6 +182,30 @@ + + + + org.codehaus.mojo + build-helper-maven-plugin + 1.9.1 + + + add-test-source + generate-test-sources + + add-test-source + + + + ${project.build.directory}/generated-test-sources/java + + + + + From d6d4a343e164e8a31bee3bd76ff995550a46c630 Mon Sep 17 00:00:00 2001 From: bchambers Date: Fri, 13 Mar 2015 14:35:42 -0700 Subject: [PATCH 0269/1541] Pull some helpful functions for extracting and formatting Classes and Methods. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88580191 --- .../sdk/options/PipelineOptionsFactory.java | 32 ++------ .../sdk/util/common/ReflectHelpers.java | 72 ++++++++++++++++++ .../options/PipelineOptionsFactoryTest.java | 2 +- .../sdk/util/common/ReflectHelpersTest.java | 76 +++++++++++++++++++ 4 files changed, 154 insertions(+), 28 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ReflectHelpers.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/ReflectHelpersTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java index 633e8095ca797..1e765470c4b46 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java @@ -19,6 +19,7 @@ import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar; import com.google.cloud.dataflow.sdk.runners.worker.DataflowWorkerHarness; +import com.google.cloud.dataflow.sdk.util.common.ReflectHelpers; import com.google.common.base.Equivalence; import com.google.common.base.Function; import com.google.common.base.Preconditions; @@ -728,10 +729,10 @@ private static void validateClass(Class iface, Iterable getterClassNames = FluentIterable.from(getters) .transform(MethodToDeclaringClassFunction.INSTANCE) - .transform(ClassNameFunction.INSTANCE); + .transform(ReflectHelpers.CLASS_NAME); Iterable gettersWithJsonIgnoreClassNames = FluentIterable.from(gettersWithJsonIgnore) .transform(MethodToDeclaringClassFunction.INSTANCE) - .transform(ClassNameFunction.INSTANCE); + .transform(ReflectHelpers.CLASS_NAME); Preconditions.checkArgument(gettersWithJsonIgnore.isEmpty() || getters.size() == gettersWithJsonIgnore.size(), @@ -746,7 +747,7 @@ private static void validateClass(Class iface, Iterable settersWithJsonIgnoreClassNames = FluentIterable.from(settersWithJsonIgnore) .transform(MethodToDeclaringClassFunction.INSTANCE) - .transform(ClassNameFunction.INSTANCE); + .transform(ReflectHelpers.CLASS_NAME); Preconditions.checkArgument(settersWithJsonIgnore.isEmpty(), "Expected setter for property [%s] to not be marked with @JsonIgnore on %s", @@ -777,7 +778,7 @@ private static void validateClass(Class iface, Set unknownMethods = Sets.difference(Sets.newHashSet(klass.getMethods()), methods); Preconditions.checkArgument(unknownMethods.isEmpty(), "Methods %s on [%s] do not conform to being bean properties.", - FluentIterable.from(unknownMethods).transform(MethodFormatterFunction.INSTANCE), + FluentIterable.from(unknownMethods).transform(ReflectHelpers.METHOD_FORMATTER), iface.getName()); } @@ -808,29 +809,6 @@ public Class apply(Method input) { } } - /** A {@link Function} which turns a method into a simple method signature. */ - private static class MethodFormatterFunction implements Function { - static final MethodFormatterFunction INSTANCE = new MethodFormatterFunction(); - @Override - public String apply(Method input) { - String parameterTypes = FluentIterable.of(input.getParameterTypes()) - .transform(ClassNameFunction.INSTANCE) - .toSortedList(String.CASE_INSENSITIVE_ORDER) - .toString(); - return ClassNameFunction.INSTANCE.apply(input.getReturnType()) + " " + input.getName() - + "(" + parameterTypes.substring(1, parameterTypes.length() - 1) + ")"; - } - } - - /** A {@link Function} with returns the classes name. */ - private static class ClassNameFunction implements Function, String> { - static final ClassNameFunction INSTANCE = new ClassNameFunction(); - @Override - public String apply(Class input) { - return input.getName(); - } - } - /** A {@link Function} with returns the declaring class for the method. */ private static class MethodToDeclaringClassFunction implements Function> { static final MethodToDeclaringClassFunction INSTANCE = new MethodToDeclaringClassFunction(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ReflectHelpers.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ReflectHelpers.java new file mode 100644 index 0000000000000..b2f5f17a43a9c --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/ReflectHelpers.java @@ -0,0 +1,72 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.collect.FluentIterable; + +import java.lang.reflect.Method; + +/** + * Utilities for working with with {@link Class Classes} and {@link Method Methods}. + */ +public class ReflectHelpers { + + private static final Joiner COMMA_SEPARATOR = Joiner.on(", "); + + /** A {@link Function} which turns a method into a simple method signature. */ + public static final Function METHOD_FORMATTER = new Function() { + @Override + public String apply(Method input) { + String parameterTypes = FluentIterable.of(input.getParameterTypes()) + .transform(CLASS_SIMPLE_NAME) + .join(COMMA_SEPARATOR); + return String.format("%s(%s)", + input.getName(), + parameterTypes); + } + }; + + /** A {@link Function} which turns a method into the declaring class + method signature. */ + public static final Function CLASS_AND_METHOD_FORMATTER = + new Function() { + @Override + public String apply(Method input) { + return String.format("%s#%s", + CLASS_NAME.apply(input.getDeclaringClass()), + METHOD_FORMATTER.apply(input)); + } + }; + + /** A {@link Function} with returns the classes name. */ + public static final Function, String> CLASS_NAME = + new Function, String>(){ + @Override + public String apply(Class input) { + return input.getName(); + } + }; + + /** A {@link Function} with returns the classes name. */ + public static final Function, String> CLASS_SIMPLE_NAME = + new Function, String>(){ + @Override + public String apply(Class input) { + return input.getSimpleName(); + } + }; +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java index a7ff30c12fe19..0068713ef24f1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java @@ -167,7 +167,7 @@ public static interface ExtraneousMethod extends PipelineOptions { public void testHavingExtraneousMethodThrows() throws Exception { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage( - "Methods [java.lang.String extraneousMethod(int, java.lang.String)] on " + "Methods [extraneousMethod(int, String)] on " + "[com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$ExtraneousMethod] " + "do not conform to being bean properties."); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/ReflectHelpersTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/ReflectHelpersTest.java new file mode 100644 index 0000000000000..db4b8179af5ec --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/ReflectHelpersTest.java @@ -0,0 +1,76 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util.common; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.List; + +/** + * Tests for {@link ReflectHelpers}. + */ +@RunWith(JUnit4.class) +public class ReflectHelpersTest { + + @Test + public void testClassName() { + assertEquals(getClass().getName(), ReflectHelpers.CLASS_NAME.apply(getClass())); + } + + @Test + public void testClassSimpleName() { + assertEquals(getClass().getSimpleName(), + ReflectHelpers.CLASS_SIMPLE_NAME.apply(getClass())); + } + + @Test + public void testMethodFormatter() throws Exception { + assertEquals("testMethodFormatter()", + ReflectHelpers.METHOD_FORMATTER.apply(getClass().getMethod("testMethodFormatter"))); + + assertEquals("oneArg(int)", + ReflectHelpers.METHOD_FORMATTER.apply(getClass().getDeclaredMethod("oneArg", int.class))); + assertEquals("twoArg(String, List)", + ReflectHelpers.METHOD_FORMATTER.apply( + getClass().getDeclaredMethod("twoArg", String.class, List.class))); + } + + @Test + public void testClassMethodFormatter() throws Exception { + assertEquals( + getClass().getName() + "#testMethodFormatter()", + ReflectHelpers.CLASS_AND_METHOD_FORMATTER + .apply(getClass().getMethod("testMethodFormatter"))); + + assertEquals( + getClass().getName() + "#oneArg(int)", + ReflectHelpers.CLASS_AND_METHOD_FORMATTER + .apply(getClass().getDeclaredMethod("oneArg", int.class))); + assertEquals( + getClass().getName() + "#twoArg(String, List)", + ReflectHelpers.CLASS_AND_METHOD_FORMATTER.apply( + getClass().getDeclaredMethod("twoArg", String.class, List.class))); + } + + @SuppressWarnings("unused") + void oneArg(int n) {} + @SuppressWarnings("unused") + void twoArg(String foo, List bar) {} +} From 4e0ecff4da9f6f49bd24eb05e9a7f975ada2e882 Mon Sep 17 00:00:00 2001 From: peihe Date: Fri, 13 Mar 2015 14:56:12 -0700 Subject: [PATCH 0270/1541] FIX: windmill uses kint64max usec as the timer max timestamp, keep the Java side in sync. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88582344 --- .../cloud/dataflow/sdk/runners/worker/ShuffleSink.java | 3 ++- .../dataflow/sdk/transforms/windowing/BoundedWindow.java | 9 +++++++++ .../dataflow/sdk/transforms/windowing/GlobalWindow.java | 2 +- .../com/google/cloud/dataflow/sdk/util/DoFnContext.java | 2 +- .../google/cloud/dataflow/sdk/util/WindowedValue.java | 4 ++-- .../sdk/runners/worker/GroupingShuffleReaderTest.java | 3 ++- 6 files changed, 17 insertions(+), 6 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSink.java index 351f4ae811f3e..12486766bc612 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ShuffleSink.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.coders.InstantCoder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowedValue.WindowedValueCoder; @@ -194,7 +195,7 @@ public long add(WindowedValue windowedElem) throws IOException { } else if (groupValues) { // Sort values by timestamp so that GroupAlsoByWindows can run efficiently. - if (windowedElem.getTimestamp().getMillis() == Long.MIN_VALUE) { + if (windowedElem.getTimestamp().equals(BoundedWindow.TIMESTAMP_MIN_VALUE)) { // Empty secondary keys sort before all other secondary keys, so we // can omit this common value here for efficiency. secondaryKeyBytes = null; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/BoundedWindow.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/BoundedWindow.java index 42f039ee33dd8..6547efbc9ff61 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/BoundedWindow.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/BoundedWindow.java @@ -18,6 +18,8 @@ import org.joda.time.Instant; +import java.util.concurrent.TimeUnit; + /** * A {@code BoundedWindow} represents a finite grouping of elements, with an * upper bound (larger timestamps represent more recent data) on the timestamps @@ -30,6 +32,13 @@ * be treated as equal by {@code equals()} and {@code hashCode()}. */ public abstract class BoundedWindow { + // The min and max timestmaps that won't overflow when they are converted to + // usec. + public static final Instant TIMESTAMP_MIN_VALUE = + new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MIN_VALUE)); + public static final Instant TIMESTAMP_MAX_VALUE = + new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE)); + /** * Returns the upper bound of timestamps for values in this window. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java index 7d788e7e63676..6cbb66beb5a4e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/GlobalWindow.java @@ -32,7 +32,7 @@ public class GlobalWindow extends BoundedWindow { @Override public Instant maxTimestamp() { - return new Instant(Long.MAX_VALUE); + return TIMESTAMP_MAX_VALUE; } private GlobalWindow() {} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java index 451fe76f4dad1..4a6fd1f2a9bb5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnContext.java @@ -123,7 +123,7 @@ WindowedValue makeWindowedValue( final Instant inputTimestamp = timestamp; if (timestamp == null) { - timestamp = new Instant(Long.MIN_VALUE); + timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; } if (windows == null) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java index 2147ef4867364..57bad7db8c648 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowedValue.java @@ -71,7 +71,7 @@ public static WindowedValue of( */ public static WindowedValue valueInGlobalWindow(V value) { return new WindowedValue<>(value, - new Instant(Long.MIN_VALUE), + BoundedWindow.TIMESTAMP_MIN_VALUE, Arrays.asList(GlobalWindow.INSTANCE)); } @@ -80,7 +80,7 @@ public static WindowedValue valueInGlobalWindow(V value) { */ public static WindowedValue valueInEmptyWindows(V value) { return new WindowedValue(value, - new Instant(Long.MIN_VALUE), + BoundedWindow.TIMESTAMP_MIN_VALUE, Collections.emptyList()); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java index c20626e70e59c..522f318649005 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java @@ -37,6 +37,7 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CoderUtils; @@ -150,7 +151,7 @@ private void runTestReadFromShuffle( WindowedValue>> windowedValue = iter.next(); // Verify value is in an empty windows. - assertEquals(Long.MIN_VALUE, windowedValue.getTimestamp().getMillis()); + assertEquals(BoundedWindow.TIMESTAMP_MIN_VALUE, windowedValue.getTimestamp()); assertEquals(0, windowedValue.getWindows().size()); KV> elem = windowedValue.getValue(); From b20519f77f739e6319502410440fc52f507b03ce Mon Sep 17 00:00:00 2001 From: davor Date: Fri, 13 Mar 2015 15:03:10 -0700 Subject: [PATCH 0271/1541] Rename Eclipse launch files to make it compatible with certain tools. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88582893 --- eclipse/starter/.classpath | 20 ++++++ eclipse/starter/.project | 23 +++++++ eclipse/starter/.settings/LOCAL.launch | 15 +++++ eclipse/starter/.settings/SERVICE.launch | 16 +++++ eclipse/starter/pom.xml | 17 +++++ .../dataflow/starter/StarterPipeline.java | 66 +++++++++++++++++++ 6 files changed, 157 insertions(+) create mode 100644 eclipse/starter/.classpath create mode 100644 eclipse/starter/.project create mode 100644 eclipse/starter/.settings/LOCAL.launch create mode 100644 eclipse/starter/.settings/SERVICE.launch create mode 100644 eclipse/starter/pom.xml create mode 100644 eclipse/starter/src/main/java/com/google/cloud/dataflow/starter/StarterPipeline.java diff --git a/eclipse/starter/.classpath b/eclipse/starter/.classpath new file mode 100644 index 0000000000000..9f9ff59805559 --- /dev/null +++ b/eclipse/starter/.classpath @@ -0,0 +1,20 @@ + + + + + + + + + + + + + + + + + + + + diff --git a/eclipse/starter/.project b/eclipse/starter/.project new file mode 100644 index 0000000000000..64c4186fcb661 --- /dev/null +++ b/eclipse/starter/.project @@ -0,0 +1,23 @@ + + + google-cloud-dataflow-starter + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/eclipse/starter/.settings/LOCAL.launch b/eclipse/starter/.settings/LOCAL.launch new file mode 100644 index 0000000000000..74145d0378c9e --- /dev/null +++ b/eclipse/starter/.settings/LOCAL.launch @@ -0,0 +1,15 @@ + + + + + + + + + + + + + + + diff --git a/eclipse/starter/.settings/SERVICE.launch b/eclipse/starter/.settings/SERVICE.launch new file mode 100644 index 0000000000000..4dce6b7bb0333 --- /dev/null +++ b/eclipse/starter/.settings/SERVICE.launch @@ -0,0 +1,16 @@ + + + + + + + + + + + + + + + + diff --git a/eclipse/starter/pom.xml b/eclipse/starter/pom.xml new file mode 100644 index 0000000000000..1f5553e74576c --- /dev/null +++ b/eclipse/starter/pom.xml @@ -0,0 +1,17 @@ + + 4.0.0 + + com.google.cloud.dataflow + google-cloud-dataflow-starter + 0.0.1-SNAPSHOT + + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-all + LATEST + + + diff --git a/eclipse/starter/src/main/java/com/google/cloud/dataflow/starter/StarterPipeline.java b/eclipse/starter/src/main/java/com/google/cloud/dataflow/starter/StarterPipeline.java new file mode 100644 index 0000000000000..db2ed6ae675a4 --- /dev/null +++ b/eclipse/starter/src/main/java/com/google/cloud/dataflow/starter/StarterPipeline.java @@ -0,0 +1,66 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.starter; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; + +/** + * A starter example for writing Google Cloud Dataflow programs. + * + *

    The example takes two strings, converts them to their upper-case + * representation and prints them on the console. + * + *

    To run this starter example locally using DirectPipelineRunner, just + * execute it without any additional parameters from your favorite development + * environment. In Eclipse, this corresponds to the existing 'LOCAL' run + * configuration. + * + *

    To run this starter example using managed resource in Google Cloud + * Platform, you should specify the following command-line options: + * --project= + * --stagingLocation= + * --runner=BlockingDataflowPipelineRunner + * In Eclipse, you can just modify the existing 'SERVICE' run configuration. + */ +@SuppressWarnings("serial") +public class StarterPipeline { + + public static void main(String[] args) { + Pipeline p = Pipeline.create( + PipelineOptionsFactory.fromArgs(args).withValidation().create()); + + p.apply(Create.of("Hello", "World")) + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().toUpperCase()); + } + })) + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + System.out.println(c.element()); + } + })); + + p.run(); + } +} From 11597b46ca605071fb41ec70d0eefd87d415d14a Mon Sep 17 00:00:00 2001 From: Davor Bonaci Date: Fri, 13 Mar 2015 16:43:39 -0700 Subject: [PATCH 0272/1541] Add serialversionuid member to several classes that are serializable in Java 8. --- .../com/google/cloud/dataflow/sdk/io/ByteOffsetBasedSource.java | 1 + .../java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java | 1 + 2 files changed, 2 insertions(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ByteOffsetBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ByteOffsetBasedSource.java index 4d48d210a5464..cde606063293e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ByteOffsetBasedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/ByteOffsetBasedSource.java @@ -45,6 +45,7 @@ * @param Type of records represented by the source. */ public abstract class ByteOffsetBasedSource extends Source { + private static final long serialVersionUID = 0; private final long startOffset; private final long endOffset; private final long minShardSize; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java index 865d923713de5..3b89fe7e39c35 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java @@ -54,6 +54,7 @@ * @param Type of records represented by the source. */ public abstract class FileBasedSource extends ByteOffsetBasedSource { + private static final long serialVersionUID = 0; private static final Logger LOG = LoggerFactory.getLogger(FileBasedSource.class); private final String fileOrPatternSpec; From 1d352f9fc0e3083aa5b3ff03682777c9ac33f0f3 Mon Sep 17 00:00:00 2001 From: Chikanaga Tomoyuki Date: Mon, 16 Mar 2015 16:07:57 +0900 Subject: [PATCH 0273/1541] Fix javadoc markups and descriptions according to WordCount. --- .../dataflow/examples/WindowingWordCount.java | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/WindowingWordCount.java b/examples/src/main/java/com/google/cloud/dataflow/examples/WindowingWordCount.java index 72e005b2c50bc..ef2eac97836b8 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/WindowingWordCount.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/WindowingWordCount.java @@ -41,18 +41,19 @@ * example see: * https://cloud.google.com/dataflow/java-sdk/wordcount-example * - * To execute this pipeline locally, specify general pipeline configuration: + *

    To execute this pipeline locally, specify general pipeline configuration: * --project= - * and example configuration: - * --output=[ | gs://] + * and a local output file or output prefix on GCS: + * --output=[ | gs://] * - * To execute this pipeline using the Dataflow service, specify pipeline configuration: - * --project= --stagingLocation=gs:// + *

    To execute this pipeline using the Dataflow service, specify pipeline configuration: + * --project= + * --stagingLocation=gs:// * --runner=BlockingDataflowPipelineRunner - * and example configuration: - * --output=gs:// + * and an output prefix on GCS: + * --output=gs:// * - * The input file defaults to gs://dataflow-samples/shakespeare/kinglear.txt and can be + *

    The input file defaults to gs://dataflow-samples/shakespeare/kinglear.txt and can be * overridden with --input. */ public class WindowingWordCount { From 6adbd754670717da80f61c69cb51c268cc23451a Mon Sep 17 00:00:00 2001 From: Chikanaga Tomoyuki Date: Mon, 16 Mar 2015 16:53:27 +0900 Subject: [PATCH 0274/1541] fix a typo. setNumShard -> setNumShards. --- .../com/google/cloud/dataflow/examples/WindowingWordCount.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/WindowingWordCount.java b/examples/src/main/java/com/google/cloud/dataflow/examples/WindowingWordCount.java index ef2eac97836b8..578bc96847202 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/WindowingWordCount.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/WindowingWordCount.java @@ -136,7 +136,7 @@ private interface Options extends PipelineOptions { @Description("Number of output shards (0 if the system should choose automatically)") @Default.Integer(0) int getNumShards(); - void setNumShard(int value); + void setNumShards(int value); } private static String getOutputLocation(Options options) { From d625f34681f978c858593ed97d0fc2f641c03418 Mon Sep 17 00:00:00 2001 From: lcwik Date: Fri, 13 Mar 2015 15:25:59 -0700 Subject: [PATCH 0275/1541] Add support for printing help via the PipelineOptionsFactory either by programmatic calling printHelp or passing in --help as an argument. ----Release Notes---- [] ------------- Created by MOE: http://code.google.com/p/moe-java MOE_MIGRATED_REVID=88584969 --- .../sdk/options/ApplicationNameOptions.java | 8 +- .../dataflow/sdk/options/BigQueryOptions.java | 6 +- .../BlockingDataflowPipelineOptions.java | 3 + .../sdk/options/CloudDebuggerOptions.java | 6 +- .../options/DataflowPipelineDebugOptions.java | 84 ++++- .../sdk/options/DataflowPipelineOptions.java | 53 ++- .../DataflowPipelineWorkerPoolOptions.java | 53 ++- .../options/DataflowWorkerHarnessOptions.java | 9 +- .../options/DataflowWorkerLoggingOptions.java | 15 +- .../dataflow/sdk/options/Description.java | 8 +- .../dataflow/sdk/options/GcpOptions.java | 75 +++-- .../dataflow/sdk/options/GcsOptions.java | 20 +- .../sdk/options/GoogleApiDebugOptions.java | 10 +- .../cloud/dataflow/sdk/options/Hidden.java | 31 ++ .../dataflow/sdk/options/PipelineOptions.java | 4 +- .../sdk/options/PipelineOptionsFactory.java | 307 ++++++++++++++++-- .../sdk/options/StreamingOptions.java | 4 +- .../options/PipelineOptionsFactoryTest.java | 156 +++++++++ 18 files changed, 723 insertions(+), 129 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Hidden.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ApplicationNameOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ApplicationNameOptions.java index edbc57af3c2b3..d6d7db8cdf9fb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ApplicationNameOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ApplicationNameOptions.java @@ -23,11 +23,11 @@ public interface ApplicationNameOptions extends PipelineOptions { /** * Name of application, for display purposes. *

    - * Defaults to the name of the class which constructs the - * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner}. + * Defaults to the name of the class which constructs the {@link PipelineOptions} + * via the {@link PipelineOptionsFactory}. */ - @Description("Application name. Defaults to the name of the class which " - + "constructs the Pipeline.") + @Description("Name of application for display purposes. Defaults to the name of the class which " + + "constructs the PipelineOptions via the PipelineOptionsFactory.") String getAppName(); void setAppName(String value); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BigQueryOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BigQueryOptions.java index 076ec5cc41d0d..d240d1c4b4076 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BigQueryOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BigQueryOptions.java @@ -19,10 +19,12 @@ /** * Properties needed when using BigQuery with the Dataflow SDK. */ +@Description("Options which are used to configure BigQuery. See " + + "https://cloud.google.com/bigquery/what-is-bigquery for details on BigQuery.") public interface BigQueryOptions extends ApplicationNameOptions, GcpOptions, PipelineOptions, StreamingOptions { - @Description("Temporary staging dataset ID for BigQuery " - + "table operations") + @Description("Temporary dataset for BigQuery table operations. " + + "Supported values are \"bigquery.googleapis.com/{dataset}\"") @Default.String("bigquery.googleapis.com/cloud_dataflow") String getTempDatasetId(); void setTempDatasetId(String value); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BlockingDataflowPipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BlockingDataflowPipelineOptions.java index ca30ea2c20e9c..d65f38c6c6b58 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BlockingDataflowPipelineOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/BlockingDataflowPipelineOptions.java @@ -25,11 +25,14 @@ /** * Options which are used to configure the {@link BlockingDataflowPipelineRunner}. */ +@Description("Configure options on the BlockingDataflowPipelineRunner.") public interface BlockingDataflowPipelineOptions extends DataflowPipelineOptions { /** * Output stream for job status messages. */ + @Description("Where messages generated during execution of the Dataflow job will be output.") @JsonIgnore + @Hidden @Default.InstanceFactory(StandardOutputFactory.class) PrintStream getJobMessageOutput(); void setJobMessageOutput(PrintStream value); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/CloudDebuggerOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/CloudDebuggerOptions.java index 3fc2b6a5ff271..84710df258451 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/CloudDebuggerOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/CloudDebuggerOptions.java @@ -22,13 +22,17 @@ /** * Options for controlling Cloud Debugger. These options are experimental and subject to change. */ +@Description("[Experimental] Used to configure the Cloud Debugger") public interface CloudDebuggerOptions { /** * User defined application version. Cloud Debugger uses it to group all - * running debuggee processes. Version should be different if users have + * running debugged processes. Version should be different if users have * multiple parallel runs of the same application with different inputs. */ + @Description("User defined application version. Cloud Debugger uses it to group all " + + "running debugged processes. cdbgVersion should be different if users have " + + "multiple parallel runs of the same application with different inputs.") String getCdbgVersion(); void setCdbgVersion(String value); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java index a5bc999be534e..937af6cf40bf7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineDebugOptions.java @@ -16,19 +16,25 @@ package com.google.cloud.dataflow.sdk.options; +import com.google.api.services.dataflow.Dataflow; import com.google.cloud.dataflow.sdk.util.DataflowPathValidator; import com.google.cloud.dataflow.sdk.util.GcsStager; import com.google.cloud.dataflow.sdk.util.InstanceBuilder; import com.google.cloud.dataflow.sdk.util.PathValidator; import com.google.cloud.dataflow.sdk.util.Stager; +import com.google.cloud.dataflow.sdk.util.Transport; import com.fasterxml.jackson.annotation.JsonIgnore; import java.util.List; /** - * Options used for testing and debugging the Dataflow SDK. + * Internal. Options used to control execution of the Dataflow SDK for + * debugging and testing purposes. */ +@Description("[Internal] Options used to control execution of the Dataflow SDK for " + + "debugging and testing purposes.") +@Hidden public interface DataflowPipelineDebugOptions extends PipelineOptions { /** * Dataflow endpoint to use. @@ -40,7 +46,8 @@ public interface DataflowPipelineDebugOptions extends PipelineOptions { * otherwise {@link #getApiRootUrl()} is used as the root * url. */ - @Description("Cloud Dataflow Endpoint") + @Description("The URL for the Dataflow API. If the string contains \"://\"" + + " will be treated as the entire URL, otherwise will be treated relative to apiRootUrl.") @Default.String("v1b3/projects/") String getDataflowEndpoint(); void setDataflowEndpoint(String value); @@ -51,62 +58,105 @@ public interface DataflowPipelineDebugOptions extends PipelineOptions { *

    Dataflow provides a number of experimental features that can be enabled * with this flag. * - *

    Please sync with the Dataflow team when enabling any experiments. + *

    Please sync with the Dataflow team before enabling any experiments. */ - @Description("Backend experiments to enable.") + @Description("[Experimental] Dataflow provides a number of experimental features that can " + + "be enabled with this flag. Please sync with the Dataflow team before enabling any " + + "experiments.") List getExperiments(); void setExperiments(List value); /** - * The API endpoint to use when communicating with the Dataflow service. + * The endpoint to use with the Dataflow API. dataflowEndpoint can override this value + * if it contains an absolute URL, otherwise apiRootUrl will be combined with dataflowEndpoint + * to generate the full URL to communicate with the Dataflow API. */ - @Description("Google Cloud root API") + @Description("The endpoint to use with the Dataflow API. dataflowEndpoint can override this " + + "value if it contains an absolute URL, otherwise apiRootUrl will be combined with " + + "dataflowEndpoint to generate the full URL to communicate with the Dataflow API.") @Default.String("https://dataflow.googleapis.com/") String getApiRootUrl(); void setApiRootUrl(String value); /** - * The path to write the translated Dataflow specification out to - * at job submission time. + * The path to write the translated Dataflow job specification out to + * at job submission time. The Dataflow job specification will be represented in JSON + * format. */ - @Description("File for writing dataflow job descriptions") + @Description("The path to write the translated Dataflow job specification out to " + + "at job submission time. The Dataflow job specification will be represented in JSON " + + "format.") String getDataflowJobFile(); void setDataflowJobFile(String value); /** - * The name of the validator class used to validate path names. + * The class of the validator that should be created and used to validate paths. + * If pathValidator has not been set explicitly, an instance of this class will be + * constructed and used as the path validator. */ - @Description("The validator class used to validate path names.") + @Description("The class of the validator that should be created and used to validate paths. " + + "If pathValidator has not been set explicitly, an instance of this class will be " + + "constructed and used as the path validator.") @Default.Class(DataflowPathValidator.class) Class getPathValidatorClass(); void setPathValidatorClass(Class validatorClass); /** - * The validator class used to validate path names. + * The path validator instance that should be created and used to validate paths. + * If no path validator has been set explicitly, the default is to use the instance factory which + * constructs a path validator based upon the currently set pathValidatorClass. */ @JsonIgnore - @Description("The validator class used to validate path names.") + @Description("The path validator instance that should be created and used to validate paths. " + + "If no path validator has been set explicitly, the default is to use the instance factory " + + "which constructs a path validator based upon the currently set pathValidatorClass.") @Default.InstanceFactory(PathValidatorFactory.class) PathValidator getPathValidator(); void setPathValidator(PathValidator validator); /** - * The class used to stage files. + * The class responsible for staging resources to be accessible by workers + * during job execution. */ - @Description("The class used to stage files.") + @Description("The class of the stager that should be created and used to stage resources. " + + "If stager has not been set explicitly, an instance of this class will be " + + "constructed and used as the resource stager.") @Default.Class(GcsStager.class) Class getStagerClass(); void setStagerClass(Class stagerClass); /** - * The stager instance used to stage files. + * The resource stager instance that should be created and used to stage resources. + * If no stager has been set explicitly, the default is to use the instance factory + * which constructs a resource stager based upon the currently set stagerClass. */ @JsonIgnore - @Description("The class use to stage packages.") + @Description("The resource stager instance that should be created and used to stage resources. " + + "If no stager has been set explicitly, the default is to use the instance factory " + + "which constructs a resource stager based upon the currently set stagerClass.") @Default.InstanceFactory(StagerFactory.class) Stager getStager(); void setStager(Stager stager); + /** + * An instance of the Dataflow client. Defaults to creating a Dataflow client + * using the current set of options. + */ + @JsonIgnore + @Description("An instance of the Dataflow client. Defaults to creating a Dataflow client " + + "using the current set of options.") + @Default.InstanceFactory(DataflowClientFactory.class) + Dataflow getDataflowClient(); + void setDataflowClient(Dataflow value); + + /** Returns the default Dataflow client built from the passed in PipelineOptions. */ + public static class DataflowClientFactory implements DefaultValueFactory { + @Override + public Dataflow create(PipelineOptions options) { + return Transport.newDataflowClient(options.as(DataflowPipelineOptions.class)).build(); + } + } + /** * Creates a {@link PathValidator} object using the class specified in * {@link #getPathValidatorClass()}. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java index 5dec12031b2f5..bfcc47c1a8912 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java @@ -16,13 +16,9 @@ package com.google.cloud.dataflow.sdk.options; -import com.google.api.services.dataflow.Dataflow; import com.google.cloud.dataflow.sdk.runners.DataflowPipeline; -import com.google.cloud.dataflow.sdk.util.Transport; import com.google.common.base.MoreObjects; -import com.fasterxml.jackson.annotation.JsonIgnore; - import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; @@ -31,13 +27,14 @@ /** * Options which can be used to configure the {@link DataflowPipeline}. */ +@Description("Options which configure the Dataflow pipeline.") public interface DataflowPipelineOptions extends PipelineOptions, GcpOptions, ApplicationNameOptions, DataflowPipelineDebugOptions, DataflowPipelineWorkerPoolOptions, BigQueryOptions, GcsOptions, StreamingOptions, CloudDebuggerOptions, DataflowWorkerLoggingOptions { /** - * GCS path for temporary files. + * GCS path for temporary files, e.g. gs://bucket/object *

    * Must be a valid Cloud Storage url, beginning with the prefix "gs://" *

    @@ -45,31 +42,37 @@ public interface DataflowPipelineOptions extends * {@link #getTempLocation()} is not set, then the Dataflow pipeline defaults to using * {@link #getStagingLocation()}. */ - @Description("GCS path for temporary files, eg \"gs://bucket/object\". " - + "Defaults to stagingLocation.") + @Description("GCS path for temporary files, eg \"gs://bucket/object\". " + + "Must be a valid Cloud Storage url, beginning with the prefix \"gs://\". " + + "At least one of tempLocation or stagingLocation must be set. If tempLocation is unset, " + + "defaults to using stagingLocation.") String getTempLocation(); void setTempLocation(String value); /** - * GCS path for staging local files. + * GCS path for staging local files, e.g. gs://bucket/object *

    - * If {@link #getStagingLocation()} is not set, then the Dataflow pipeline defaults to a staging - * directory within {@link #getTempLocation}. + * Must be a valid Cloud Storage url, beginning with the prefix "gs://" *

    - * At least one of {@link #getTempLocation()} or {@link #getStagingLocation()} must be set. + * At least one of {@link #getTempLocation()} or {@link #getStagingLocation()} must be set. If + * {@link #getTempLocation()} is not set, then the Dataflow pipeline defaults to using + * {@link #getStagingLocation()}. */ - @Description("GCS staging path. Defaults to a staging directory" - + " with the tempLocation") + @Description("GCS path for staging local files, e.g. \"gs://bucket/object\". " + + "Must be a valid Cloud Storage url, beginning with the prefix \"gs://\". " + + "At least one of stagingLocation or tempLocation must be set. If stagingLocation is unset, " + + "defaults to using tempLocation.") String getStagingLocation(); void setStagingLocation(String value); /** - * The job name is used as an idempotence key within the Dataflow service. If there - * is an existing job which is currently active, another job with the same name will - * not be able to be created. + * The Dataflow job name is used as an idempotence key within the Dataflow service. + * If there is an existing job which is currently active, another active job with the same + * name will not be able to be created. Defaults to using the ApplicationName-UserName-Date. */ - @Description("Dataflow job name, to uniquely identify active jobs. " - + "Defaults to using the ApplicationName-UserName-Date.") + @Description("The Dataflow job name is used as an idempotence key within the Dataflow service. " + + "If there is an existing job which is currently active, another active job with the same " + + "name will not be able to be created. Defaults to using the ApplicationName-UserName-Date.") @Default.InstanceFactory(JobNameFactory.class) String getJobName(); void setJobName(String value); @@ -101,18 +104,4 @@ public String create(PipelineOptions options) { return normalizedAppName + "-" + normalizedUserName + "-" + datePart; } } - - /** Alternative Dataflow client. */ - @JsonIgnore - @Default.InstanceFactory(DataflowClientFactory.class) - Dataflow getDataflowClient(); - void setDataflowClient(Dataflow value); - - /** Returns the default Dataflow client built from the passed in PipelineOptions. */ - public static class DataflowClientFactory implements DefaultValueFactory { - @Override - public Dataflow create(PipelineOptions options) { - return Transport.newDataflowClient(options.as(DataflowPipelineOptions.class)).build(); - } - } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java index 09571f1d8beec..9c06e6f3acb29 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java @@ -21,19 +21,23 @@ /** * Options which are used to configure the Dataflow pipeline worker pool. */ +@Description("Options which are used to configure the Dataflow pipeline worker pool.") public interface DataflowPipelineWorkerPoolOptions extends PipelineOptions { /** * Disk source image to use by VMs for jobs. * @see Compute Engine Images */ - @Description("Dataflow VM disk image.") + @Description("Disk source image to use by VMs for jobs. See " + + "https://developers.google.com/compute/docs/images for further details.") String getDiskSourceImage(); void setDiskSourceImage(String value); /** - * Number of workers to use in remote execution. + * Number of workers to use when executing the Dataflow job. */ - @Description("Number of workers, when using remote execution") + @Description("Number of workers to use when executing the Dataflow job. Note that " + + "selection of an autoscaling algorithm other then \"NONE\" will effect the " + + "size of the worker pool.") @Default.Integer(3) int getNumWorkers(); void setNumWorkers(int value); @@ -60,16 +64,19 @@ public String getAlgorithm() { } } - @Description("(experimental) The autoscaling algorithm to use for the workerpool.") + @Description("[Experimental] The autoscaling algorithm to use for the workerpool. " + + "NONE: does not change the size of the worker pool. " + + "BASIC: autoscale the worker pool size up to maxNumWorkers until the job completes.") @Default.Enum("NONE") AutoscalingAlgorithmType getAutoscalingAlgorithm(); void setAutoscalingAlgorithm(AutoscalingAlgorithmType value); /** - * Max number of workers to use when using workerpool autoscaling. + * The maximum number of workers to use when using workerpool autoscaling. * This option is experimental and subject to change. */ - @Description("Max number of workers to use, when using autoscaling") + @Description("[Experimental] The maximum number of workers to use when using workerpool " + + "autoscaling.") @Default.Integer(20) int getMaxNumWorkers(); void setMaxNumWorkers(int value); @@ -85,15 +92,15 @@ public String getAlgorithm() { * GCE availability zone for launching workers. * - *

    Default is up to the service. + *

    Default is up to the Dataflow service. */ @Description("GCE availability zone for launching workers. " - + "Default is up to the service") + + "Default is up to the Dataflow service.") String getZone(); void setZone(String value); /** - * Type of API for handling cluster management,i.e. resizing, healthchecking, etc. + * Type of API for handling cluster management, i.e. resizing, healthchecking, etc. */ public enum ClusterManagerApiType { COMPUTE_ENGINE("compute.googleapis.com"), @@ -110,15 +117,25 @@ public String getApiServiceName() { } } - @Description("Type of API for handling cluster management,i.e. resizing, healthchecking, etc.") + /** + * Type of API for handling cluster management, i.e. resizing, healthchecking, etc. + */ + @Description("Type of API for handling cluster management, i.e. resizing, healthchecking, etc.") @Default.Enum("COMPUTE_ENGINE") ClusterManagerApiType getClusterManagerApi(); void setClusterManagerApi(ClusterManagerApiType value); /** - * Machine type to create worker VMs as. + * Machine type to create Dataflow worker VMs as. + *

    + * See GCE machine types + * for a list of valid options. + *

    + * If unset, the Dataflow service will choose a reasonable default. */ - @Description("Dataflow VM machine type for workers.") + @Description("Machine type to create Dataflow worker VMs as. See " + + "https://cloud.google.com/compute/docs/machine-types for a list of valid options. " + + "If unset, the Dataflow service will choose a reasonable default.") String getWorkerMachineType(); void setWorkerMachineType(String value); @@ -141,23 +158,25 @@ public String getTeardownPolicyName() { } /** - * Teardown policy for the VMs. + * The teardown policy for the VMs. * *

    By default this is left unset and the service sets the default policy. */ - @Description("The teardown policy for the VMs.") + @Description("The teardown policy for the VMs. By default this is left unset " + + "and the service sets the default policy.") TeardownPolicy getTeardownPolicy(); void setTeardownPolicy(TeardownPolicy value); /** * List of local files to make available to workers. *

    - * Jars are placed on the worker's classpath. + * Files are placed on the worker's classpath. *

    * The default value is the list of jars from the main program's classpath. */ - @Description("Files to stage on GCS and make available to " - + "workers. The default value is all files from the classpath.") + @Description("Files to stage on GCS and make available to workers. " + + "Files are placed on the worker's classpath. " + + "The default value is all files from the classpath.") List getFilesToStage(); void setFilesToStage(List value); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerHarnessOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerHarnessOptions.java index adb45c8a7481b..6f69db26f96c1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerHarnessOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerHarnessOptions.java @@ -20,16 +20,21 @@ * Options which are used exclusively within the Dataflow worker harness. * These options have no effect at pipeline creation time. */ +@Description("[Internal] Options which are used exclusively within the Dataflow worker harness. " + + "These options have no effect at pipeline creation time.") +@Hidden public interface DataflowWorkerHarnessOptions extends DataflowPipelineOptions { /** - * ID of the worker running this pipeline. + * The identity of the worker running this pipeline. */ + @Description("The identity of the worker running this pipeline.") String getWorkerId(); void setWorkerId(String value); /** - * ID of the job this pipeline represents. + * The identity of the Dataflow job. */ + @Description("The identity of the Dataflow job.") String getJobId(); void setJobId(String value); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java index 3f59c8365495c..2ba61c6ce42b0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowWorkerLoggingOptions.java @@ -25,6 +25,7 @@ /** * Options which are used to control logging configuration on the Dataflow worker. */ +@Description("Options which are used to control logging configuration on the Dataflow worker.") public interface DataflowWorkerLoggingOptions extends PipelineOptions { /** * The set of log levels which can be used on the Dataflow worker. @@ -34,9 +35,9 @@ public enum Level { } /** - * This option controls the default log level of all loggers without a - * log level override. + * This option controls the default log level of all loggers without a log level override. */ + @Description("Controls the default log level of all loggers without a log level override.") @Default.Enum("INFO") Level getDefaultWorkerLogLevel(); void setDefaultWorkerLogLevel(Level level); @@ -51,6 +52,16 @@ public enum Level { * the expected format is {@code Name#Level}, further details on * {@link WorkerLogLevelOverride#create(String)}. */ + @Description("This option controls the log levels for specifically named loggers. " + + "The expected format is Name#Level. The Dataflow worker uses java.util.logging which " + + "supports a logging hierarchy based off of names which are \".\" separated. " + + "For example, by specifying the value \"a.b.c.Foo#DEBUG\", the logger for the class " + + "\"a.b.c.Foo\" will be configured to output logs at the DEBUG level. Similarly, " + + "by specifying the value \"a.b.c#WARN\", all loggers underneath the \"a.b.c\" package " + + "will be configured to output logs at the WARN level. Note that multiple overrides can " + + "be specified and that later values with equivalent names override earlier values. Also, " + + "note that when multiple overrides are specified, the exact name followed by the closest " + + "parent takes precedence.") WorkerLogLevelOverride[] getWorkerLogLevelOverrides(); void setWorkerLogLevelOverrides(WorkerLogLevelOverride... workerLogLevelOverrides); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Description.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Description.java index b02d3b9f13984..eaf7aac7d1fa6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Description.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Description.java @@ -22,9 +22,13 @@ import java.lang.annotation.Target; /** - * Machine-readable description for options in {@link PipelineOptions}. + * Descriptions are used to generate human readable output when the --help + * command is specified. Description annotations placed on interfaces which extend + * {@link PipelineOptions} to describe groups of related options. Description annotations + * placed on getter methods will be used to provide human readable information + * for the specific option. */ -@Target(value = ElementType.METHOD) +@Target({ElementType.METHOD, ElementType.TYPE}) @Retention(RetentionPolicy.RUNTIME) public @interface Description { String value(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java index a884d3ec7ebb4..7da5277e48777 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java @@ -58,23 +58,16 @@ * application default credentials falling back to gcloud. The other options can be * used by setting the corresponding properties. */ +@Description("Options used to configure Google Cloud Platform project and credentials.") public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions { /** * Project id to use when launching jobs. */ - @Description("Project id. Required when running a Dataflow in the cloud.") + @Description("Project id. Required when running a Dataflow in the cloud. " + + "See https://cloud.google.com/storage/docs/projects for further details.") String getProject(); void setProject(String value); - /** - * This option controls which file to use when attempting to create the credentials using the - * OAuth 2 webflow. - */ - @JsonIgnore - @Description("Path to a file containing Google API secret") - String getSecretsFile(); - void setSecretsFile(String value); - /** * This option controls which file to use when attempting to create the credentials using the * service account method. @@ -83,7 +76,9 @@ public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions { * {@link GcpOptions#getServiceAccountName() serviceAccountName}. */ @JsonIgnore - @Description("Path to a file containing the P12 service credentials") + @Description("Controls which file to use when attempting to create the credentials " + + "using the service account method. This option if specified, needs to be combined with " + + "the serviceAccountName option.") String getServiceAccountKeyfile(); void setServiceAccountKeyfile(String value); @@ -95,7 +90,9 @@ public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions { * {@link GcpOptions#getServiceAccountKeyfile() serviceAccountKeyfile}. */ @JsonIgnore - @Description("Name of the service account for Google APIs") + @Description("Controls which service account to use when attempting to create the credentials " + + "using the service account method. This option if specified, needs to be combined with " + + "the serviceAccountKeyfile option.") String getServiceAccountName(); void setServiceAccountName(String value); @@ -106,9 +103,33 @@ public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions { void setGCloudPath(String value); /** - * Directory for storing dataflow credentials. + * This option controls which file to use when attempting to create the credentials + * using the OAuth 2 webflow. After the OAuth2 webflow, the credentials will be stored + * within credentialDir. */ - @Description("Directory for storing dataflow credentials") + @JsonIgnore + @Description("This option controls which file to use when attempting to create the credentials " + + "using the OAuth 2 webflow. After the OAuth2 webflow, the credentials will be stored " + + "within credentialDir.") + String getSecretsFile(); + void setSecretsFile(String value); + + /** + * This option controls which credential store to use when creating the credentials + * using the OAuth 2 webflow. + */ + @Description("This option controls which credential store to use when creating the credentials " + + "using the OAuth 2 webflow.") + @Default.String("cloud_dataflow") + String getCredentialId(); + void setCredentialId(String value); + + /** + * Directory for storing dataflow credentials after execution of the OAuth 2 webflow. Defaults + * to using the $HOME/.store/data-flow directory. + */ + @Description("Directory for storing dataflow credentials after execution of the OAuth 2 webflow. " + + "Defaults to using the $HOME/.store/data-flow directory.") @Default.InstanceFactory(CredentialDirFactory.class) String getCredentialDir(); void setCredentialDir(String value); @@ -126,22 +147,30 @@ public String create(PipelineOptions options) { } } - @Description("The credential identifier when using a persistent" - + " credential store") - @Default.String("cloud_dataflow") - String getCredentialId(); - void setCredentialId(String value); - - @Description("The factory class used to create oauth credentials") + /** + * The class of the credential factory that should be created and used to create + * credentials. If gcpCredential has not been set explicitly, an instance of this class will + * be constructed and used as a credential factory. + */ + @Description("The class of the credential factory that should be created and used to create " + + "credentials. If gcpCredential has not been set explicitly, an instance of this class will " + + "be constructed and used as a credential factory. The default credential factory will") @Default.Class(GcpCredentialFactory.class) Class getCredentialFactoryClass(); void setCredentialFactoryClass( Class credentialFactoryClass); - /** Alternative Google Cloud Platform Credential. */ + /** + * The credential instance that should be used to authenticate against GCP services. + * If no credential has been set explicitly, the default is to use the instance factory + * which constructs a credential based upon the currently set credentialFactoryClass. + */ @JsonIgnore - @Description("Google Cloud Platform user credentials.") + @Description("The credential instance that should be used to authenticate against GCP services. " + + "If no credential has been set explicitly, the default is to use the instance factory " + + "which constructs a credential based upon the currently set credentialFactoryClass.") @Default.InstanceFactory(GcpUserCredentialsFactory.class) + @Hidden Credential getGcpCredential(); void setGcpCredential(Credential value); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java index 3136a8a42985e..9c52d5a3f97d7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcsOptions.java @@ -33,17 +33,29 @@ */ public interface GcsOptions extends ApplicationNameOptions, GcpOptions, PipelineOptions { - /** Alternative GcsUtil instance. */ + /** + * The GcsUtil instance that should be used to communicate with Google Cloud Storage. + */ @JsonIgnore + @Description("The GcsUtil instance that should be used to communicate with Google Cloud Storage.") @Default.InstanceFactory(GcsUtil.GcsUtilFactory.class) + @Hidden GcsUtil getGcsUtil(); void setGcsUtil(GcsUtil value); - //////////////////////////////////////////////////////////////////////////// - // Allows the user to provide an alternative ExecutorService if their - // environment does not support the default implementation. + /** + * The ExecutorService instance to use to create threads, can be overridden to specify an + * ExecutorService which is compatible with the users environment. If unset, the + * default is to create an ExecutorService with an unbounded number of threads which + * is compatible with Google AppEngine. + */ @JsonIgnore + @Description("The ExecutorService instance to use to create multiple threads. can be overridden " + + "to specify an ExecutorService which is compatible with the users environment. If unset, " + + "the default is to create an ExecutorService with an unbounded number of threads which " + + "is compatible with Google AppEngine.") @Default.InstanceFactory(ExecutorServiceFactory.class) + @Hidden ExecutorService getExecutorService(); void setExecutorService(ExecutorService value); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java index 4b3d69c87a9e8..3af9b000aebf0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GoogleApiDebugOptions.java @@ -29,7 +29,7 @@ import java.util.regex.Pattern; /** - * These options configure debug settings for Google API clients generated by the Dataflow SDK. + * These options configure debug settings for Google API clients created within the Dataflow SDK. */ public interface GoogleApiDebugOptions extends PipelineOptions { /** @@ -37,6 +37,14 @@ public interface GoogleApiDebugOptions extends PipelineOptions { * A tracing token must be requested from Google to be able to use this option. * An invalid tracing token will result in 400 errors from Google when the API is invoked. */ + @Description("This option enables tracing of API calls to Google services used within the " + + "Dataflow SDK. Values are expected in the format \"ApiName#TracingToken\" where the " + + "ApiName represents the request classes canonical name. The TracingToken must be requested " + + "from Google to be able to use this option. An invalid tracing token will result in HTTP " + + "400 errors from Google when the API is invoked. Note, that by enabling this option, the " + + "contents of the requests to and from Google Cloud services will be made available to " + + "Google. For example, by specifiying \"Dataflow#TracingToken\", all calls to the Dataflow " + + "service will be made available to Google.") GoogleApiTracer[] getGoogleApiTrace(); void setGoogleApiTrace(GoogleApiTracer... commands); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Hidden.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Hidden.java new file mode 100644 index 0000000000000..8cfa6b7ba2f0f --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/Hidden.java @@ -0,0 +1,31 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Methods and/or interfaces annotated with {@code @Hidden} will be suppressed from + * being output when {@code --help} is specified on the command-line. + */ +@Target({ElementType.METHOD, ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +public @interface Hidden { +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java index 1fb5d9aabb63c..9e75875ebfa0b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java @@ -56,7 +56,9 @@ public interface PipelineOptions { T as(Class kls); @Validation.Required - @Description("The runner which will be used when executing the pipeline.") + @Description("The pipeline runner which will be used to execute the pipeline. " + + "For registered runners, the class name can be specified, otherwise the fully" + + "qualified name needs to be specified.") @Default.Class(DirectPipelineRunner.class) Class> getRunner(); void setRunner(Class> kls); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java index 1e765470c4b46..e8afb3452aeed 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java @@ -22,12 +22,15 @@ import com.google.cloud.dataflow.sdk.util.common.ReflectHelpers; import com.google.common.base.Equivalence; import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Throwables; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Collections2; import com.google.common.collect.FluentIterable; import com.google.common.collect.HashMultimap; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableListMultimap; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -52,6 +55,8 @@ import java.beans.Introspector; import java.beans.PropertyDescriptor; import java.io.IOException; +import java.io.PrintStream; +import java.lang.annotation.Annotation; import java.lang.reflect.Method; import java.lang.reflect.Modifier; import java.lang.reflect.Proxy; @@ -62,11 +67,13 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; import java.util.Queue; import java.util.ServiceLoader; import java.util.Set; import java.util.SortedMap; import java.util.SortedSet; +import java.util.TreeSet; /** * Constructs a {@link PipelineOptions} or any derived interface which is composable to any other @@ -136,6 +143,15 @@ public static T as(Class klass) { * By default, strict parsing is enabled and arguments must conform to be either * {@code --booleanArgName} or {@code --argName=argValue}. Strict parsing can be disabled with * {@link Builder#withoutStrictParsing()}. + *

    + * Help information can be output to {@link System#out} by specifying {@code --help} as an + * argument. After help is printed, the application will exit. Specifying only {@code --help} + * will print out the list of + * {@link PipelineOptionsFactory#getRegisteredOptions() registered options} + * by invoking {@link PipelineOptionsFactory#printHelp(PrintStream)}. Specifying + * {@code --help=PipelineOptionsClassName} will print out detailed usage information about the + * specifically requested PipelineOptions by invoking + * {@link PipelineOptionsFactory#printHelp(PrintStream, Class)}. */ public static Builder fromArgs(String[] args) { return new Builder().fromArgs(args); @@ -193,6 +209,15 @@ private Builder(String[] args, boolean validation, * By default, strict parsing is enabled and arguments must conform to be either * {@code --booleanArgName} or {@code --argName=argValue}. Strict parsing can be disabled with * {@link Builder#withoutStrictParsing()}. + *

    + * Help information can be output to {@link System#out} by specifying {@code --help} as an + * argument. After help is printed, the application will exit. Specifying only {@code --help} + * will print out the list of + * {@link PipelineOptionsFactory#getRegisteredOptions() registered options} + * by invoking {@link PipelineOptionsFactory#printHelp(PrintStream)}. Specifying + * {@code --help=PipelineOptionsClassName} will print out detailed usage information about the + * specifically requested PipelineOptions by invoking + * {@link PipelineOptionsFactory#printHelp(PrintStream, Class)}. */ public Builder fromArgs(String[] args) { Preconditions.checkNotNull(args, "Arguments should not be null."); @@ -243,6 +268,7 @@ public T as(Class klass) { if (args != null) { ListMultimap options = parseCommandLine(args, strictParsing); LOG.debug("Provided Arguments: {}", options); + printHelpUsageAndExitIfNeeded(options, System.out, true /* exit */); initialOptions = parseObjects(klass, options, strictParsing); } @@ -263,6 +289,73 @@ public T as(Class klass) { } } + /** + * Determines whether the generic {@code --help} was requested or help was + * requested for a specific class and invokes the appropriate + * {@link PipelineOptionsFactory#printHelp(PrintStream)} and + * {@link PipelineOptionsFactory#printHelp(PrintStream, Class)} variant. + * Prints to the specified {@link PrintStream}, and exits if requested. + *

    + * Visible for testing. + * {@code printStream} and {@code exit} used for testing. + */ + @SuppressWarnings("unchecked") + static boolean printHelpUsageAndExitIfNeeded(ListMultimap options, + PrintStream printStream, boolean exit) { + if (options.containsKey("help")) { + final String helpOption = Iterables.getOnlyElement(options.get("help")); + + // Print the generic help if only --help was specified. + if (Boolean.TRUE.toString().equals(helpOption)) { + printHelp(printStream); + if (exit) { + System.exit(0); + } else { + return true; + } + } + + // Otherwise attempt to print the specific help option. + try { + Class klass = Class.forName(helpOption); + if (!PipelineOptions.class.isAssignableFrom(klass)) { + throw new ClassNotFoundException("PipelineOptions of type " + klass + " not found."); + } + printHelp(printStream, (Class) klass); + } catch (ClassNotFoundException e) { + // If we didn't find an exact match, look for any that match the class name. + Iterable> matches = Iterables.filter( + getRegisteredOptions(), + new Predicate>() { + @Override + public boolean apply(Class input) { + if (helpOption.contains(".")) { + return input.getName().endsWith(helpOption); + } else { + return input.getSimpleName().equals(helpOption); + } + } + }); + try { + printHelp(printStream, Iterables.getOnlyElement(matches)); + } catch (NoSuchElementException exception) { + printStream.format("Unable to find option %s.%n", helpOption); + printHelp(printStream); + } catch (IllegalArgumentException exception) { + printStream.format("Multiple matches found for %s: %s.%n", helpOption, + Iterables.transform(matches, ReflectHelpers.CLASS_NAME)); + printHelp(printStream); + } + } + if (exit) { + System.exit(0); + } else { + return true; + } + } + return false; + } + /** * Returns the simple name of the calling class using the current threads stack. */ @@ -341,6 +434,9 @@ Class getProxyClass() { private static final Map>, Registration> COMBINED_CACHE = Maps.newConcurrentMap(); + /** The width at which options should be output. */ + private static final int TERMINAL_WIDTH = 80; + static { try { IGNORED_METHODS = ImmutableSet.builder() @@ -479,6 +575,162 @@ public static Set> getRegisteredOptions() { return Collections.unmodifiableSet(REGISTERED_OPTIONS); } + /** + * Outputs the set of registered options with the PipelineOptionsFactory + * with a description for each one if available to the output stream. This output + * is pretty printed and meant to be human readable. This method will attempt to + * format its output to be compatible with a terminal window. + */ + public static void printHelp(PrintStream out) { + Preconditions.checkNotNull(out); + out.println("The set of registered options are:"); + Set> sortedOptions = + new TreeSet<>(ClassNameComparator.INSTANCE); + sortedOptions.addAll(REGISTERED_OPTIONS); + for (Class kls : sortedOptions) { + out.format(" %s%n", kls.getName()); + } + out.format("%nUse --help= for detailed help. For example:%n" + + " --help=DataflowPipelineOptions %n" + + " --help=com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions%n"); + } + + /** + * Outputs the set of options available to be set for the passed in {@link PipelineOptions} + * interface. The output is in a human readable format. The format is: + *

    +   * OptionGroup:
    +   *     ... option group description ...
    +   *
    +   *  --option1={@code } or list of valid enum choices
    +   *     Default: value (if available, see {@link Default})
    +   *     ... option description ... (if available, see {@link Description})
    +   *  --option2={@code } or list of valid enum choices
    +   *     Default: value (if available, see {@link Default})
    +   *     ... option description ... (if available, see {@link Description})
    +   * 
    + * This method will attempt to format its output to be compatible with a terminal window. + */ + public static void printHelp(PrintStream out, Class iface) { + Preconditions.checkNotNull(out); + Preconditions.checkNotNull(iface); + validateWellFormed(iface, REGISTERED_OPTIONS); + + Iterable methods = getClosureOfMethodsOnInterface(iface); + ListMultimap, Method> ifaceToMethods = ArrayListMultimap.create(); + for (Method method : methods) { + // Process only methods which are not marked as hidden. + if (method.getAnnotation(Hidden.class) == null) { + ifaceToMethods.put(method.getDeclaringClass(), method); + } + } + SortedSet> ifaces = new TreeSet<>(ClassNameComparator.INSTANCE); + // Keep interfaces which are not marked as hidden. + ifaces.addAll(Collections2.filter(ifaceToMethods.keySet(), new Predicate>() { + @Override + public boolean apply(Class input) { + return input.getAnnotation(Hidden.class) == null; + } + })); + for (Class currentIface : ifaces) { + Map propertyNamesToGetters = + getPropertyNamesToGetters(ifaceToMethods.get(currentIface)); + + // Don't output anything if there are no defined options + if (propertyNamesToGetters.isEmpty()) { + continue; + } + + out.format("%s:%n", currentIface.getName()); + prettyPrintDescription(out, currentIface.getAnnotation(Description.class)); + + out.println(); + + List lists = Lists.newArrayList(propertyNamesToGetters.keySet()); + Collections.sort(lists, String.CASE_INSENSITIVE_ORDER); + for (String propertyName : lists) { + Method method = propertyNamesToGetters.get(propertyName); + String printableType = method.getReturnType().getSimpleName(); + if (method.getReturnType().isEnum()) { + printableType = Joiner.on(" | ").join(method.getReturnType().getEnumConstants()); + } + out.format(" --%s=<%s>%n", propertyName, printableType); + Optional defaultValue = getDefaultValueFromAnnotation(method); + if (defaultValue.isPresent()) { + out.format(" Default: %s%n", defaultValue.get()); + } + prettyPrintDescription(out, method.getAnnotation(Description.class)); + } + out.println(); + } + } + + /** + * Outputs the value of the description, breaking up long lines on white space characters + * and attempting to honor a line limit of {@code TERMINAL_WIDTH}. + */ + private static void prettyPrintDescription(PrintStream out, Description description) { + final String spacing = " "; + if (description == null || description.value() == null) { + return; + } + + String[] words = description.value().split("\\s+"); + if (words.length == 0) { + return; + } + + out.print(spacing); + int lineLength = spacing.length(); + for (int i = 0; i < words.length; ++i) { + out.print(" "); + out.print(words[i]); + lineLength += 1 + words[i].length(); + + // If the next word takes us over the terminal width, then goto the next line. + if (i + 1 != words.length && words[i + 1].length() + lineLength + 1 > TERMINAL_WIDTH) { + out.println(); + out.print(spacing); + lineLength = spacing.length(); + } + } + out.println(); + } + + /** + * Returns a string representation of the {@link Default} value on the passed in method. + */ + private static Optional getDefaultValueFromAnnotation(Method method) { + for (Annotation annotation : method.getAnnotations()) { + if (annotation instanceof Default.Class) { + return Optional.of(((Default.Class) annotation).value().getSimpleName()); + } else if (annotation instanceof Default.String) { + return Optional.of(((Default.String) annotation).value()); + } else if (annotation instanceof Default.Boolean) { + return Optional.of(Boolean.toString(((Default.Boolean) annotation).value())); + } else if (annotation instanceof Default.Character) { + return Optional.of(Character.toString(((Default.Character) annotation).value())); + } else if (annotation instanceof Default.Byte) { + return Optional.of(Byte.toString(((Default.Byte) annotation).value())); + } else if (annotation instanceof Default.Short) { + return Optional.of(Short.toString(((Default.Short) annotation).value())); + } else if (annotation instanceof Default.Integer) { + return Optional.of(Integer.toString(((Default.Integer) annotation).value())); + } else if (annotation instanceof Default.Long) { + return Optional.of(Long.toString(((Default.Long) annotation).value())); + } else if (annotation instanceof Default.Float) { + return Optional.of(Float.toString(((Default.Float) annotation).value())); + } else if (annotation instanceof Default.Double) { + return Optional.of(Double.toString(((Default.Double) annotation).value())); + } else if (annotation instanceof Default.Enum) { + return Optional.of(((Default.Enum) annotation).value()); + } else if (annotation instanceof Default.InstanceFactory) { + return Optional.of(((Default.InstanceFactory) annotation).value().getSimpleName()); + } + } + return Optional.absent(); + } + static Map>> getRegisteredRunners() { return SUPPORTED_PIPELINE_RUNNERS; } @@ -582,7 +834,7 @@ public Iterable apply(Class input) { static Iterable getClosureOfMethodsOnInterface(Class iface) { Preconditions.checkNotNull(iface); Preconditions.checkArgument(iface.isInterface()); - ImmutableList.Builder builder = ImmutableList.builder(); + ImmutableSet.Builder builder = ImmutableSet.builder(); Queue> interfacesToProcess = Queues.newArrayDeque(); interfacesToProcess.add(iface); while (!interfacesToProcess.isEmpty()) { @@ -604,21 +856,7 @@ private static List getPropertyDescriptors(Class beanClas // The sorting is important to make this method stable. SortedSet methods = Sets.newTreeSet(MethodComparator.INSTANCE); methods.addAll(Arrays.asList(beanClass.getMethods())); - // Build a map of property names to getters. - SortedMap propertyNamesToGetters = Maps.newTreeMap(); - for (Method method : methods) { - String methodName = method.getName(); - if ((!methodName.startsWith("get") - && !methodName.startsWith("is")) - || method.getParameterTypes().length != 0 - || method.getReturnType() == void.class) { - continue; - } - String propertyName = Introspector.decapitalize( - methodName.startsWith("is") ? methodName.substring(2) : methodName.substring(3)); - propertyNamesToGetters.put(propertyName, method); - } - + SortedMap propertyNamesToGetters = getPropertyNamesToGetters(methods); List descriptors = Lists.newArrayList(); /* @@ -645,6 +883,28 @@ private static List getPropertyDescriptors(Class beanClas return descriptors; } + /** + * Returns a map of the property name to the getter method it represents. + * If there are duplicate methods with the same bean name, then it is indeterminate + * as to which method will be returned. + */ + private static SortedMap getPropertyNamesToGetters(Iterable methods) { + SortedMap propertyNamesToGetters = Maps.newTreeMap(); + for (Method method : methods) { + String methodName = method.getName(); + if ((!methodName.startsWith("get") + && !methodName.startsWith("is")) + || method.getParameterTypes().length != 0 + || method.getReturnType() == void.class) { + continue; + } + String propertyName = Introspector.decapitalize( + methodName.startsWith("is") ? methodName.substring(2) : methodName.substring(3)); + propertyNamesToGetters.put(propertyName, method); + } + return propertyNamesToGetters; + } + /** * Validates that a given class conforms to the following properties: *

%s)", PROJECT_ID_REGEXP, @@ -220,6 +221,7 @@ public class BigQueryIO { private static final Pattern TABLE_SPEC = Pattern.compile(DATASET_TABLE_REGEXP); + // TODO: make this private and remove improper access from BigQueryIOTranslator. public static final String SET_PROJECT_FROM_OPTIONS_WARNING = "No project specified for BigQuery table \"%1$s.%2$s\". Assuming it is in \"%3$s\". If the" + " table is in a different project please specify it as a part of the BigQuery table" @@ -237,7 +239,7 @@ public class BigQueryIO { /** * Parse a table specification in the form - * "[project_id]:[dataset_id].[table_id]" or "[dataset_id].[table_id]". + * {@code "[project_id]:[dataset_id].[table_id]"} or {@code "[dataset_id].[table_id]"}. * *

If the project id is omitted, the default project id is used. */ @@ -256,7 +258,7 @@ public static TableReference parseTableSpec(String tableSpec) { } /** - * Returns a canonical string representation of the TableReference. + * Returns a canonical string representation of the {@link TableReference}. */ public static String toTableSpec(TableReference ref) { StringBuilder sb = new StringBuilder(); @@ -273,11 +275,10 @@ public static String toTableSpec(TableReference ref) { * A {@link PTransform} that reads from a BigQuery table and returns a * {@link PCollection} of {@link TableRow TableRows} containing each of the rows of the table. * - *

Each TableRow record contains values indexed by column name. Here is a + *

Each {@link TableRow} contains values indexed by column name. Here is a * sample processing function that processes a "line" column from rows: - *


-   * static class ExtractWordsFn extends DoFn{@literal } {
-   *   {@literal @}Override
+   * 
{@code
+   * static class ExtractWordsFn extends DoFn {
    *   public void processElement(ProcessContext c) {
    *     // Get the "line" field of the TableRow object, split it into words, and emit them.
    *     TableRow row = c.element();
@@ -288,18 +289,20 @@ public static String toTableSpec(TableReference ref) {
    *       }
    *     }
    *   }
-   * }
-   * 
+ * }}
*/ public static class Read { + /** + * Returns a {@link Read.Bound} with the given name. The BigQuery table or query to be read + * from has not yet been configured. + */ public static Bound named(String name) { return new Bound().named(name); } /** - * Reads a BigQuery table specified as - * "[project_id]:[dataset_id].[table_id]" or "[dataset_id].[table_id]" for - * tables within the current project. + * Reads a BigQuery table specified as {@code "[project_id]:[dataset_id].[table_id]"} or + * {@code "[dataset_id].[table_id]"} for tables within the current project. */ public static Bound from(String tableSpec) { return new Bound().from(tableSpec); @@ -313,7 +316,7 @@ public static Bound fromQuery(String query) { } /** - * Reads a BigQuery table specified as a TableReference object. + * Reads a BigQuery table specified as a {@link TableReference} object. */ public static Bound from(TableReference table) { return new Bound().from(table); @@ -339,13 +342,11 @@ public static class Bound extends PTransform> { "Validation of query \"%1$s\" failed. If the query depends on an earlier stage of the" + " pipeline, This validation can be disabled using #withoutValidation."; - Bound() { - query = null; - table = null; - this.validate = true; + private Bound() { + this(null, null, null, true); } - Bound(String name, String query, TableReference reference, boolean validate) { + private Bound(String name, String query, TableReference reference, boolean validate) { super(name); this.table = reference; this.query = query; @@ -353,32 +354,39 @@ public static class Bound extends PTransform> { } /** - * Sets the name associated with this transformation. + * Returns a copy of this transform using the name associated with this transformation. + * + *

Does not modify this object. */ public Bound named(String name) { return new Bound(name, query, table, validate); } /** - * Sets the table specification. + * Returns a copy of this transform that reads from the specified table. Refer to + * {@link #parseTableSpec(String)} for the specification format. * - *

Refer to {@link #parseTableSpec(String)} for the specification format. + *

Does not modify this object. */ public Bound from(String tableSpec) { return from(parseTableSpec(tableSpec)); } /** - * Sets the BigQuery query to be used. + * Returns a copy of this transform that reads from the specified table. + * + *

Does not modify this object. */ - public Bound fromQuery(String query) { + public Bound from(TableReference table) { return new Bound(name, query, table, validate); } /** - * Sets the table specification. + * Returns a copy of this transform that reads the results of the specified query. + * + *

Does not modify this object. */ - public Bound from(TableReference table) { + public Bound fromQuery(String query) { return new Bound(name, query, table, validate); } @@ -471,12 +479,15 @@ public void evaluate( } /** - * Returns the table to write. + * Returns the table to write, or {@code null} if reading from a query instead. */ public TableReference getTable() { return table; } + /** + * Returns the query to be read, or {@code null} if reading from a table instead. + */ public String getQuery() { return query; } @@ -488,6 +499,9 @@ public boolean getValidate() { return validate; } } + + /** Disallow construction of utility class. */ + private Read() {} } ///////////////////////////////////////////////////////////////////////////// @@ -496,36 +510,36 @@ public boolean getValidate() { * A {@link PTransform} that writes a {@link PCollection} containing {@link TableRow TableRows} * to a BigQuery table. * - *

By default, tables will be created if they do not exist, which - * corresponds to a {@code CreateDisposition.CREATE_IF_NEEDED} disposition - * that matches the default of BigQuery's Jobs API. A schema must be - * provided (via {@link BigQueryIO.Write#withSchema}), or else the transform may fail - * at runtime with an {@link java.lang.IllegalArgumentException}. + *

In BigQuery, each table has an encosing dataset. The dataset being written must already + * exist. * - *

The dataset being written must already exist. + *

By default, tables will be created if they do not exist, which corresponds to a + * {@link CreateDisposition#CREATE_IF_NEEDED} disposition that matches the default of BigQuery's + * Jobs API. A schema must be provided (via {@link BigQueryIO.Write#withSchema(TableSchema)}), + * or else the transform may fail at runtime with an {@link IllegalArgumentException}. * *

By default, writes require an empty table, which corresponds to - * a {@code WriteDisposition.WRITE_EMPTY} disposition that matches the + * a {@link WriteDisposition#WRITE_EMPTY} disposition that matches the * default of BigQuery's Jobs API. * *

Here is a sample transform that produces TableRow values containing * "word" and "count" columns: - *


-   * static class FormatCountsFn extends DoFnP{@literal , TableRow>} {
-   *   {@literal @}Override
+   * 
{@code
+   * static class FormatCountsFn extends DoFn, TableRow> {
    *   public void processElement(ProcessContext c) {
    *     TableRow row = new TableRow()
    *         .set("word", c.element().getKey())
    *         .set("count", c.element().getValue().intValue());
    *     c.output(row);
    *   }
-   * }
-   * 
+ * }}
*/ public static class Write { /** - * An enumeration type for the BigQuery create disposition strings publicly - * documented as {@code CREATE_NEVER}, and {@code CREATE_IF_NEEDED}. + * An enumeration type for the BigQuery create disposition strings. + * + * @see + * configuration.query.createDisposition in the BigQuery Jobs API */ public enum CreateDisposition { /** @@ -552,9 +566,10 @@ public enum CreateDisposition { } /** - * An enumeration type for the BigQuery write disposition strings publicly - * documented as {@code WRITE_TRUNCATE}, {@code WRITE_APPEND}, and - * {@code WRITE_EMPTY}. + * An enumeration type for the BigQuery write disposition strings. + * + * @see + * configuration.query.writeDisposition in the BigQuery Jobs API */ public enum WriteDisposition { /** @@ -562,7 +577,7 @@ public enum WriteDisposition { * *

The replacement may occur in multiple steps - for instance by first * removing the existing table, then creating a replacement, then filling - * it in. This is not an atomic operation, and external programs may + * it in. This is not an atomic operation, and external programs may * see the table in any of these intermediate steps. */ WRITE_TRUNCATE, @@ -579,16 +594,17 @@ public enum WriteDisposition { *

If the output table is not empty, the write fails at runtime. * *

This check may occur long before data is written, and does not - * guarantee exclusive access to the table. If two programs are run + * guarantee exclusive access to the table. If two programs are run * concurrently, each specifying the same output table and - * a {@link WriteDisposition} of {@code WRITE_EMPTY}, it is possible + * a {@link WriteDisposition} of {@link WriteDisposition#WRITE_EMPTY}, it is possible * for both to succeed. */ WRITE_EMPTY } /** - * Sets the name associated with this transformation. + * Creates a write transformation with the given transform name. The BigQuery table to be + * written has not yet been configured. */ public static Bound named(String name) { return new Bound().named(name); @@ -608,20 +624,27 @@ public static Bound to(TableReference table) { return new Bound().to(table); } - /** Creates a write transformation from a function that maps windows to table specifications. + /** + * Creates a write transformation from a function that maps windows to table specifications. * Each time a new window is encountered, this function will be called and the resulting table * will be created. Records within that window will be written to the associated table. * - *

See {@link #parseTableSpec(String)} for the format that tableSpecFunction should return. + *

See {@link #parseTableSpec(String)} for the format that {@code tableSpecFunction} should + * return. * - *

tableSpecFunction should be determinstic. When given the same window, it should always - * return the same table specification. + *

{@code tableSpecFunction} should be deterministic. When given the same window, it should + * always return the same table specification. */ public static Bound to(SerializableFunction tableSpecFunction) { return new Bound().to(tableSpecFunction); } - /** Creates a write transformation from a function that maps windows to TableReference objects. + /** + * Creates a write transformation from a function that maps windows to {@link TableReference} + * objects. + * + *

{@code tableRefFunction} should be deterministic. When given the same window, it should + * always return the same table reference. */ public static Bound toTableReference( SerializableFunction tableRefFunction) { @@ -629,28 +652,28 @@ public static Bound toTableReference( } /** - * Specifies a table schema to use in table creation. + * Creates a write transformation with the specified schema to use in table creation. * - *

The schema is required only if writing to a table that does not already - * exist, and {@link BigQueryIO.Write.CreateDisposition} is set to - * {@code CREATE_IF_NEEDED}. + *

The schema is required only if writing to a table that does not already + * exist, and {@link CreateDisposition} is set to + * {@link CreateDisposition#CREATE_IF_NEEDED}. */ public static Bound withSchema(TableSchema schema) { return new Bound().withSchema(schema); } - /** Specifies options for creating the table. */ + /** Creates a write transformation with the specified options for creating the table. */ public static Bound withCreateDisposition(CreateDisposition disposition) { return new Bound().withCreateDisposition(disposition); } - /** Specifies options for writing to the table. */ + /** Creates a write transformation with the specified options for writing to the table. */ public static Bound withWriteDisposition(WriteDisposition disposition) { return new Bound().withWriteDisposition(disposition); } /** - * Disables BigQuery table validation, which is enabled by default. + * Creates a write transformation with BigQuery table validation disabled. */ public static Bound withoutValidation() { return new Bound().withoutValidation(); @@ -693,16 +716,18 @@ public TableReference apply(BoundedWindow value) { } } + /** + * @deprecated Should be private. Instead, use one of the factory methods in + * {@link BigQueryIO.Write}, such as {@link BigQueryIO.Write#to(String)}, to create an + * instance of this class. + */ + @Deprecated public Bound() { - this.table = null; - this.tableRefFunction = null; - this.schema = null; - this.createDisposition = CreateDisposition.CREATE_IF_NEEDED; - this.writeDisposition = WriteDisposition.WRITE_EMPTY; - this.validate = true; + this(null, null, null, null, CreateDisposition.CREATE_IF_NEEDED, + WriteDisposition.WRITE_EMPTY, true); } - Bound(String name, TableReference ref, + private Bound(String name, TableReference ref, SerializableFunction tableRefFunction, TableSchema schema, CreateDisposition createDisposition, WriteDisposition writeDisposition, boolean validate) { @@ -716,7 +741,9 @@ public Bound() { } /** - * Sets the name associated with this transformation. + * Returns a copy of this write transformation, but with the specified transform name. + * + *

Does not modify this object. */ public Bound named(String name) { return new Bound(name, table, tableRefFunction, schema, createDisposition, @@ -724,27 +751,48 @@ public Bound named(String name) { } /** - * Specifies the table specification. + * Returns a copy of this write transformation, but writing to the specified table. Refer to + * {@link #parseTableSpec(String)} for the specification format. * - *

Refer to {@link #parseTableSpec(String)} for the specification format. + *

Does not modify this object. */ public Bound to(String tableSpec) { return to(parseTableSpec(tableSpec)); } /** - * Specifies the table to be written to. + * Returns a copy of this write transformation, but writing to the specified table. + * + *

Does not modify this object. */ public Bound to(TableReference table) { return new Bound(name, table, tableRefFunction, schema, createDisposition, writeDisposition, validate); } + /** + * Returns a copy of this write transformation, but using the specified function to determine + * which table to write to for each window. + * + *

Does not modify this object. + * + *

{@code tableSpecFunction} should be deterministic. When given the same window, it + * should always return the same table specification. + */ public Bound to( SerializableFunction tableSpecFunction) { return toTableReference(new TranslateTableSpecFunction(tableSpecFunction)); } + /** + * Returns a copy of this write transformation, but using the specified function to determine + * which table to write to for each window. + * + *

Does not modify this object. + * + *

{@code tableRefFunction} should be deterministic. When given the same window, it should + * always return the same table reference. + */ public Bound toTableReference( SerializableFunction tableRefFunction) { return new Bound(name, table, tableRefFunction, schema, createDisposition, @@ -752,27 +800,40 @@ public Bound toTableReference( } /** - * Specifies the table schema, used if the table is created. + * Returns a copy of this write transformation, but using the specified schema for rows + * to be written. + * + *

Does not modify this object. */ public Bound withSchema(TableSchema schema) { return new Bound(name, table, tableRefFunction, schema, createDisposition, writeDisposition, validate); } - /** Specifies options for creating the table. */ + /** + * Returns a copy of this write transformation, but using the specified create disposition. + * + *

Does not modify this object. + */ public Bound withCreateDisposition(CreateDisposition createDisposition) { return new Bound(name, table, tableRefFunction, schema, createDisposition, writeDisposition, validate); } - /** Specifies options for writing the table. */ + /** + * Returns a copy of this write transformation, but using the specified write disposition. + * + *

Does not modify this object. + */ public Bound withWriteDisposition(WriteDisposition writeDisposition) { return new Bound(name, table, tableRefFunction, schema, createDisposition, writeDisposition, validate); } /** - * Disable table validation. + * Returns a copy of this write transformation, but without BigQuery table validation. + * + *

Does not modify this object. */ public Bound withoutValidation() { return new Bound(name, table, tableRefFunction, schema, createDisposition, @@ -894,19 +955,22 @@ public TableSchema getSchema() { return schema; } - /** Returns the table reference. */ + /** Returns the table reference, or {@code null} if a . */ public TableReference getTable() { return table; } - /** Returns true if table validation is enabled. */ + /** Returns {@code true} if table validation is enabled. */ public boolean getValidate() { return validate; } } + + /** Disallow construction of utility class. */ + private Write() {} } - public static void verifyDatasetPresence(BigQueryOptions options, TableReference table) { + private static void verifyDatasetPresence(BigQueryOptions options, TableReference table) { try { Bigquery client = Transport.newBigQueryClient(options).build(); BigQueryTableRowIterator.executeWithBackOff( @@ -927,7 +991,7 @@ public static void verifyDatasetPresence(BigQueryOptions options, TableReference } } - public static void verifyTablePresence(BigQueryOptions options, TableReference table) { + private static void verifyTablePresence(BigQueryOptions options, TableReference table) { try { Bigquery client = Transport.newBigQueryClient(options).build(); BigQueryTableRowIterator.executeWithBackOff( @@ -954,7 +1018,7 @@ public static void verifyTablePresence(BigQueryOptions options, TableReference t */ private static class StreamingWriteFn extends DoFn, TableRowInfo>, Void> { - /** TableSchema in JSON. Use String to make the class Serializable. */ + /** TableSchema in JSON. Use String to make the class Serializable. */ private final String jsonTableSchema; /** JsonTableRows to accumulate BigQuery rows in order to batch writes. */ @@ -1067,9 +1131,9 @@ public int getShardNumber() { } /** - * A {@link Coder} for {@code ShardedKey}, using a wrapped key {@code Coder}. + * A {@link Coder} for {@link ShardedKey}, using a wrapped key {@link Coder}. */ - public static class ShardedKeyCoder + private static class ShardedKeyCoder extends StandardCoder> { public static ShardedKeyCoder of(Coder keyCoder) { return new ShardedKeyCoder<>(keyCoder); @@ -1177,7 +1241,7 @@ private static class TagWithUniqueIdsAndTable /** TableSpec to write to. */ private final String tableSpec; - /** User function mapping windows to TableReference in JSON. */ + /** User function mapping windows to {@link TableReference} in JSON. */ private final SerializableFunction tableRefFunction; private transient String randomUUID; @@ -1281,7 +1345,7 @@ public PDone apply(PCollection input) { .apply(ParDo.of(new StreamingWriteFn(tableSchema))); // Note that the implementation to return PDone here breaks the - // implicit assumption about the job execution order. If a user + // implicit assumption about the job execution order. If a user // implements a PTransform that takes PDone returned here as its // input, the transform may not necessarily be executed after // the BigQueryIO.Write. @@ -1292,6 +1356,9 @@ public PDone apply(PCollection input) { ///////////////////////////////////////////////////////////////////////////// + /** Disallow construction of utility class. */ + private BigQueryIO() {} + /** * Direct mode read evaluator. * From 0416263a0d8aac778fffef0856b620213d3dba1b Mon Sep 17 00:00:00 2001 From: dhalperi Date: Wed, 18 Nov 2015 16:47:18 -0800 Subject: [PATCH 1179/1541] PubSubIO: improve Javadoc ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108196067 --- .../cloud/dataflow/sdk/io/PubsubIO.java | 560 +++++++++++------- 1 file changed, 335 insertions(+), 225 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index 951e6e4141262..99c802964baf7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -28,10 +28,14 @@ import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.Transport; import com.google.cloud.dataflow.sdk.util.WindowingStrategy; @@ -43,7 +47,6 @@ import org.joda.time.Duration; import org.joda.time.Instant; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,24 +57,24 @@ import java.util.List; import java.util.Map; import java.util.Random; - import java.util.regex.Matcher; import java.util.regex.Pattern; import javax.annotation.Nullable; /** - * Read and Write {@link PTransform}s for Pub/Sub streams. These transforms create - * and consume unbounded {@link com.google.cloud.dataflow.sdk.values.PCollection}s. + * Read and Write {@link PTransform}s for Cloud Pub/Sub streams. These transforms create + * and consume unbounded {@link PCollection PCollections}. * - *

Permissions

- * Permission requirements depend on the - * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner PipelineRunner} that is - * used to execute the Dataflow job. Please refer to the documentation of corresponding - * {@code PipelineRunner}s for more details. + *

Permissions

+ *

Permission requirements depend on the {@link PipelineRunner} that is used to execute the + * Dataflow job. Please refer to the documentation of corresponding + * {@link PipelineRunner PipelineRunners} for more details. */ public class PubsubIO { private static final Logger LOG = LoggerFactory.getLogger(PubsubIO.class); + + /** The default {@link Coder} used to translate to/from Cloud Pub/Sub messages. */ public static final Coder DEFAULT_PUBSUB_CODER = StringUtf8Coder.of(); /** @@ -86,17 +89,14 @@ public class PubsubIO { private static final Pattern SUBSCRIPTION_REGEXP = Pattern.compile("projects/([^/]+)/subscriptions/(.+)"); - private static final Pattern TOPIC_REGEXP = - Pattern.compile("projects/([^/]+)/topics/(.+)"); + private static final Pattern TOPIC_REGEXP = Pattern.compile("projects/([^/]+)/topics/(.+)"); private static final Pattern V1BETA1_SUBSCRIPTION_REGEXP = Pattern.compile("/subscriptions/([^/]+)/(.+)"); - private static final Pattern V1BETA1_TOPIC_REGEXP = - Pattern.compile("/topics/([^/]+)/(.+)"); + private static final Pattern V1BETA1_TOPIC_REGEXP = Pattern.compile("/topics/([^/]+)/(.+)"); - private static final Pattern PUBSUB_NAME_REGEXP = - Pattern.compile("[a-zA-Z][-._~%+a-zA-Z0-9]+"); + private static final Pattern PUBSUB_NAME_REGEXP = Pattern.compile("[a-zA-Z][-._~%+a-zA-Z0-9]+"); private static final int PUBSUB_NAME_MAX_LENGTH = 255; @@ -119,20 +119,18 @@ private static void validatePubsubName(String name) { } if (name.startsWith("goog")) { - throw new IllegalArgumentException( - "Pubsub object name cannot start with goog: " + name); + throw new IllegalArgumentException("Pubsub object name cannot start with goog: " + name); } Matcher match = PUBSUB_NAME_REGEXP.matcher(name); if (!match.matches()) { - throw new IllegalArgumentException( - "Illegal Pubsub object name specified: " + name + throw new IllegalArgumentException("Illegal Pubsub object name specified: " + name + " Please see Javadoc for naming rules."); } } /** - * Class representing a Pubsub Subscription. + * Class representing a Cloud Pub/Sub Subscription. */ public static class PubsubSubscription implements Serializable { private enum Type { NORMAL, FAKE } @@ -147,6 +145,23 @@ private PubsubSubscription(Type type, String project, String subscription) { this.subscription = subscription; } + /** + * Creates a class representing a Pub/Sub subscription from the specified subscription path. + * + *

Cloud Pub/Sub subscription names should be of the form + * {@code projects//subscriptions/}, where {@code } is the name + * of the project the subscription belongs to. The {@code } component must comply + * with the following requirements: + * + *

    + *
  • Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods + * ('.').
  • + *
  • Must be between 3 and 255 characters.
  • + *
  • Must begin with a letter.
  • + *
  • Must end with a letter or a number.
  • + *
  • Cannot begin with {@code 'goog'} prefix.
  • + *
+ */ public static PubsubSubscription fromPath(String path) { if (path.startsWith(SUBSCRIPTION_RANDOM_TEST_PREFIX) || path.startsWith(SUBSCRIPTION_STARTING_SIGNAL)) { @@ -164,8 +179,7 @@ public static PubsubSubscription fromPath(String path) { } else { Matcher match = SUBSCRIPTION_REGEXP.matcher(path); if (!match.matches()) { - throw new IllegalArgumentException( - "Pubsub subscription is not in " + throw new IllegalArgumentException("Pubsub subscription is not in " + "projects//subscriptions/ format: " + path); } projectName = match.group(1); @@ -177,6 +191,13 @@ public static PubsubSubscription fromPath(String path) { return new PubsubSubscription(Type.NORMAL, projectName, subscriptionName); } + /** + * Returns the string representation of this subscription as a path used in the Cloud Pub/Sub + * v1beta1 API. + * + * @deprecated the v1beta1 API for Cloud Pub/Sub is deprecated. + */ + @Deprecated public String asV1Beta1Path() { if (type == Type.NORMAL) { return "/subscriptions/" + project + "/" + subscription; @@ -185,6 +206,13 @@ public String asV1Beta1Path() { } } + /** + * Returns the string representation of this subscription as a path used in the Cloud Pub/Sub + * v1beta2 API. + * + * @deprecated the v1beta2 API for Cloud Pub/Sub is deprecated. + */ + @Deprecated public String asV1Beta2Path() { if (type == Type.NORMAL) { return "projects/" + project + "/subscriptions/" + subscription; @@ -192,10 +220,22 @@ public String asV1Beta2Path() { return subscription; } } + + /** + * Returns the string representation of this subscription as a path used in the Cloud Pub/Sub + * API. + */ + public String asPath() { + if (type == Type.NORMAL) { + return "projects/" + project + "/subscriptions/" + subscription; + } else { + return subscription; + } + } } /** - * Class representing a Pubsub Topic. + * Class representing a Cloud Pub/Sub Topic. */ public static class PubsubTopic implements Serializable { private enum Type { NORMAL, FAKE } @@ -204,12 +244,29 @@ private enum Type { NORMAL, FAKE } private final String project; private final String topic; - public PubsubTopic(Type type, String project, String topic) { + private PubsubTopic(Type type, String project, String topic) { this.type = type; this.project = project; this.topic = topic; } + /** + * Creates a class representing a Cloud Pub/Sub topic from the specified topic path. + * + *

Cloud Pub/Sub topic names should be of the form + * {@code /topics//}, where {@code } is the name of + * the publishing project. The {@code } component must comply with + * the following requirements: + * + *

    + *
  • Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods + * ('.').
  • + *
  • Must be between 3 and 255 characters.
  • + *
  • Must begin with a letter.
  • + *
  • Must end with a letter or a number.
  • + *
  • Cannot begin with 'goog' prefix.
  • + *
+ */ public static PubsubTopic fromPath(String path) { if (path.equals(TOPIC_DEV_NULL_TEST_NAME)) { return new PubsubTopic(Type.FAKE, "", path); @@ -227,8 +284,7 @@ public static PubsubTopic fromPath(String path) { Matcher match = TOPIC_REGEXP.matcher(path); if (!match.matches()) { throw new IllegalArgumentException( - "Pubsub topic is not in projects//topics/ format: " - + path); + "Pubsub topic is not in projects//topics/ format: " + path); } projectName = match.group(1); topicName = match.group(2); @@ -239,6 +295,13 @@ public static PubsubTopic fromPath(String path) { return new PubsubTopic(Type.NORMAL, projectName, topicName); } + /** + * Returns the string representation of this topic as a path used in the Cloud Pub/Sub + * v1beta1 API. + * + * @deprecated the v1beta1 API for Cloud Pub/Sub is deprecated. + */ + @Deprecated public String asV1Beta1Path() { if (type == Type.NORMAL) { return "/topics/" + project + "/" + topic; @@ -247,6 +310,13 @@ public String asV1Beta1Path() { } } + /** + * Returns the string representation of this topic as a path used in the Cloud Pub/Sub + * v1beta2 API. + * + * @deprecated the v1beta2 API for Cloud Pub/Sub is deprecated. + */ + @Deprecated public String asV1Beta2Path() { if (type == Type.NORMAL) { return "projects/" + project + "/topics/" + topic; @@ -254,39 +324,46 @@ public String asV1Beta2Path() { return topic; } } + + /** + * Returns the string representation of this topic as a path used in the Cloud Pub/Sub + * API. + */ + public String asPath() { + if (type == Type.NORMAL) { + return "projects/" + project + "/topics/" + topic; + } else { + return topic; + } + } } /** - * A {@link PTransform} that continuously reads from a Pubsub stream and - * returns a {@code PCollection} containing the items from + * A {@link PTransform} that continuously reads from a Cloud Pub/Sub stream and + * returns a {@link PCollection} of {@link String Strings} containing the items from * the stream. * - *

When running with a runner that only supports bounded {@code PCollection}s - * (such as DirectPipelineRunner or DataflowPipelineRunner without --streaming), only a - * bounded portion of the input Pubsub stream can be processed. As such, either - * {@link Bound#maxNumRecords} or {@link Bound#maxReadTime} must be set. + *

When running with a {@link PipelineRunner} that only supports bounded + * {@link PCollection PCollections} (such as {@link DirectPipelineRunner} or + * {@link DataflowPipelineRunner} without {@code --streaming}), only a bounded portion of the + * input Pub/Sub stream can be processed. As such, either {@link Bound#maxNumRecords(int)} or + * {@link Bound#maxReadTime(Duration)} must be set. */ public static class Read { + /** + * Creates and returns a transform for reading from Cloud Pub/Sub with the specified transform + * name. + */ public static Bound named(String name) { return new Bound<>(DEFAULT_PUBSUB_CODER).named(name); } /** - * Creates and returns a PubsubIO.Read PTransform for reading from - * a Pubsub topic with the specified publisher topic. Format for - * Cloud Pubsub topic names should be of the form - * {@code /topics//}, where {@code } is the name of - * the publishing project. The {@code } component must comply with - * the below requirements. + * Creates and returns a transform for reading from a Cloud Pub/Sub topic. Mutually exclusive + * with {@link #subscription(String)}. * - *

    - *
  • Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods - * ('.').
  • - *
  • Must be between 3 and 255 characters.
  • - *
  • Must begin with a letter.
  • - *
  • Must end with a letter or a number.
  • - *
  • Cannot begin with 'goog' prefix.
  • - *
+ *

See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the format + * of the {@code topic} string. * *

Dataflow will start reading data published on this topic from the time the pipeline is * started. Any data published on the topic before the pipeline is started will not be read by @@ -297,42 +374,31 @@ public static Bound topic(String topic) { } /** - * Creates and returns a PubsubIO.Read PTransform for reading from - * a specific Pubsub subscription. Mutually exclusive with - * PubsubIO.Read.topic(). - * Cloud Pubsub subscription names should be of the form - * {@code projects//subscriptions/}, - * where {@code } is the name of the project the subscription belongs to. - * The {@code } component must comply with the below requirements. + * Creates and returns a transform for reading from a specific Cloud Pub/Sub subscription. + * Mutually exclusive with {@link #topic(String)}. * - *

    - *
  • Can only contain lowercase letters, numbers, dashes ('-'), underscores ('_') and periods - * ('.').
  • - *
  • Must be between 3 and 255 characters.
  • - *
  • Must begin with a letter.
  • - *
  • Must end with a letter or a number.
  • - *
  • Cannot begin with 'goog' prefix.
  • - *
+ *

See {@link PubsubIO.PubsubSubscription#fromPath(String)} for more details on the format + * of the {@code subscription} string. */ public static Bound subscription(String subscription) { return new Bound<>(DEFAULT_PUBSUB_CODER).subscription(subscription); } /** - * Creates and returns a PubsubIO.Read PTransform where record timestamps are expected - * to be provided using the PubSub labeling API. The {@code } parameter - * specifies the label name. The label value sent to PubsSub is a numerical value representing - * the number of milliseconds since the Unix epoch. For example, if using the joda time classes, - * org.joda.time.Instant.getMillis() returns the correct value for this label. + * Creates and returns a transform reading from Cloud Pub/Sub where record timestamps are + * expected to be provided as Pub/Sub message attributes. The {@code timestampLabel} + * parameter specifies the name of the attribute that contains the timestamp. The value of the + * attribute should be a numerical value representing the number of milliseconds since the Unix + * epoch. For example, if using the Joda time classes, + * {@link Instant#getMillis()} returns the correct value for this label. * - *

If {@code } is not provided, the system will generate record timestamps + *

If {@code timestampLabel} is not provided, the system will generate record timestamps * the first time it sees each record. All windowing will be done relative to these timestamps. * - *

By default windows are emitted based on an estimate of when this source is likely + *

By default, windows are emitted based on an estimate of when this source is likely * done producing data for a given timestamp (referred to as the Watermark; see - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark} for more details). - * Any late data will be handled by the trigger specified with the windowing strategy -- by - * default it will be output immediately. + * {@link AfterWatermark} for more details). Any late data will be handled by the trigger + * specified with the windowing strategy – by default it will be output immediately. * *

Note that the system can guarantee that no late data will ever be seen when it assigns * timestamps by arrival time (i.e. {@code timestampLabel} is not provided). @@ -342,22 +408,22 @@ public static Bound timestampLabel(String timestampLabel) { } /** - * Creates and returns a PubSubIO.Read PTransform where unique record identifiers are - * expected to be provided using the PubSub labeling API. The {@code } parameter - * specifies the label name. The label value sent to PubSub can be any string value that - * uniquely identifies this record. + * Creates and returns a transform for reading from Cloud Pub/Sub where unique record + * identifiers are expected to be provided as Pub/Sub message attributes. The {@code idLabel} + * parameter specifies the attribute name. The value of the attribute can be any string + * that uniquely identifies this record. * - *

If idLabel is not provided, Dataflow cannot guarantee that no duplicate data will be - * delivered on the PubSub stream. In this case, deduplication of the stream will be - * stricly best effort. + *

If {@code idLabel} is not provided, Dataflow cannot guarantee that no duplicate data will + * be delivered on the Pub/Sub stream. In this case, deduplication of the stream will be + * strictly best effort. */ public static Bound idLabel(String idLabel) { return new Bound<>(DEFAULT_PUBSUB_CODER).idLabel(idLabel); } - /** - * Creates and returns a PubsubIO.Read PTransform that uses the given - * {@code Coder} to decode PubSub record into a value of type {@code T}. + /** + * Creates and returns a transform for reading from Cloud Pub/Sub that uses the given + * {@link Coder} to decode Pub/Sub messages into a value of type {@code T}. * *

By default, uses {@link StringUtf8Coder}, which just * returns the text lines as Java strings. @@ -370,60 +436,64 @@ public static Bound withCoder(Coder coder) { } /** - * Sets the maximum number of records that will be read from Pubsub. + * Creates and returns a transform for reading from Cloud Pub/Sub with a maximum number of + * records that will be read. The transform produces a bounded {@link PCollection}. * - *

Either this or {@link #maxReadTime} must be set for use as a bounded - * {@code PCollection}. + *

Either this option or {@link #maxReadTime(Duration)} must be set in order to create a + * bounded source. */ public static Bound maxNumRecords(int maxNumRecords) { return new Bound<>(DEFAULT_PUBSUB_CODER).maxNumRecords(maxNumRecords); } /** - * Sets the maximum duration during which records will be read from Pubsub. + * Creates and returns a transform for reading from Cloud Pub/Sub with a maximum number of + * duration during which records will be read. The transform produces a bounded + * {@link PCollection}. * - *

Either this or {@link #maxNumRecords} must be set for use as a bounded - * {@code PCollection}. + *

Either this option or {@link #maxNumRecords(int)} must be set in order to create a bounded + * source. */ public static Bound maxReadTime(Duration maxReadTime) { return new Bound<>(DEFAULT_PUBSUB_CODER).maxReadTime(maxReadTime); } /** - * A {@link PTransform} that reads from a PubSub source and returns - * a unbounded PCollection containing the items from the stream. + * A {@link PTransform} that reads from a Cloud Pub/Sub source and returns + * a unbounded {@link PCollection} containing the items from the stream. */ public static class Bound extends PTransform> { - /** The Pubsub topic to read from. */ - PubsubTopic topic; - /** The Pubsub subscription to read from. */ - PubsubSubscription subscription; - /** The Pubsub label to read timestamps from. */ - String timestampLabel; - /** The Pubsub label to read ids from. */ - String idLabel; + /** The Cloud Pub/Sub topic to read from. */ + @Nullable private final PubsubTopic topic; + + /** The Cloud Pub/Sub subscription to read from. */ + @Nullable private final PubsubSubscription subscription; + + /** The name of the message attribute to read timestamps from. */ + @Nullable private final String timestampLabel; + + /** The name of the message attribute to read unique message IDs from. */ + @Nullable private final String idLabel; + /** The coder used to decode each record. */ - @Nullable - final Coder coder; + @Nullable private final Coder coder; + /** Stop after reading this many records. */ - int maxNumRecords; + private final int maxNumRecords; + /** Stop after reading for this much time. */ - Duration maxReadTime; + @Nullable private final Duration maxReadTime; - Bound(Coder coder) { - this.coder = coder; + private Bound(Coder coder) { + this(null, null, null, null, coder, null, 0, null); } - Bound(String name, PubsubSubscription subscription, PubsubTopic topic, String timestampLabel, - Coder coder, String idLabel, - int maxNumRecords, Duration maxReadTime) { + private Bound(String name, PubsubSubscription subscription, PubsubTopic topic, + String timestampLabel, Coder coder, String idLabel, int maxNumRecords, + Duration maxReadTime) { super(name); - if (subscription != null) { - this.subscription = subscription; - } - if (topic != null) { - this.topic = topic; - } + this.subscription = subscription; + this.topic = topic; this.timestampLabel = timestampLabel; this.coder = coder; this.idLabel = idLabel; @@ -432,21 +502,27 @@ public static class Bound extends PTransform> { } /** - * Returns a new PubsubIO.Read PTransform that's like this one but with the given - * step name. Does not modify the object. + * Returns a transform that's like this one but with the given step name. + * + *

Does not modify this object. */ public Bound named(String name) { - return new Bound<>(name, subscription, topic, timestampLabel, - coder, idLabel, maxNumRecords, maxReadTime); + return new Bound<>( + name, subscription, topic, timestampLabel, coder, idLabel, maxNumRecords, maxReadTime); } /** - * Returns a new PubsubIO.Read PTransform that's like this one but reading from the - * given subscription. Does not modify the object. + * Returns a transform that's like this one but reading from the + * given subscription. + * + *

See {@link PubsubIO.PubsubSubscription#fromPath(String)} for more details on the format + * of the {@code subscription} string. * *

Multiple readers reading from the same subscription will each receive - * some arbirary portion of the data. Most likely, separate readers should + * some arbitrary portion of the data. Most likely, separate readers should * use their own subscriptions. + * + *

Does not modify this object. */ public Bound subscription(String subscription) { return new Bound<>(name, PubsubSubscription.fromPath(subscription), topic, timestampLabel, @@ -454,77 +530,84 @@ public Bound subscription(String subscription) { } /** - * Returns a new PubsubIO.Read PTransform that's like this one but reading from the - * give topic. Does not modify the object. + * Returns a transform that's like this one but that reads from the specified topic. + * + *

See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the + * format of the {@code topic} string. + * + *

Does not modify this object. */ public Bound topic(String topic) { - return new Bound<>(name, subscription, PubsubTopic.fromPath(topic), timestampLabel, - coder, idLabel, maxNumRecords, maxReadTime); + return new Bound<>(name, subscription, PubsubTopic.fromPath(topic), timestampLabel, coder, + idLabel, maxNumRecords, maxReadTime); } /** - * Returns a new PubsubIO.Read PTransform that's like this one but reading timestamps - * from the given PubSub label. Does not modify the object. + * Returns a transform that's like this one but that reads message timestamps + * from the given message attribute. See {@link PubsubIO.Read#timestampLabel(String)} for + * more details on the format of the timestamp attribute. + * + *

Does not modify this object. */ public Bound timestampLabel(String timestampLabel) { - return new Bound<>(name, subscription, topic, timestampLabel, coder, idLabel, - maxNumRecords, maxReadTime); + return new Bound<>( + name, subscription, topic, timestampLabel, coder, idLabel, maxNumRecords, maxReadTime); } /** - * Returns a new PubsubIO.Read PTransform that's like this one but reading unique ids - * from the given PubSub label. Does not modify the object. + * Returns a transform that's like this one but that reads unique message IDs + * from the given message attribute. See {@link PubsubIO.Read#idLabel(String)} for more + * details on the format of the ID attribute. + * + *

Does not modify this object. */ public Bound idLabel(String idLabel) { - return new Bound<>(name, subscription, topic, timestampLabel, coder, idLabel, - maxNumRecords, maxReadTime); + return new Bound<>( + name, subscription, topic, timestampLabel, coder, idLabel, maxNumRecords, maxReadTime); } /** - * Returns a new PubsubIO.Read PTransform that's like this one but that uses the given - * {@code Coder} to decode each record into a value of type {@code X}. Does not modify - * this object. + * Returns a transform that's like this one but that uses the given + * {@link Coder} to decode each record into a value of type {@code X}. + * + *

Does not modify this object. * * @param the type of the decoded elements, and the * elements of the resulting PCollection. */ public Bound withCoder(Coder coder) { - return new Bound<>(name, subscription, topic, timestampLabel, coder, idLabel, - maxNumRecords, maxReadTime); + return new Bound<>( + name, subscription, topic, timestampLabel, coder, idLabel, maxNumRecords, maxReadTime); } /** - * Sets the maximum number of records that will be read from Pubsub. - * - *

Setting either this or {@link #maxReadTime} will cause the output {@code PCollection} - * to be bounded. + * Returns a transform that's like this one but will only read up to the specified + * maximum number of records from Cloud Pub/Sub. The transform produces a bounded + * {@link PCollection}. See {@link PubsubIO.Read#maxNumRecords(int)} for more details. */ public Bound maxNumRecords(int maxNumRecords) { - return new Bound<>(name, subscription, topic, timestampLabel, - coder, idLabel, maxNumRecords, maxReadTime); + return new Bound<>( + name, subscription, topic, timestampLabel, coder, idLabel, maxNumRecords, maxReadTime); } /** - * Sets the maximum duration during which records will be read from Pubsub. - * - *

Setting either this or {@link #maxNumRecords} will cause the output {@code PCollection} - * to be bounded. + * Returns a transform that's like this one but will only read during the specified + * duration from Cloud Pub/Sub. The transform produces a bounded {@link PCollection}. + * See {@link PubsubIO.Read#maxReadTime(Duration)} for more details. */ public Bound maxReadTime(Duration maxReadTime) { - return new Bound<>(name, subscription, topic, timestampLabel, - coder, idLabel, maxNumRecords, maxReadTime); + return new Bound<>( + name, subscription, topic, timestampLabel, coder, idLabel, maxNumRecords, maxReadTime); } @Override public PCollection apply(PInput input) { if (topic == null && subscription == null) { - throw new IllegalStateException( - "need to set either the topic or the subscription for " + throw new IllegalStateException("need to set either the topic or the subscription for " + "a PubsubIO.Read transform"); } if (topic != null && subscription != null) { - throw new IllegalStateException( - "Can't set both the topic and the subscription for a " + throw new IllegalStateException("Can't set both the topic and the subscription for a " + "PubsubIO.Read transform"); } @@ -533,8 +616,7 @@ public PCollection apply(PInput input) { if (boundedOutput) { return input.getPipeline().begin() .apply(Create.of((Void) null)).setCoder(VoidCoder.of()) - .apply(ParDo.of(new PubsubReader())) - .setCoder(coder); + .apply(ParDo.of(new PubsubReader())).setCoder(coder); } else { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) @@ -586,23 +668,22 @@ public void processElement(ProcessContext c) throws IOException { String subscription; if (getSubscription() == null) { - String topic = getTopic().asV1Beta2Path(); + String topic = getTopic().asPath(); String[] split = topic.split("/"); - subscription = "projects/" + split[1] + "/subscriptions/" + split[3] - + "_dataflow_" + new Random().nextLong(); - Subscription subInfo = new Subscription() - .setAckDeadlineSeconds(60) - .setTopic(topic); + subscription = + "projects/" + split[1] + "/subscriptions/" + split[3] + "_dataflow_" + + new Random().nextLong(); + Subscription subInfo = new Subscription().setAckDeadlineSeconds(60).setTopic(topic); try { pubsubClient.projects().subscriptions().create(subscription, subInfo).execute(); } catch (Exception e) { throw new RuntimeException("Failed to create subscription: ", e); } } else { - subscription = getSubscription().asV1Beta2Path(); + subscription = getSubscription().asPath(); } - Instant endTime = getMaxReadTime() == null + Instant endTime = (getMaxReadTime() == null) ? new Instant(Long.MAX_VALUE) : Instant.now().plus(getMaxReadTime()); List messages = new ArrayList<>(); @@ -662,70 +743,82 @@ public void processElement(ProcessContext c) throws IOException { throw new RuntimeException( "Message from pubsub missing timestamp label: " + getTimestampLabel()); } - timestamp = new Instant(Long.parseLong( - message.getAttributes().get(getTimestampLabel()))); + timestamp = + new Instant(Long.parseLong(message.getAttributes().get(getTimestampLabel()))); } c.outputWithTimestamp( - CoderUtils.decodeFromByteArray(getCoder(), message.decodeData()), - timestamp); + CoderUtils.decodeFromByteArray(getCoder(), message.decodeData()), timestamp); } } } } + + /** Disallow construction of utility class. */ + private Read() {} } ///////////////////////////////////////////////////////////////////////////// + /** Disallow construction of utility class. */ + private PubsubIO() {} + /** * A {@link PTransform} that continuously writes a - * {@code PCollection} to a Pubsub stream. + * {@link PCollection} of {@link String Strings} to a Cloud Pub/Sub stream. */ // TODO: Support non-String encodings. public static class Write { + /** + * Creates a transform that writes to Pub/Sub with the given step name. + */ public static Bound named(String name) { return new Bound<>(DEFAULT_PUBSUB_CODER).named(name); } - /** The topic to publish to. - * Cloud Pubsub topic names should be {@code /topics//}, - * where {@code } is the name of the publishing project. + /** + * Creates a transform that publishes to the specified topic. + * + *

See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the format of the + * {@code topic} string. */ public static Bound topic(String topic) { return new Bound<>(DEFAULT_PUBSUB_CODER).topic(topic); } /** - * If specified, Dataflow will add a Pubsub label to each output record specifying the logical - * timestamp of the record. {@code } determines the label name. The label value - * is a numerical value representing the number of milliseconds since the Unix epoch. For - * example, if using the joda time classes, the org.joda.time.Instant(long) constructor can be - * used to parse this value. If the output from this sink is being read by another Dataflow - * source, then PubsubIO.Read.timestampLabel can be used to ensure that the other source reads - * these timestamps from the appropriate label. + * Creates a transform that writes to Pub/Sub, adds each record's timestamp to the published + * messages in an attribute with the specified name. The value of the attribute will be a number + * representing the number of milliseconds since the Unix epoch. For example, if using the Joda + * time classes, {@link Instant#Instant(long)} can be used to parse this value. + * + *

If the output from this sink is being read by another Dataflow source, then + * {@link PubsubIO.Read#timestampLabel(String)} can be used to ensure the other source reads + * these timestamps from the appropriate attribute. */ public static Bound timestampLabel(String timestampLabel) { return new Bound<>(DEFAULT_PUBSUB_CODER).timestampLabel(timestampLabel); } /** - * If specified, Dataflow will add a Pubsub label to each output record containing a unique - * identifier for that record. {@code } determines the label name. The label value - * is an opaque string value. This is useful if the the output from this sink is being read - * by another Dataflow source, in which case PubsubIO.Read.idLabel can be used to ensure that - * the other source reads these ids from the appropriate label. + * Creates a transform that writes to Pub/Sub, adding each record's unique identifier to the + * published messages in an attribute with the specified name. The value of the attribute is an + * opaque string. + * + *

If the the output from this sink is being read by another Dataflow source, then + * {@link PubsubIO.Read#idLabel(String)} can be used to ensure that* the other source reads + * these unique identifiers from the appropriate attribute. */ public static Bound idLabel(String idLabel) { return new Bound<>(DEFAULT_PUBSUB_CODER).idLabel(idLabel); } - /** - * Returns a TextIO.Write PTransform that uses the given - * {@code Coder} to encode each of the elements of the input - * {@code PCollection} into an output PubSub record. + /** + * Creates a transform that uses the given {@link Coder} to encode each of the + * elements of the input collection into an output message. * - *

By default, uses {@link StringUtf8Coder}, which writes input - * Java strings directly as records. + *

By default, uses {@link StringUtf8Coder}, which writes input Java strings directly as + * records. * * @param the type of the elements of the input PCollection */ @@ -734,69 +827,85 @@ public static Bound withCoder(Coder coder) { } /** - * A {@link PTransform} that writes an unbounded {@code PCollection} - * to a PubSub stream. + * A {@link PTransform} that writes an unbounded {@link PCollection} of {@link String Strings} + * to a Cloud Pub/Sub stream. */ public static class Bound extends PTransform, PDone> { - /** The Pubsub topic to publish to. */ - PubsubTopic topic; - String timestampLabel; - String idLabel; - final Coder coder; + /** The Cloud Pub/Sub topic to publish to. */ + @Nullable private final PubsubTopic topic; + /** The name of the message attribute to publish message timestamps in. */ + @Nullable private final String timestampLabel; + /** The name of the message attribute to publish unique message IDs in. */ + @Nullable private final String idLabel; + private final Coder coder; - Bound(Coder coder) { - this.coder = coder; + private Bound(Coder coder) { + this(null, null, null, null, coder); } - Bound(String name, PubsubTopic topic, String timestampLabel, String idLabel, Coder coder) { + private Bound( + String name, PubsubTopic topic, String timestampLabel, String idLabel, Coder coder) { super(name); - if (topic != null) { - this.topic = topic; - } + this.topic = topic; this.timestampLabel = timestampLabel; this.idLabel = idLabel; this.coder = coder; } /** - * Returns a new PubsubIO.Write PTransform that's like this one but with the given step - * name. Does not modify the object. + * Returns a new transform that's like this one but with the specified step + * name. + * + *

Does not modify this object. */ public Bound named(String name) { return new Bound<>(name, topic, timestampLabel, idLabel, coder); } /** - * Returns a new PubsubIO.Write PTransform that's like this one but writing to the given - * topic. Does not modify the object. + * Returns a new transform that's like this one but that writes to the specified + * topic. + * + *

See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the format of the + * {@code topic} string. + * + *

Does not modify this object. */ public Bound topic(String topic) { return new Bound<>(name, PubsubTopic.fromPath(topic), timestampLabel, idLabel, coder); } /** - * Returns a new PubsubIO.Write PTransform that's like this one but publishing timestamps - * to the given PubSub label. Does not modify the object. + * Returns a new transform that's like this one but that publishes record timestamps + * to a message attribute with the specified name. See + * {@link PubsubIO.Write#timestampLabel(String)} for more details. + * + *

Does not modify this object. */ public Bound timestampLabel(String timestampLabel) { return new Bound<>(name, topic, timestampLabel, idLabel, coder); } /** - * Returns a new PubsubIO.Write PTransform that's like this one but publishing record ids - * to the given PubSub label. Does not modify the object. + * Returns a new transform that's like this one but that publishes unique record IDs + * to a message attribute with the specified name. See {@link PubsubIO.Write#idLabel(String)} + * for more details. + * + *

Does not modify this object. */ - public Bound idLabel(String idLabel) { - return new Bound<>(name, topic, timestampLabel, idLabel, coder); + public Bound idLabel(String idLabel) { + return new Bound<>(name, topic, timestampLabel, idLabel, coder); } - /** - * Returns a new PubsubIO.Write PTransform that's like this one - * but that uses the given {@code Coder} to encode each of - * the elements of the input {@code PCollection} into an - * output record. Does not modify this object. + /** + * Returns a new transform that's like this one + * but that uses the given {@link Coder} to encode each of + * the elements of the input {@link PCollection} into an + * output record. + * + *

Does not modify this object. * - * @param the type of the elements of the input PCollection + * @param the type of the elements of the input {@link PCollection} */ public Bound withCoder(Coder coder) { return new Bound<>(name, topic, timestampLabel, idLabel, coder); @@ -805,8 +914,7 @@ public Bound withCoder(Coder coder) { @Override public PDone apply(PCollection input) { if (topic == null) { - throw new IllegalStateException( - "need to set the topic of a PubsubIO.Write transform"); + throw new IllegalStateException("need to set the topic of a PubsubIO.Write transform"); } input.apply(ParDo.of(new PubsubWriter())); return PDone.in(input.getPipeline()); @@ -838,7 +946,6 @@ private class PubsubWriter extends DoFn { private transient List output; private transient Pubsub pubsubClient; - @Override public void startBundle(Context c) { this.output = new ArrayList<>(); @@ -849,16 +956,15 @@ public void startBundle(Context c) { @Override public void processElement(ProcessContext c) throws IOException { - PubsubMessage message = new PubsubMessage().encodeData( - CoderUtils.encodeToByteArray(getCoder(), c.element())); + PubsubMessage message = + new PubsubMessage().encodeData(CoderUtils.encodeToByteArray(getCoder(), c.element())); if (getTimestampLabel() != null) { Map attributes = message.getAttributes(); if (attributes == null) { attributes = new HashMap<>(); message.setAttributes(attributes); } - attributes.put( - getTimestampLabel(), String.valueOf(c.timestamp().getMillis())); + attributes.put(getTimestampLabel(), String.valueOf(c.timestamp().getMillis())); } output.add(message); @@ -877,10 +983,14 @@ public void finishBundle(Context c) throws IOException { private void publish() throws IOException { PublishRequest publishRequest = new PublishRequest().setMessages(output); pubsubClient.projects().topics() - .publish(getTopic().asV1Beta2Path(), publishRequest).execute(); + .publish(getTopic().asPath(), publishRequest) + .execute(); output.clear(); } } } + + /** Disallow construction of utility class. */ + private Write() {} } } From 10dc3b2e20e8c0d1e90b2e60323783cd44caf443 Mon Sep 17 00:00:00 2001 From: tgroh Date: Wed, 18 Nov 2015 16:59:33 -0800 Subject: [PATCH 1180/1541] Fix WithTimestamps Java 8 Example ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108197218 --- .../google/cloud/dataflow/sdk/transforms/WithTimestamps.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithTimestamps.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithTimestamps.java index 8b581b4ea47a3..85a93bfe18d43 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithTimestamps.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithTimestamps.java @@ -59,7 +59,7 @@ public class WithTimestamps extends PTransform, PCollection *

Example of use in Java 8: *

{@code
    * PCollection timestampedRecords = records.apply(
-   *     TimestampElements.via((Record rec) -> rec.getInstant());
+   *     WithTimestamps.of((Record rec) -> rec.getInstant());
    * }
*/ public static WithTimestamps of(SerializableFunction fn) { From 3ff65d008cda98312b117278962b0593c05e5de5 Mon Sep 17 00:00:00 2001 From: peihe Date: Thu, 19 Nov 2015 00:12:46 -0800 Subject: [PATCH 1181/1541] Deprecate the incorrect chaining in RetryHttpRequestInitializer Move to use ChainingHttpRequestInitializer for chaining. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108220465 --- .../cloud/dataflow/sdk/io/DatastoreIO.java | 4 +-- .../sdk/util/RetryHttpRequestInitializer.java | 35 +++++++++++++++++- .../cloud/dataflow/sdk/util/Transport.java | 36 ++++++++++++++----- 3 files changed, 64 insertions(+), 11 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java index a9d5c13c47b74..353cacac67d21 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java @@ -466,7 +466,7 @@ private long queryLatestStatisticsTimestamp(Datastore datastore) throws Datastor private Datastore getDatastore(PipelineOptions pipelineOptions) { DatastoreOptions.Builder builder = new DatastoreOptions.Builder().host(host).dataset(datasetId).initializer( - new RetryHttpRequestInitializer(null)); + new RetryHttpRequestInitializer()); Credential credential = pipelineOptions.as(GcpOptions.class).getGcpCredential(); if (credential != null) { @@ -606,7 +606,7 @@ public DatastoreWriter createWriter(PipelineOptions options) throws Exception { new DatastoreOptions.Builder() .host(sink.host) .dataset(sink.datasetId) - .initializer(new RetryHttpRequestInitializer(null)); + .initializer(new RetryHttpRequestInitializer()); Credential credential = options.as(GcpOptions.class).getGcpCredential(); if (credential != null) { builder.credential(credential); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java index 6259428649804..756dce0a99859 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializer.java @@ -123,6 +123,7 @@ public boolean handleResponse(HttpRequest request, HttpResponse response, } } + @Deprecated private final HttpRequestInitializer chained; private final HttpResponseInterceptor responseInterceptor; // response Interceptor to use @@ -133,22 +134,51 @@ public boolean handleResponse(HttpRequest request, HttpResponse response, private Set ignoredResponseCodes = new HashSet<>(DEFAULT_IGNORED_RESPONSE_CODES); + public RetryHttpRequestInitializer() { + this(Collections.emptyList()); + } + /** * @param chained a downstream HttpRequestInitializer, which will also be * applied to HttpRequest initialization. May be null. + * + * @deprecated use {@link #RetryHttpRequestInitializer}. */ + @Deprecated public RetryHttpRequestInitializer(@Nullable HttpRequestInitializer chained) { this(chained, Collections.emptyList()); } + /** + * @param additionalIgnoredResponseCodes a list of HTTP status codes that should not be logged. + */ + public RetryHttpRequestInitializer(Collection additionalIgnoredResponseCodes) { + this(additionalIgnoredResponseCodes, null); + } + + /** * @param chained a downstream HttpRequestInitializer, which will also be * applied to HttpRequest initialization. May be null. * @param additionalIgnoredResponseCodes a list of HTTP status codes that should not be logged. + * + * @deprecated use {@link #RetryHttpRequestInitializer(Collection)}. */ + @Deprecated public RetryHttpRequestInitializer(@Nullable HttpRequestInitializer chained, Collection additionalIgnoredResponseCodes) { - this(chained, NanoClock.SYSTEM, Sleeper.DEFAULT, additionalIgnoredResponseCodes, null); + this(chained, additionalIgnoredResponseCodes, null); + } + + /** + * @param additionalIgnoredResponseCodes a list of HTTP status codes that should not be logged. + * @param responseInterceptor HttpResponseInterceptor to be applied on all requests. May be null. + */ + public RetryHttpRequestInitializer( + Collection additionalIgnoredResponseCodes, + @Nullable HttpResponseInterceptor responseInterceptor) { + this(null, NanoClock.SYSTEM, Sleeper.DEFAULT, additionalIgnoredResponseCodes, + responseInterceptor); } /** @@ -156,7 +186,10 @@ public RetryHttpRequestInitializer(@Nullable HttpRequestInitializer chained, * initialization. May be null. * @param additionalIgnoredResponseCodes a list of HTTP status codes that should not be logged. * @param responseInterceptor HttpResponseInterceptor to be applied on all requests. May be null. + * + * @deprecated use {@link #RetryHttpRequestInitializer(Collection, HttpResponseInterceptor)}. */ + @Deprecated public RetryHttpRequestInitializer( @Nullable HttpRequestInitializer chained, Collection additionalIgnoredResponseCodes, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java index 72febae67d239..7735a9e01fcc0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Transport.java @@ -16,7 +16,9 @@ package com.google.cloud.dataflow.sdk.util; +import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport; +import com.google.api.client.http.HttpRequestInitializer; import com.google.api.client.http.HttpTransport; import com.google.api.client.json.JsonFactory; import com.google.api.client.json.jackson2.JacksonFactory; @@ -28,6 +30,7 @@ import com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions; import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; import com.google.cloud.dataflow.sdk.options.GcsOptions; +import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer; import com.google.common.collect.ImmutableList; import java.io.IOException; @@ -95,8 +98,10 @@ private static ApiComponents apiComponentsFromUrl(String urlString) { public static Bigquery.Builder newBigQueryClient(BigQueryOptions options) { return new Bigquery.Builder(getTransport(), getJsonFactory(), - // Do not log 404. It clutters the output and is possibly even required by the caller. - new RetryHttpRequestInitializer(options.getGcpCredential(), ImmutableList.of(404))) + chainHttpRequestInitializer( + options.getGcpCredential(), + // Do not log 404. It clutters the output and is possibly even required by the caller. + new RetryHttpRequestInitializer(ImmutableList.of(404)))) .setApplicationName(options.getAppName()) .setGoogleClientRequestInitializer(options.getGoogleApiTrace()); } @@ -110,8 +115,10 @@ private static ApiComponents apiComponentsFromUrl(String urlString) { public static Pubsub.Builder newPubsubClient(DataflowPipelineOptions options) { return new Pubsub.Builder(getTransport(), getJsonFactory(), - // Do not log 404. It clutters the output and is possibly even required by the caller. - new RetryHttpRequestInitializer(options.getGcpCredential(), ImmutableList.of(404))) + chainHttpRequestInitializer( + options.getGcpCredential(), + // Do not log 404. It clutters the output and is possibly even required by the caller. + new RetryHttpRequestInitializer(ImmutableList.of(404)))) .setRootUrl(options.getPubsubRootUrl()) .setApplicationName(options.getAppName()) .setGoogleClientRequestInitializer(options.getGoogleApiTrace()); @@ -131,8 +138,10 @@ public static Dataflow.Builder newDataflowClient(DataflowPipelineOptions options return new Dataflow.Builder(getTransport(), getJsonFactory(), - // Do not log 404. It clutters the output and is possibly even required by the caller. - new RetryHttpRequestInitializer(options.getGcpCredential(), ImmutableList.of(404))) + chainHttpRequestInitializer( + options.getGcpCredential(), + // Do not log 404. It clutters the output and is possibly even required by the caller. + new RetryHttpRequestInitializer(ImmutableList.of(404)))) .setApplicationName(options.getAppName()) .setRootUrl(components.rootUrl) .setServicePath(components.servicePath) @@ -160,10 +169,12 @@ public static Dataflow.Builder newDataflowClient(DataflowPipelineOptions options newStorageClient(GcsOptions options) { String servicePath = options.getGcsEndpoint(); Storage.Builder storageBuilder = new Storage.Builder(getTransport(), getJsonFactory(), - new RetryHttpRequestInitializer( + chainHttpRequestInitializer( + options.getGcpCredential(), // Do not log the code 404. Code up the stack will deal with 404's if needed, and // logging it by default clutters the output during file staging. - options.getGcpCredential(), ImmutableList.of(404), new UploadIdResponseInterceptor())) + new RetryHttpRequestInitializer( + ImmutableList.of(404), new UploadIdResponseInterceptor()))) .setApplicationName(options.getAppName()) .setGoogleClientRequestInitializer(options.getGoogleApiTrace()); if (servicePath != null) { @@ -173,4 +184,13 @@ public static Dataflow.Builder newDataflowClient(DataflowPipelineOptions options } return storageBuilder; } + + private static HttpRequestInitializer chainHttpRequestInitializer( + Credential credential, HttpRequestInitializer httpRequestInitializer) { + if (credential == null) { + return httpRequestInitializer; + } else { + return new ChainingHttpRequestInitializer(credential, httpRequestInitializer); + } + } } From 056279ed7a28b618ebb3af256bce33997086f98b Mon Sep 17 00:00:00 2001 From: tgroh Date: Thu, 19 Nov 2015 11:10:39 -0800 Subject: [PATCH 1182/1541] Extend input type of Flatten#iterables This allows any PCollection with a subclass of iterable as the type of element to be the input to a Flatten#iterables PTransform. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108265658 --- .../dataflow/sdk/transforms/Flatten.java | 16 ++--- .../dataflow/sdk/transforms/FlattenTest.java | 58 ++++++++++++++++++- 2 files changed, 65 insertions(+), 9 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java index c186353b02097..2e1a3ab04f85e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java @@ -18,7 +18,7 @@ import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException; import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.coders.IterableLikeCoder; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.WindowingStrategy; @@ -164,17 +164,17 @@ protected Coder getDefaultOutputCoder(PCollectionList input) * the output {@code PCollection} */ public static class FlattenIterables - extends PTransform>, PCollection> { + extends PTransform>, PCollection> { @Override - public PCollection apply(PCollection> in) { - Coder> inCoder = in.getCoder(); - if (!(inCoder instanceof IterableCoder)) { + public PCollection apply(PCollection> in) { + Coder> inCoder = in.getCoder(); + if (!(inCoder instanceof IterableLikeCoder)) { throw new IllegalArgumentException( - "expecting the input Coder to be an IterableCoder"); + "expecting the input Coder to be an IterableLikeCoder"); } - IterableCoder iterableCoder = (IterableCoder) inCoder; - Coder elemCoder = iterableCoder.getElemCoder(); + @SuppressWarnings("unchecked") + Coder elemCoder = ((IterableLikeCoder) inCoder).getElemCoder(); return in.apply(ParDo.named("FlattenIterables").of( new DoFn, T>() { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FlattenTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FlattenTest.java index b85ec021e8102..0c9d3315db686 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FlattenTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FlattenTest.java @@ -24,7 +24,10 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CollectionCoder; import com.google.cloud.dataflow.sdk.coders.IterableCoder; +import com.google.cloud.dataflow.sdk.coders.ListCoder; +import com.google.cloud.dataflow.sdk.coders.SetCoder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; @@ -36,6 +39,7 @@ import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionList; import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.common.collect.ImmutableSet; import org.joda.time.Duration; import org.junit.Assert; @@ -49,7 +53,9 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.List; +import java.util.Set; /** * Tests for Flatten. @@ -183,6 +189,57 @@ public void testFlattenIterables() { p.run(); } + @Test + @Category(RunnableOnService.class) + public void testFlattenIterablesLists() { + Pipeline p = TestPipeline.create(); + + PCollection> input = + p.apply(Create.>of(LINES).withCoder(ListCoder.of(StringUtf8Coder.of()))); + + PCollection output = input.apply(Flatten.iterables()); + + DataflowAssert.that(output).containsInAnyOrder(LINES_ARRAY); + + p.run(); + } + + @Test + @Category(RunnableOnService.class) + public void testFlattenIterablesSets() { + Pipeline p = TestPipeline.create(); + + Set linesSet = ImmutableSet.copyOf(LINES); + + PCollection> input = + p.apply(Create.>of(linesSet).withCoder(SetCoder.of(StringUtf8Coder.of()))); + + PCollection output = input.apply(Flatten.iterables()); + + DataflowAssert.that(output).containsInAnyOrder(LINES_ARRAY); + + p.run(); + } + + @Test + @Category(RunnableOnService.class) + public void testFlattenIterablesCollections() { + + Pipeline p = TestPipeline.create(); + + Set linesSet = ImmutableSet.copyOf(LINES); + + PCollection> input = + p.apply(Create.>of(linesSet) + .withCoder(CollectionCoder.of(StringUtf8Coder.of()))); + + PCollection output = input.apply(Flatten.iterables()); + + DataflowAssert.that(output).containsInAnyOrder(LINES_ARRAY); + + p.run(); + } + @Test @Category(RunnableOnService.class) public void testFlattenIterablesEmpty() { @@ -201,7 +258,6 @@ public void testFlattenIterablesEmpty() { p.run(); } - ///////////////////////////////////////////////////////////////////////////// @Test From cc6853579056e15a29681901ca469f388cf42fc3 Mon Sep 17 00:00:00 2001 From: markshields Date: Thu, 19 Nov 2015 12:48:24 -0800 Subject: [PATCH 1183/1541] Stream Iterable state through MergedBag This avoids OOMs on GBK results with many elements. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108275870 --- .../util/state/InMemoryStateInternals.java | 30 +++++++++---------- .../dataflow/sdk/util/state/MergedBag.java | 8 ++--- .../worker/StreamingDataflowWorkerTest.java | 10 +++++-- 3 files changed, 25 insertions(+), 23 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternals.java index da18405be3ddc..a710dd050cb5c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternals.java @@ -31,7 +31,6 @@ * and for running tests that need state. */ public class InMemoryStateInternals extends MergingStateInternals { - private interface InMemoryState { boolean isEmptyForTesting(); } @@ -40,7 +39,6 @@ private interface InMemoryState { @Override protected StateBinder binderForNamespace(final StateNamespace namespace) { return new StateBinder() { - @Override public ValueState bindValue(StateTag> address, Coder coder) { return new InMemoryValue(); @@ -52,11 +50,10 @@ public BagState bindBag(final StateTag> address, Coder ele } @Override - public - CombiningValueStateInternal bindCombiningValue( - StateTag> address, - Coder accumCoder, - final CombineFn combineFn) { + public CombiningValueStateInternal + bindCombiningValue( + StateTag> address, + Coder accumCoder, final CombineFn combineFn) { return new InMemoryCombiningValue(combineFn); } @@ -115,13 +112,12 @@ public void set(T input) { @Override public boolean isEmptyForTesting() { - return isCleared; + return isCleared; } } private final class WatermarkStateInternalImplementation implements WatermarkStateInternal, InMemoryState { - private Instant minimumHold = null; @Override @@ -150,7 +146,7 @@ public void add(Instant watermarkHold) { @Override public boolean isEmptyForTesting() { - return minimumHold == null; + return minimumHold == null; } @Override @@ -171,7 +167,6 @@ public String toString() { private final class InMemoryCombiningValue implements CombiningValueStateInternal, InMemoryState { - private boolean isCleared = true; private final CombineFn combineFn; private AccumT accum; @@ -233,18 +228,23 @@ public void addAccum(AccumT accum) { @Override public boolean isEmptyForTesting() { - return isCleared; + return isCleared; } } private static final class InMemoryBag implements BagState, InMemoryState { - private final List contents = new ArrayList<>(); + private List contents = new ArrayList<>(); @Override public void clear() { // Even though we're clearing we can't remove this from the in-memory state map, since // other users may already have a handle on this Bag. - contents.clear(); + // The result of get/read below must be stable for the lifetime of the bundle within which it + // was generated. In batch and direct runners the bundle lifetime can be + // greater than the window lifetime, in which case this method can be called while + // the result is still in use. We protect against this by hot-swapping instead of + // clearing the contents. + contents = new ArrayList<>(); } @Override @@ -264,7 +264,7 @@ public void add(T input) { @Override public boolean isEmptyForTesting() { - return contents.isEmpty(); + return contents.isEmpty(); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergedBag.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergedBag.java index 76c8c67a15f3e..f39a681347b3e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergedBag.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergedBag.java @@ -27,7 +27,6 @@ * @param the type of elements in the bag */ class MergedBag implements BagState { - private final Collection> sources; private final BagState result; @@ -61,12 +60,11 @@ public StateContents> get() { return new StateContents>() { @Override public Iterable read() { - // Can't use FluentIterables#toList because some values may be legitimately null. - List result = new ArrayList<>(); + List> allIterables = new ArrayList<>(); for (StateContents> future : futures) { - Iterables.addAll(result, future.read()); + allIterables.add(future.read()); } - return result; + return Iterables.concat(allIterables); } }; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java index 5394d8fb0e5eb..938975cd3e405 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java @@ -783,7 +783,11 @@ public void testMergeWindows() throws Exception { ByteString.copyFromUtf8(window + "+shold"); String stateFamily = "MergeWindows"; ByteString bufferData = ByteString.copyFromUtf8("\000data0"); - ByteString outputData = ByteString.copyFromUtf8("\000\000\000\001\005data0"); + // Encoded form for Iterable: -1, true, 'data0', false + ByteString outputData = ByteString.copyFrom( + new byte[] { (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, + 0x01, 0x05, 0x64, 0x61, 0x74, 0x61, 0x30, 0x00 }); + // These values are not essential to the change detector test long timerTimestamp = 999000L; @@ -881,8 +885,8 @@ public void testMergeWindows() throws Exception { actualOutput.getOutputMessages(0).getBundles(0).getKey().toStringUtf8()); assertEquals(0, actualOutput.getOutputMessages(0).getBundles(0).getMessages(0).getTimestamp()); - assertEquals( - outputData, actualOutput.getOutputMessages(0).getBundles(0).getMessages(0).getData()); + assertEquals(outputData, + actualOutput.getOutputMessages(0).getBundles(0).getMessages(0).getData()); ByteString metadata = actualOutput.getOutputMessages(0).getBundles(0).getMessages(0).getMetadata(); From 0f3b8a52cc169840cf92c825da89f9f286f28787 Mon Sep 17 00:00:00 2001 From: tgroh Date: Thu, 19 Nov 2015 13:49:36 -0800 Subject: [PATCH 1184/1541] Condense PipelineOptionsFactory Exceptions This allows runtime failures to all be resolved at once rather than rerunning every time and fixing each failure individually. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108281549 --- .../sdk/options/PipelineOptionsFactory.java | 227 +++++++++++++++--- .../options/PipelineOptionsFactoryTest.java | 188 +++++++++++++++ 2 files changed, 381 insertions(+), 34 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java index c46c59e4177d2..bbdc8d5406808 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java @@ -61,6 +61,7 @@ import java.lang.reflect.Method; import java.lang.reflect.Modifier; import java.lang.reflect.Proxy; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -880,6 +881,7 @@ private static List getPropertyDescriptors(Class beanClas SortedMap propertyNamesToGetters = getPropertyNamesToGetters(methods); List descriptors = Lists.newArrayList(); + List mismatches = new ArrayList<>(); /* * Add all the getter/setter pairs to the list of descriptors removing the getter once * it has been paired up. @@ -898,15 +900,20 @@ private static List getPropertyDescriptors(Class beanClas if (getterMethod != null) { Class getterPropertyType = getterMethod.getReturnType(); Class setterPropertyType = method.getParameterTypes()[0]; - Preconditions.checkArgument(getterPropertyType == setterPropertyType, - "Type mismatch between getter and setter methods for property [%s]. " - + "Getter is of type [%s] whereas setter is of type [%s].", - propertyName, getterPropertyType.getName(), setterPropertyType.getName()); + if (getterPropertyType != setterPropertyType) { + TypeMismatch mismatch = new TypeMismatch(); + mismatch.propertyName = propertyName; + mismatch.getterPropertyType = getterPropertyType; + mismatch.setterPropertyType = setterPropertyType; + mismatches.add(mismatch); + continue; + } } descriptors.add(new PropertyDescriptor( propertyName, getterMethod, method)); } + throwForTypeMismatches(mismatches); // Add the remaining getters with missing setters. for (Map.Entry getterToMethod : propertyNamesToGetters.entrySet()) { @@ -916,6 +923,35 @@ private static List getPropertyDescriptors(Class beanClas return descriptors; } + private static class TypeMismatch { + private String propertyName; + private Class getterPropertyType; + private Class setterPropertyType; + } + + private static void throwForTypeMismatches(List mismatches) { + if (mismatches.size() == 1) { + TypeMismatch mismatch = mismatches.get(0); + throw new IllegalArgumentException(String.format( + "Type mismatch between getter and setter methods for property [%s]. " + + "Getter is of type [%s] whereas setter is of type [%s].", + mismatch.propertyName, + mismatch.getterPropertyType.getName(), + mismatch.setterPropertyType.getName())); + } else if (mismatches.size() > 1) { + StringBuilder builder = new StringBuilder( + String.format("Type mismatches between getters and setters detected:")); + for (TypeMismatch mismatch : mismatches) { + builder.append(String.format( + "%n - Property [%s]: Getter is of type [%s] whereas setter is of type [%s].", + mismatch.propertyName, + mismatch.getterPropertyType.getName(), + mismatch.setterPropertyType.getName())); + } + throw new IllegalArgumentException(builder.toString()); + } + } + /** * Returns a map of the property name to the getter method it represents. * If there are duplicate methods with the same bean name, then it is indeterminate @@ -1006,18 +1042,21 @@ private static List validateClass(Class multipleDefinitions = Lists.newArrayList(); for (Map.Entry> entry : methodNameToMethodMap.asMap().entrySet()) { Set> returnTypes = FluentIterable.from(entry.getValue()) .transform(ReturnTypeFetchingFunction.INSTANCE).toSet(); SortedSet collidingMethods = FluentIterable.from(entry.getValue()) .toSortedSet(MethodComparator.INSTANCE); - Preconditions.checkArgument(returnTypes.size() == 1, - "Method [%s] has multiple definitions %s with different return types for [%s].", - entry.getKey().getName(), - collidingMethods, - iface.getName()); + if (returnTypes.size() > 1) { + MultipleDefinitions defs = new MultipleDefinitions(); + defs.method = entry.getKey(); + defs.collidingMethods = collidingMethods; + multipleDefinitions.add(defs); + } } + throwForMultipleDefinitions(iface, multipleDefinitions); // Verify that there is no getter with a mixed @JsonIgnore annotation and verify // that no setter has @JsonIgnore. @@ -1033,6 +1072,9 @@ private static List validateClass(Class descriptors = getPropertyDescriptors(klass); + List incompletelyIgnoredGetters = new ArrayList<>(); + List ignoredSetters = new ArrayList<>(); + for (PropertyDescriptor descriptor : descriptors) { if (descriptor.getReadMethod() == null || descriptor.getWriteMethod() == null @@ -1050,44 +1092,58 @@ private static List validateClass(Class settersWithJsonIgnore = Sets.filter(methodNameToAllMethodMap.get(descriptor.getWriteMethod()), JsonIgnorePredicate.INSTANCE); Iterable settersWithJsonIgnoreClassNames = FluentIterable.from(settersWithJsonIgnore) - .transform(MethodToDeclaringClassFunction.INSTANCE) - .transform(ReflectHelpers.CLASS_NAME); - - Preconditions.checkArgument(settersWithJsonIgnore.isEmpty(), - "Expected setter for property [%s] to not be marked with @JsonIgnore on %s", - descriptor.getName(), settersWithJsonIgnoreClassNames); + .transform(MethodToDeclaringClassFunction.INSTANCE) + .transform(ReflectHelpers.CLASS_NAME); + + if (!settersWithJsonIgnore.isEmpty()) { + IgnoredSetter ignored = new IgnoredSetter(); + ignored.descriptor = descriptor; + ignored.settersWithJsonIgnoreClassNames = settersWithJsonIgnoreClassNames; + ignoredSetters.add(ignored); + } } + throwForGettersWithInconsistentJsonIgnore(incompletelyIgnoredGetters); + throwForSettersWithJsonIgnore(ignoredSetters); + List missingBeanMethods = new ArrayList<>(); // Verify that each property has a matching read and write method. for (PropertyDescriptor propertyDescriptor : descriptors) { - Preconditions.checkArgument( - IGNORED_METHODS.contains(propertyDescriptor.getWriteMethod()) - || propertyDescriptor.getReadMethod() != null, - "Expected getter for property [%s] of type [%s] on [%s].", - propertyDescriptor.getName(), - propertyDescriptor.getPropertyType().getName(), - iface.getName()); - Preconditions.checkArgument( - IGNORED_METHODS.contains(propertyDescriptor.getReadMethod()) - || propertyDescriptor.getWriteMethod() != null, - "Expected setter for property [%s] of type [%s] on [%s].", - propertyDescriptor.getName(), - propertyDescriptor.getPropertyType().getName(), - iface.getName()); + if (!(IGNORED_METHODS.contains(propertyDescriptor.getWriteMethod()) + || propertyDescriptor.getReadMethod() != null)) { + MissingBeanMethod method = new MissingBeanMethod(); + method.property = propertyDescriptor; + method.methodType = "getter"; + missingBeanMethods.add(method); + continue; + } + if (!(IGNORED_METHODS.contains(propertyDescriptor.getReadMethod()) + || propertyDescriptor.getWriteMethod() != null)) { + MissingBeanMethod method = new MissingBeanMethod(); + method.property = propertyDescriptor; + method.methodType = "setter"; + missingBeanMethods.add(method); + continue; + } methods.add(propertyDescriptor.getReadMethod()); methods.add(propertyDescriptor.getWriteMethod()); } + throwForMissingBeanMethod(iface, missingBeanMethods); // Verify that no additional methods are on an interface that aren't a bean property. SortedSet unknownMethods = new TreeSet<>(MethodComparator.INSTANCE); @@ -1100,6 +1156,109 @@ private static List validateClass(Class collidingMethods; + } + + private static void throwForMultipleDefinitions( + Class iface, List definitions) { + if (definitions.size() == 1) { + MultipleDefinitions errDef = definitions.get(0); + throw new IllegalArgumentException(String.format( + "Method [%s] has multiple definitions %s with different return types for [%s].", + errDef.method.getName(), errDef.collidingMethods, iface.getName())); + } else if (definitions.size() > 1) { + StringBuilder errorBuilder = new StringBuilder(String.format( + "Interface [%s] has Methods with multiple definitions with different return types:", + iface.getName())); + for (MultipleDefinitions errDef : definitions) { + errorBuilder.append(String.format( + "%n - Method [%s] has multiple definitions %s", + errDef.method.getName(), + errDef.collidingMethods)); + } + throw new IllegalArgumentException(errorBuilder.toString()); + } + } + + private static class InconsistentlyIgnoredGetters { + PropertyDescriptor descriptor; + Iterable getterClassNames; + Iterable gettersWithJsonIgnoreClassNames; + } + + private static void throwForGettersWithInconsistentJsonIgnore( + List getters) { + if (getters.size() == 1) { + InconsistentlyIgnoredGetters getter = getters.get(0); + throw new IllegalArgumentException(String.format( + "Expected getter for property [%s] to be marked with @JsonIgnore on all %s, " + + "found only on %s", + getter.descriptor.getName(), getter.getterClassNames, + getter.gettersWithJsonIgnoreClassNames)); + } else if (getters.size() > 1) { + StringBuilder errorBuilder = + new StringBuilder("Property getters are inconsistently marked with @JsonIgnore:"); + for (InconsistentlyIgnoredGetters getter : getters) { + errorBuilder.append( + String.format("%n - Expected for property [%s] to be marked on all %s, " + + "found only on %s", + getter.descriptor.getName(), getter.getterClassNames, + getter.gettersWithJsonIgnoreClassNames)); + } + throw new IllegalArgumentException(errorBuilder.toString()); + } + } + + private static class IgnoredSetter { + PropertyDescriptor descriptor; + Iterable settersWithJsonIgnoreClassNames; + } + + private static void throwForSettersWithJsonIgnore(List setters) { + if (setters.size() == 1) { + IgnoredSetter setter = setters.get(0); + throw new IllegalArgumentException( + String.format("Expected setter for property [%s] to not be marked with @JsonIgnore on %s", + setter.descriptor.getName(), setter.settersWithJsonIgnoreClassNames)); + } else if (setters.size() > 1) { + StringBuilder builder = new StringBuilder("Found setters marked with @JsonIgnore:"); + for (IgnoredSetter setter : setters) { + builder.append( + String.format("%n - Setter for property [%s] should not be marked with @JsonIgnore " + + "on %s", + setter.descriptor.getName(), setter.settersWithJsonIgnoreClassNames)); + } + throw new IllegalArgumentException(builder.toString()); + } + } + + private static class MissingBeanMethod { + String methodType; + PropertyDescriptor property; + } + + private static void throwForMissingBeanMethod( + Class iface, List missingBeanMethods) { + if (missingBeanMethods.size() == 1) { + MissingBeanMethod missingBeanMethod = missingBeanMethods.get(0); + throw new IllegalArgumentException( + String.format("Expected %s for property [%s] of type [%s] on [%s].", + missingBeanMethod.methodType, missingBeanMethod.property.getName(), + missingBeanMethod.property.getPropertyType().getName(), iface.getName())); + } else if (missingBeanMethods.size() > 1) { + StringBuilder builder = new StringBuilder(String.format( + "Found missing property methods on [%s]:", iface.getName())); + for (MissingBeanMethod method : missingBeanMethods) { + builder.append( + String.format("%n - Expected %s for property [%s] of type [%s]", method.methodType, + method.property.getName(), method.property.getPropertyType().getName())); + } + throw new IllegalArgumentException(builder.toString()); + } + } + /** A {@link Comparator} that uses the classes name to compare them. */ private static class ClassNameComparator implements Comparator> { static final ClassNameComparator INSTANCE = new ClassNameComparator(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java index b47ac3337d972..e687f2798946e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryTest.java @@ -37,6 +37,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -130,6 +131,23 @@ public void testMissingGetterThrows() throws Exception { PipelineOptionsFactory.as(MissingGetter.class); } + /** A test interface missing multiple getters. */ + public static interface MissingMultipleGetters extends MissingGetter { + void setOtherObject(Object value); + } + + @Test + public void testMultipleMissingGettersThrows() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage( + "missing property methods on [com.google.cloud.dataflow.sdk.options." + + "PipelineOptionsFactoryTest$MissingMultipleGetters]"); + expectedException.expectMessage("getter for property [object] of type [java.lang.Object]"); + expectedException.expectMessage("getter for property [otherObject] of type [java.lang.Object]"); + + PipelineOptionsFactory.as(MissingMultipleGetters.class); + } + /** A test interface missing a setter. */ public static interface MissingSetter extends PipelineOptions { Object getObject(); @@ -145,6 +163,40 @@ public void testMissingSetterThrows() throws Exception { PipelineOptionsFactory.as(MissingSetter.class); } + /** A test interface missing multiple setters. */ + public static interface MissingMultipleSetters extends MissingSetter { + Object getOtherObject(); + } + + @Test + public void testMissingMultipleSettersThrows() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage( + "missing property methods on [com.google.cloud.dataflow.sdk.options." + + "PipelineOptionsFactoryTest$MissingMultipleSetters]"); + expectedException.expectMessage("setter for property [object] of type [java.lang.Object]"); + expectedException.expectMessage("setter for property [otherObject] of type [java.lang.Object]"); + + PipelineOptionsFactory.as(MissingMultipleSetters.class); + } + + /** A test interface missing a setter and a getter. */ + public static interface MissingGettersAndSetters extends MissingGetter { + Object getOtherObject(); + } + + @Test + public void testMissingGettersAndSettersThrows() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage( + "missing property methods on [com.google.cloud.dataflow.sdk.options." + + "PipelineOptionsFactoryTest$MissingGettersAndSetters]"); + expectedException.expectMessage("getter for property [object] of type [java.lang.Object]"); + expectedException.expectMessage("setter for property [otherObject] of type [java.lang.Object]"); + + PipelineOptionsFactory.as(MissingGettersAndSetters.class); + } + /** A test interface with a type mismatch between the getter and setter. */ public static interface GetterSetterTypeMismatch extends PipelineOptions { boolean getValue(); @@ -161,6 +213,24 @@ public void testGetterSetterTypeMismatchThrows() throws Exception { PipelineOptionsFactory.as(GetterSetterTypeMismatch.class); } + /** A test interface with multiple type mismatches between getters and setters. */ + public static interface MultiGetterSetterTypeMismatch extends GetterSetterTypeMismatch { + long getOther(); + void setOther(String other); + } + + @Test + public void testMultiGetterSetterTypeMismatchThrows() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Type mismatches between getters and setters detected:"); + expectedException.expectMessage("Property [value]: Getter is of type " + + "[boolean] whereas setter is of type [int]."); + expectedException.expectMessage("Property [other]: Getter is of type [long] " + + "whereas setter is of type [java.lang.String]."); + + PipelineOptionsFactory.as(MultiGetterSetterTypeMismatch.class); + } + /** A test interface representing a composite interface. */ public static interface CombinedObject extends MissingGetter, MissingSetter { } @@ -206,6 +276,48 @@ public void testReturnTypeConflictThrows() throws Exception { PipelineOptionsFactory.as(ReturnTypeConflict.class); } + /** An interface to provide multiple methods with return type conflicts. */ + public static interface MultiReturnTypeConflictBase extends CombinedObject { + Object getOther(); + void setOther(Object object); + } + + /** A test interface that has multiple conflicting return types with its parent. */ + public static interface MultiReturnTypeConflict extends MultiReturnTypeConflictBase { + @Override + String getObject(); + void setObject(String value); + + @Override + Long getOther(); + void setOther(Long other); + } + + @Test + public void testMultipleReturnTypeConflictsThrows() throws Exception { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("[com.google.cloud.dataflow.sdk.options." + + "PipelineOptionsFactoryTest$MultiReturnTypeConflict]"); + expectedException.expectMessage( + "Methods with multiple definitions with different return types"); + expectedException.expectMessage("Method [getObject] has multiple definitions"); + expectedException.expectMessage("public abstract java.lang.Object " + + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$" + + "MissingSetter.getObject()"); + expectedException.expectMessage( + "public abstract java.lang.String com.google.cloud.dataflow.sdk.options." + + "PipelineOptionsFactoryTest$MultiReturnTypeConflict.getObject()"); + expectedException.expectMessage("Method [getOther] has multiple definitions"); + expectedException.expectMessage("public abstract java.lang.Object " + + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$" + + "MultiReturnTypeConflictBase.getOther()"); + expectedException.expectMessage( + "public abstract java.lang.Long com.google.cloud.dataflow.sdk.options." + + "PipelineOptionsFactoryTest$MultiReturnTypeConflict.getOther()"); + + PipelineOptionsFactory.as(MultiReturnTypeConflict.class); + } + /** Test interface that has {@link JsonIgnore @JsonIgnore} on a setter for a property. */ public static interface SetterWithJsonIgnore extends PipelineOptions { String getValue(); @@ -222,6 +334,27 @@ public void testSetterAnnotatedWithJsonIgnore() throws Exception { PipelineOptionsFactory.as(SetterWithJsonIgnore.class); } + /** Test interface that has {@link JsonIgnore @JsonIgnore} on multiple setters. */ + public static interface MultiSetterWithJsonIgnore extends SetterWithJsonIgnore { + Integer getOther(); + @JsonIgnore + void setOther(Integer other); + } + + @Test + public void testMultipleSettersAnnotatedWithJsonIgnore() throws Exception { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Found setters marked with @JsonIgnore:"); + expectedException.expectMessage( + "property [other] should not be marked with @JsonIgnore on [com" + + ".google.cloud.dataflow.sdk.options." + + "PipelineOptionsFactoryTest$MultiSetterWithJsonIgnore]"); + expectedException.expectMessage( + "property [value] should not be marked with @JsonIgnore on [com." + + "google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$SetterWithJsonIgnore]"); + PipelineOptionsFactory.as(MultiSetterWithJsonIgnore.class); + } + /** * This class is has a conflicting field with {@link CombinedObject} that doesn't have * {@link JsonIgnore @JsonIgnore}. @@ -249,6 +382,61 @@ public void testNotAllGettersAnnotatedWithJsonIgnore() throws Exception { options.as(CombinedObject.class); } + private static interface MultiGetters extends PipelineOptions { + Object getObject(); + void setObject(Object value); + + @JsonIgnore + Integer getOther(); + void setOther(Integer value); + + Void getConsistent(); + void setConsistent(Void consistent); + } + + private static interface MultipleGettersWithInconsistentJsonIgnore extends PipelineOptions { + @JsonIgnore + Object getObject(); + void setObject(Object value); + + Integer getOther(); + void setOther(Integer value); + + Void getConsistent(); + void setConsistent(Void consistent); + } + + @Test + public void testMultipleGettersWithInconsistentJsonIgnore() { + // Initial construction is valid. + MultiGetters options = PipelineOptionsFactory.as(MultiGetters.class); + + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Property getters are inconsistently marked with @JsonIgnore:"); + expectedException.expectMessage( + "property [object] to be marked on all"); + expectedException.expectMessage("found only on [com.google.cloud.dataflow.sdk.options." + + "PipelineOptionsFactoryTest$MultiGetters]"); + expectedException.expectMessage( + "property [other] to be marked on all"); + expectedException.expectMessage("found only on [com.google.cloud.dataflow.sdk.options." + + "PipelineOptionsFactoryTest$MultipleGettersWithInconsistentJsonIgnore]"); + + expectedException.expectMessage(Matchers.anyOf( + containsString(java.util.Arrays.toString(new String[] + {"com.google.cloud.dataflow.sdk.options." + + "PipelineOptionsFactoryTest$MultipleGettersWithInconsistentJsonIgnore", + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$MultiGetters"})), + containsString(java.util.Arrays.toString(new String[] + {"com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryTest$MultiGetters", + "com.google.cloud.dataflow.sdk.options." + + "PipelineOptionsFactoryTest$MultipleGettersWithInconsistentJsonIgnore"})))); + expectedException.expectMessage(not(containsString("property [consistent]"))); + + // When we attempt to convert, we should error immediately + options.as(MultipleGettersWithInconsistentJsonIgnore.class); + } + @Test public void testAppNameIsNotOverriddenWhenPassedInViaCommandLine() { ApplicationNameOptions options = PipelineOptionsFactory From 70918753f6bad8a2a64b0231a427abb3411dece2 Mon Sep 17 00:00:00 2001 From: dhalperi Date: Fri, 20 Nov 2015 01:31:27 -0800 Subject: [PATCH 1185/1541] TextIO: improve javadoc ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108325447 --- .../google/cloud/dataflow/sdk/io/TextIO.java | 264 ++++++++++-------- 1 file changed, 144 insertions(+), 120 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java index 11047e4a1ed2f..875f84b664451 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java @@ -19,6 +19,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.runners.worker.TextReader; import com.google.cloud.dataflow.sdk.runners.worker.TextSink; @@ -48,22 +49,27 @@ /** * {@link PTransform}s for reading and writing text files. * - *

To read a {@link PCollection} from one or more text files, use - * {@link TextIO.Read}, specifying {@link TextIO.Read#from} to specify + *

To read a {@link PCollection} from one or more text files, use {@link TextIO.Read}. + * You can instantiate a transform using {@link TextIO.Read#from(String)} to specify * the path of the file(s) to read from (e.g., a local filename or * filename pattern if running locally, or a Google Cloud Storage * filename or filename pattern of the form - * {@code "gs:///"}), and optionally - * {@link TextIO.Read#named} to specify the name of the pipeline step - * and/or {@link TextIO.Read#withCoder} to specify the Coder to use to - * decode the text lines into Java values. For example: + * {@code "gs:///"}). You may optionally call + * {@link TextIO.Read#named(String)} to specify the name of the pipeline step. + * + *

By default, {@link TextIO.Read} returns a {@link PCollection} of {@link String Strings}, + * each corresponding to one line of an input UTF-8 text file. To convert directly from the raw + * bytes (split into lines delimited by '\n', '\r', or '\r\n') to another object of type {@code T}, + * supply a {@code Coder} using {@link TextIO.Read#withCoder(Coder)}. + * + *

See the following examples: * *

 {@code
  * Pipeline p = ...;
  *
  * // A simple Read of a local file (only runs locally):
  * PCollection lines =
- *     p.apply(TextIO.Read.from("/path/to/file.txt"));
+ *     p.apply(TextIO.Read.from("/local/path/to/file.txt"));
  *
  * // A fully-specified Read from a GCS file (runs locally and via the
  * // Google Cloud Dataflow service):
@@ -74,14 +80,13 @@
  * } 
* *

To write a {@link PCollection} to one or more text files, use - * {@link TextIO.Write}, specifying {@link TextIO.Write#to} to specify + * {@link TextIO.Write}, specifying {@link TextIO.Write#to(String)} to specify * the path of the file to write to (e.g., a local filename or sharded * filename pattern if running locally, or a Google Cloud Storage * filename or sharded filename pattern of the form - * {@code "gs:///"}), and optionally - * {@link TextIO.Write#named} to specify the name of the pipeline step - * and/or {@link TextIO.Write#withCoder} to specify the Coder to use - * to encode the Java values into text lines. + * {@code "gs:///"}). You can optionally name the resulting transform using + * {@link TextIO.Write#named(String)}, and you can use {@link TextIO.Write#withCoder(Coder)} + * to specify the Coder to use to encode the Java values into text lines. * *

Any existing files with the same names as generated output files * will be overwritten. @@ -101,37 +106,40 @@ * .withCoder(TextualIntegerCoder.of())); * } * - *

Permissions

- * Permission requirements depend on the - * {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner PipelineRunner} that is - * used to execute the Dataflow job. Please refer to the documentation of corresponding - * {@code PipelineRunner}s for more details. + *

Permissions

+ *

When run using the {@link DirectPipelineRunner}, your pipeline can read and write text files + * on your local drive and remote text files on Google Cloud Storage that you have access to using + * your {@code gcloud} credentials. When running in the Dataflow service using + * {@link DataflowPipelineRunner}, the pipeline can only read and write files from GCS. For more + * information about permissions, see the Cloud Dataflow documentation on + * Security and + * Permissions. */ public class TextIO { + /** The default coder, which returns each line of the input file as a string. */ public static final Coder DEFAULT_TEXT_CODER = StringUtf8Coder.of(); /** * A {@link PTransform} that reads from a text file (or multiple text * files matching a pattern) and returns a {@link PCollection} containing - * the decoding of each of the lines of the text file(s). The - * default decoding just returns the lines. + * the decoding of each of the lines of the text file(s). The + * default decoding just returns each line as a {@link String}, but you may call + * {@link #withCoder(Coder)} to change the return type. */ public static class Read { /** - * Returns a {@link TextIO.Read} {@link PTransform} with the given step name. + * Returns a transform for reading text files that uses the given step name. */ public static Bound named(String name) { return new Bound<>(DEFAULT_TEXT_CODER).named(name); } /** - * Returns a {@link TextIO.Read} {@link PTransform} that reads from the file(s) - * with the given name or pattern. This can be a local filename - * or filename pattern (if running locally), or a Google Cloud - * Storage filename or filename pattern of the form - * {@code "gs:///"}) (if running locally or via - * the Google Cloud Dataflow service). Standard - * /"} (if running locally or via the Google Cloud Dataflow + * service). Standard Java Filesystem glob patterns ("*", "?", "[..]") are supported. */ public static Bound from(String filepattern) { @@ -139,7 +147,7 @@ public static Bound from(String filepattern) { } /** - * Returns a TextIO.Read PTransform that uses the given + * Returns a transform for reading text files that uses the given * {@code Coder} to decode each of the lines of the file into a * value of type {@code T}. * @@ -154,7 +162,7 @@ public static Bound withCoder(Coder coder) { } /** - * Returns a TextIO.Read PTransform that has GCS path validation on + * Returns a transform for reading text files that has GCS path validation on * pipeline creation disabled. * *

This can be useful in the case where the GCS input does not @@ -166,12 +174,12 @@ public static Bound withoutValidation() { } /** - * Returns a TextIO.Read PTransform that reads from a file with the - * specified compression type. + * Returns a transform for reading text files that decompresses all input files + * using the specified compression type. * - *

If no compression type is specified, the default is AUTO. In this - * mode, the compression type of the file is determined by its extension - * (e.g., *.gz is gzipped, *.bz2 is bzipped, all other extensions are + *

If no compression type is specified, the default is {@link TextIO.CompressionType#AUTO}. + * In this mode, the compression type of the file is determined by its extension + * (e.g., {@code *.gz} is gzipped, {@code *.bz2} is bzipped, and all other extensions are * uncompressed). */ public static Bound withCompressionType(TextIO.CompressionType compressionType) { @@ -181,34 +189,32 @@ public static Bound withCompressionType(TextIO.CompressionType compressi // TODO: strippingNewlines, etc. /** - * A {@link PTransform} that reads from a text file (or multiple text files - * matching a pattern) and returns a bounded PCollection containing the - * decoding of each of the lines of the text file(s). The default - * decoding just returns the lines. + * A {@link PTransform} that reads from one or more text files and returns a bounded + * {@link PCollection} containing one element for each line of the input files. * * @param the type of each of the elements of the resulting - * PCollection, decoded from the lines of the text file + * {@link PCollection}. By default, each line is returned as a {@link String}, however you + * may use {@link #withCoder(Coder)} to supply a {@code Coder} to produce a + * {@code PCollection} instead. */ public static class Bound extends PTransform> { /** The filepattern to read from. */ - @Nullable - final String filepattern; + @Nullable private final String filepattern; /** The Coder to use to decode each line. */ - @Nullable - final Coder coder; + @Nullable private final Coder coder; /** An option to indicate if input validation is desired. Default is true. */ - final boolean validate; + private final boolean validate; /** Option to indicate the input source's compression type. Default is AUTO. */ - final TextIO.CompressionType compressionType; + private final TextIO.CompressionType compressionType; Bound(Coder coder) { this(null, null, coder, true, TextIO.CompressionType.AUTO); } - Bound(String name, String filepattern, Coder coder, boolean validate, + private Bound(String name, String filepattern, Coder coder, boolean validate, TextIO.CompressionType compressionType) { super(name); this.coder = coder; @@ -218,28 +224,33 @@ public static class Bound extends PTransform> { } /** - * Returns a new TextIO.Read PTransform that's like this one but - * with the given step name. Does not modify this object. + * Returns a new transform for reading from text files that's like this one but + * with the given step name. + * + *

Does not modify this object. */ public Bound named(String name) { return new Bound<>(name, filepattern, coder, validate, compressionType); } /** - * Returns a new TextIO.Read PTransform that's like this one but - * that reads from the file(s) with the given name or pattern. - * (See {@link TextIO.Read#from} for a description of - * filepatterns.) Does not modify this object. + * Returns a new transform for reading from text files that's like this one but + * that reads from the file(s) with the given name or pattern. See {@link TextIO.Read#from} + * for a description of filepatterns. + * + *

Does not modify this object. + */ public Bound from(String filepattern) { return new Bound<>(name, filepattern, coder, validate, compressionType); } /** - * Returns a new TextIO.Read PTransform that's like this one but - * that uses the given {@code Coder} to decode each of the - * lines of the file into a value of type {@code X}. Does not - * modify this object. + * Returns a new transform for reading from text files that's like this one but + * that uses the given {@link Coder Coder} to decode each of the + * lines of the file into a value of type {@code X}. + * + *

Does not modify this object. * * @param the type of the decoded elements, and the * elements of the resulting PCollection @@ -249,30 +260,27 @@ public Bound withCoder(Coder coder) { } /** - * Returns a new TextIO.Read PTransform that's like this one but + * Returns a new transform for reading from text files that's like this one but * that has GCS path validation on pipeline creation disabled. - * Does not modify this object. * *

This can be useful in the case where the GCS input does not * exist at the pipeline creation time, but is expected to be * available at execution time. + * + *

Does not modify this object. */ public Bound withoutValidation() { return new Bound<>(name, filepattern, coder, false, compressionType); } /** - * Returns a new TextIO.Read PTransform that's like this one but + * Returns a new transform for reading from text files that's like this one but * reads from input sources using the specified compression type. - * Does not modify this object. * - *

If AUTO compression type is specified, a compression type is - * selected on a per-file basis, based on the file's extension (e.g., - * .gz will be processed as a gzipped file, .bz will be processed - * as a bzipped file, other extensions with be treated as uncompressed - * input). + *

If no compression type is specified, the default is {@link TextIO.CompressionType#AUTO}. + * See {@link TextIO.Read#withCompressionType(CompressionType)} for more details. * - *

If no compression type is specified, the default is AUTO. + *

Does not modify this object. */ public Bound withCompressionType(TextIO.CompressionType compressionType) { return new Bound<>(name, filepattern, coder, validate, compressionType); @@ -321,52 +329,55 @@ public void evaluate( }); } } + + /** Disallow construction of utility classes. */ + private Read() {} } ///////////////////////////////////////////////////////////////////////////// /** - * A {@link PTransform} that writes a {@link PCollection} to a text file (or + * A {@link PTransform} that writes a {@link PCollection} to text file (or * multiple text files matching a sharding pattern), with each - * PCollection element being encoded into its own line. + * element of the input collection encoded into its own line. */ public static class Write { /** - * Returns a TextIO.Write PTransform with the given step name. + * Returns a transform for writing to text files with the given step name. */ public static Bound named(String name) { return new Bound<>(DEFAULT_TEXT_CODER).named(name); } /** - * Returns a TextIO.Write PTransform that writes to the file(s) - * with the given prefix. This can be a local filename + * Returns a transform for writing to text files that writes to the file(s) + * with the given prefix. This can be a local filename * (if running locally), or a Google Cloud Storage filename of - * the form {@code "gs:///"}) + * the form {@code "gs:///"} * (if running locally or via the Google Cloud Dataflow service). * *

The files written will begin with this prefix, followed by - * a shard identifier (see {@link Bound#withNumShards}, and end - * in a common extension, if given by {@link Bound#withSuffix}. + * a shard identifier (see {@link Bound#withNumShards(int)}, and end + * in a common extension, if given by {@link Bound#withSuffix(String)}. */ public static Bound to(String prefix) { return new Bound<>(DEFAULT_TEXT_CODER).to(prefix); } /** - * Returns a TextIO.Write PTransform that writes to the file(s) with the - * given filename suffix. + * Returns a transform for writing to text files that appends the specified suffix + * to the created files. */ public static Bound withSuffix(String nameExtension) { return new Bound<>(DEFAULT_TEXT_CODER).withSuffix(nameExtension); } /** - * Returns a TextIO.Write PTransform that uses the provided shard count. + * Returns a transform for writing to text files that uses the provided shard count. * *

Constraining the number of shards is likely to reduce - * the performance of a pipeline. Setting this value is not recommended + * the performance of a pipeline. Setting this value is not recommended * unless you require a specific number of output files. * * @param numShards the number of shards to use, or 0 to let the system @@ -377,7 +388,7 @@ public static Bound withNumShards(int numShards) { } /** - * Returns a TextIO.Write PTransform that uses the given shard name + * Returns a transform for writing to text files that uses the given shard name * template. * *

See {@link ShardNameTemplate} for a description of shard templates. @@ -387,7 +398,7 @@ public static Bound withShardNameTemplate(String shardTemplate) { } /** - * Returns a TextIO.Write PTransform that forces a single file as + * Returns a transform for writing to text files that forces a single file as * output. */ public static Bound withoutSharding() { @@ -395,21 +406,21 @@ public static Bound withoutSharding() { } /** - * Returns a TextIO.Write PTransform that uses the given - * {@code Coder} to encode each of the elements of the input - * {@code PCollection} into an output text line. + * Returns a transform for writing to text files that uses the given + * {@link Coder} to encode each of the elements of the input + * {@link PCollection} into an output text line. * *

By default, uses {@link StringUtf8Coder}, which writes input * Java strings directly as output lines. * - * @param the type of the elements of the input PCollection + * @param the type of the elements of the input {@link PCollection} */ public static Bound withCoder(Coder coder) { return new Bound<>(coder); } /** - * Returns a TextIO.Write PTransform that has GCS path validation on + * Returns a transform for writing to text files that has GCS path validation on * pipeline creation disabled. * *

This can be useful in the case where the GCS output location does @@ -430,32 +441,31 @@ public static Bound withoutValidation() { * @param the type of the elements of the input PCollection */ public static class Bound extends PTransform, PDone> { - /** The filename to write to. */ - @Nullable - final String filenamePrefix; - /** Suffix to use for each filename. */ - final String filenameSuffix; + /** The prefix of each file written, combined with suffix and shardTemplate. */ + @Nullable private final String filenamePrefix; + /** The suffix of each file written, combined with prefix and shardTemplate. */ + private final String filenameSuffix; /** The Coder to use to decode each line. */ - final Coder coder; + private final Coder coder; - /** Requested number of shards. 0 for automatic. */ - final int numShards; + /** Requested number of shards. 0 for automatic. */ + private final int numShards; /** Insert a shuffle before writing to decouple parallelism when numShards != 0. */ - final boolean forceReshard; + private final boolean forceReshard; - /** Shard template string. */ - final String shardTemplate; + /** The shard template of each file written, combined with prefix and suffix. */ + private final String shardTemplate; /** An option to indicate if output validation is desired. Default is true. */ - final boolean validate; + private final boolean validate; Bound(Coder coder) { this(null, null, "", coder, 0, true, ShardNameTemplate.INDEX_OF_MAX, true); } - Bound(String name, String filenamePrefix, String filenameSuffix, Coder coder, + private Bound(String name, String filenamePrefix, String filenameSuffix, Coder coder, int numShards, boolean forceReshard, String shardTemplate, boolean validate) { super(name); this.coder = coder; @@ -468,8 +478,10 @@ public static class Bound extends PTransform, PDone> { } /** - * Returns a new TextIO.Write PTransform that's like this one but - * with the given step name. Does not modify this object. + * Returns a transform for writing to text files that's like this one but + * with the given step name. + * + *

Does not modify this object. */ public Bound named(String name) { return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, @@ -477,7 +489,7 @@ public Bound named(String name) { } /** - * Returns a new TextIO.Write PTransform that's like this one but + * Returns a transform for writing to text files that's like this one but * that writes to the file(s) with the given filename prefix. * *

See {@link TextIO.Write#to(String) Write.to(String)} for more information. @@ -491,7 +503,7 @@ public Bound to(String filenamePrefix) { } /** - * Returns a new TextIO.Write PTransform that's like this one but + * Returns a transform for writing to text files that that's like this one but * that writes to the file(s) with the given filename suffix. * *

Does not modify this object. @@ -505,11 +517,11 @@ public Bound withSuffix(String nameExtension) { } /** - * Returns a new TextIO.Write PTransform that's like this one but + * Returns a transform for writing to text files that's like this one but * that uses the provided shard count. * *

Constraining the number of shards is likely to reduce - * the performance of a pipeline. Setting this value is not recommended + * the performance of a pipeline. Setting this value is not recommended * unless you require a specific number of output files. * *

Does not modify this object. @@ -523,14 +535,14 @@ public Bound withNumShards(int numShards) { } /** - * Returns a new TextIO.Write PTransform that's like this one but + * Returns a transform for writing to text files that's like this one but * that uses the provided shard count. * *

Constraining the number of shards is likely to reduce - * the performance of a pipeline. If forceReshard is true, the output - * will be shuffled to obtain the desired sharding. If it is false, + * the performance of a pipeline. If forceReshard is true, the output + * will be shuffled to obtain the desired sharding. If it is false, * data will not be reshuffled, but parallelism of preceeding stages - * may be constrained. Setting this value is not recommended + * may be constrained. Setting this value is not recommended * unless you require a specific number of output files. * *

Does not modify this object. @@ -547,7 +559,7 @@ private Bound withNumShards(int numShards, boolean forceReshard) { } /** - * Returns a new TextIO.Write PTransform that's like this one but + * Returns a transform for writing to text files that's like this one but * that uses the given shard name template. * *

Does not modify this object. @@ -560,9 +572,13 @@ public Bound withShardNameTemplate(String shardTemplate) { } /** - * Returns a new TextIO.Write PTransform that's like this one but + * Returns a transform for writing to text files that's like this one but * that forces a single file as output. * + *

Constraining the number of shards is likely to reduce + * the performance of a pipeline. Using this setting is not recommended + * unless you truly require a single output file. + * *

This is a shortcut for * {@code .withNumShards(1).withShardNameTemplate("")} * @@ -573,9 +589,13 @@ public Bound withoutSharding() { } /** - * Returns a new TextIO.Write PTransform that's like this one but + * Returns a transform for writing to text files that's like this one but * that forces a single file as output. * + *

Constraining the number of shards is likely to reduce + * the performance of a pipeline. Using this setting is not recommended + * unless you truly require a single output file. + * *

This is a shortcut for * {@code .withNumShards(1, forceReshard).withShardNameTemplate("")} * @@ -587,12 +607,12 @@ private Bound withoutSharding(boolean forceReshard) { } /** - * Returns a new TextIO.Write PTransform that's like this one - * but that uses the given {@code Coder} to encode each of - * the elements of the input {@code PCollection} into an - * output text line. Does not modify this object. + * Returns a transform for writing to text files that's like this one + * but that uses the given {@link Coder Coder} to encode each of + * the elements of the input {@link PCollection PCollection} into an + * output text line. Does not modify this object. * - * @param the type of the elements of the input PCollection + * @param the type of the elements of the input {@link PCollection} */ public Bound withCoder(Coder coder) { return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, forceReshard, @@ -600,13 +620,14 @@ public Bound withCoder(Coder coder) { } /** - * Returns a new TextIO.Write PTransform that's like this one but + * Returns a transform for writing to text files that's like this one but * that has GCS output path validation on pipeline creation disabled. - * Does not modify this object. * *

This can be useful in the case where the GCS output location does * not exist at the pipeline creation time, but is expected to be * available at execution time. + * + *

Does not modify this object. */ public Bound withoutValidation() { return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, forceReshard, @@ -696,7 +717,7 @@ public static enum CompressionType implements TextReader.DecompressingStreamFact GZIP(".gz") { @Override public InputStream createInputStream(InputStream inputStream) throws IOException { - // Determine if the input stream is gzipped. The input stream returned from the + // Determine if the input stream is gzipped. The input stream returned from the // GCS connector may already be decompressed, and no action is required. PushbackInputStream stream = new PushbackInputStream(inputStream, 2); byte[] headerBytes = new byte[2]; @@ -753,6 +774,9 @@ private static void validateOutputComponent(String partialFilePattern) { ////////////////////////////////////////////////////////////////////////////// + /** Disable construction of utility class. */ + private TextIO() {} + private static void evaluateReadHelper( Read.Bound transform, DirectPipelineRunner.EvaluationContext context) { TextReader reader = @@ -767,7 +791,7 @@ private static void evaluateWriteHelper( List elems = context.getPCollection(context.getInput(transform)); int numShards = transform.numShards; if (numShards < 1) { - // System gets to choose. For direct mode, choose 1. + // System gets to choose. For direct mode, choose 1. numShards = 1; } TextSink> writer = TextSink.createForDirectPipelineRunner( From c7f0421c8a345de72137ff2006a7e5fbf128bf85 Mon Sep 17 00:00:00 2001 From: lcwik Date: Fri, 20 Nov 2015 09:49:18 -0800 Subject: [PATCH 1186/1541] Make the IterableLikeCoder efficient for many small values Use a small buffer to count the number of elements that are buffered before prefixing the count. For small values like ints, this reduces the overhead from 1 byte per value, to a few bytes per 64k of data written. ----Release Notes---- Modified IterableLikeCoder in a backwards compatible manner but not in a forwards compatible manner preventing users from "updating" a pipeline to a previous minor release. [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108352267 --- .../sdk/coders/IterableLikeCoder.java | 46 ++-- .../BufferedElementCountingOutputStream.java | 184 ++++++++++++++++ ...fferedElementCountingOutputStreamTest.java | 205 ++++++++++++++++++ 3 files changed, 421 insertions(+), 14 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferedElementCountingOutputStream.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BufferedElementCountingOutputStreamTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java index 3adcb6d9a9008..522e9b1ee74b2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java @@ -16,6 +16,8 @@ package com.google.cloud.dataflow.sdk.coders; +import com.google.cloud.dataflow.sdk.util.BufferedElementCountingOutputStream; +import com.google.cloud.dataflow.sdk.util.VarInt; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObservableIterable; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; import com.google.common.base.Preconditions; @@ -111,15 +113,17 @@ public void encode( elementCoder.encode(elem, dataOutStream, nestedContext); } } else { - // We don't know the size without traversing it. So use a - // "hasNext" sentinel before each element. - // TODO: Don't use the sentinel if context.isWholeStream. + // We don't know the size without traversing it so use a fixed size buffer + // and encode as many elements as possible into it before outputting the size followed + // by the elements. dataOutStream.writeInt(-1); + BufferedElementCountingOutputStream countingOutputStream = + new BufferedElementCountingOutputStream(dataOutStream); for (T elem : iterable) { - dataOutStream.writeBoolean(true); - elementCoder.encode(elem, dataOutStream, nestedContext); + countingOutputStream.markElementStart(); + elementCoder.encode(elem, countingOutputStream, nestedContext); } - dataOutStream.writeBoolean(false); + countingOutputStream.finish(); } // Make sure all our output gets pushed to the underlying outStream. dataOutStream.flush(); @@ -138,11 +142,15 @@ public IterableT decode(InputStream inStream, Context context) } return decodeToIterable(elements); } else { - // We don't know the size a priori. Check if we're done with - // each element. List elements = new ArrayList<>(); - while (dataInStream.readBoolean()) { - elements.add(elementCoder.decode(dataInStream, nestedContext)); + long count; + // We don't know the size a priori. Check if we're done with + // each block of elements. + while ((count = VarInt.decodeLong(dataInStream)) > 0) { + while (count > 0) { + elements.add(elementCoder.decode(dataInStream, nestedContext)); + count -= 1; + } } return decodeToIterable(elements); } @@ -205,14 +213,24 @@ public void registerByteSizeObserver( elementCoder.registerByteSizeObserver(elem, observer, nestedContext); } } else { - // We don't know the size without traversing it. So use a - // "hasNext" sentinel before each element. - // TODO: Don't use the sentinel if context.isWholeStream. + // TODO: Update to use an accurate count depending on size and count, currently we + // are under estimating the size by up to 10 bytes per block of data since we are + // not encoding the count prefix which occurs at most once per 64k of data and is upto + // 10 bytes long. Since we include the total count we can upper bound the underestimate + // to be 10 / 65536 ~= 0.0153% of the actual size. observer.update(4L); + long count = 0; for (T elem : iterable) { - observer.update(1L); + count += 1; elementCoder.registerByteSizeObserver(elem, observer, nestedContext); } + if (count > 0) { + // Update the length based upon the number of counted elements, this helps + // eliminate the case where all the elements are encoded in the first block and + // it is quite short (e.g. Long.MAX_VALUE nulls encoded with VoidCoder). + observer.update(VarInt.getLength(count)); + } + // Update with the terminator byte. observer.update(1L); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferedElementCountingOutputStream.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferedElementCountingOutputStream.java new file mode 100644 index 0000000000000..e8e693a996c19 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BufferedElementCountingOutputStream.java @@ -0,0 +1,184 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.coders.Coder.Context; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import javax.annotation.concurrent.NotThreadSafe; + +/** + * Provides an efficient encoding for {@link Iterable}s containing small values by + * buffering up to {@code bufferSize} bytes of data before prefixing the count. + * Note that each element needs to be encoded in a nested context. See + * {@link Context Coder.Context} for more details. + * + *

To use this stream: + *


+ * BufferedElementCountingOutputStream os = ...
+ * for (Element E : elements) {
+ *   os.markElementStart();
+ *   // write an element to os
+ * }
+ * os.finish();
+ * 
+ * + *

The resulting output stream is: + *

+ * countA element(0) element(1) ... element(countA - 1)
+ * countB element(0) element(1) ... element(countB - 1)
+ * ...
+ * countX element(0) element(1) ... element(countX - 1)
+ * countY
+ * 
+ * + *

To read this stream: + *


+ * InputStream is = ...
+ * long count;
+ * do {
+ *   count = VarInt.decodeLong(is);
+ *   for (int i = 0; i < count; ++i) {
+ *     // read an element from is
+ *   }
+ * } while(count > 0);
+ * 
+ * + *

The counts are encoded as variable length longs. See {@link VarInt#encode(long, OutputStream)} + * for more details. The end of the iterable is detected by reading a count of 0. + */ +@NotThreadSafe +public class BufferedElementCountingOutputStream extends OutputStream { + public static final int DEFAULT_BUFFER_SIZE = 64 * 1024; + private final ByteBuffer buffer; + private final OutputStream os; + private boolean finished; + private long count; + + /** + * Creates an output stream which encodes the number of elements output to it in a streaming + * manner. + */ + public BufferedElementCountingOutputStream(OutputStream os) { + this(os, DEFAULT_BUFFER_SIZE); + } + + /** + * Creates an output stream which encodes the number of elements output to it in a streaming + * manner with the given {@code bufferSize}. + */ + BufferedElementCountingOutputStream(OutputStream os, int bufferSize) { + this.buffer = ByteBuffer.allocate(bufferSize); + this.os = os; + this.finished = false; + this.count = 0; + } + + /** + * Finishes the encoding by flushing any buffered data, + * and outputting a final count of 0. + */ + public void finish() throws IOException { + if (finished) { + return; + } + flush(); + // Finish the stream by stating that there are 0 elements that follow. + VarInt.encode(0, os); + finished = true; + } + + /** + * Marks that a new element is being output. This allows this output stream + * to use the buffer if it had previously overflowed marking the start of a new + * block of elements. + */ + public void markElementStart() throws IOException { + if (finished) { + throw new IOException("Stream has been finished. Can not add any more elements."); + } + count++; + } + + @Override + public void write(int b) throws IOException { + if (finished) { + throw new IOException("Stream has been finished. Can not write any more data."); + } + if (count == 0) { + os.write(b); + return; + } + + if (buffer.hasRemaining()) { + buffer.put((byte) b); + } else { + outputBuffer(); + os.write(b); + } + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + if (finished) { + throw new IOException("Stream has been finished. Can not write any more data."); + } + if (count == 0) { + os.write(b, off, len); + return; + } + + if (buffer.remaining() >= len) { + buffer.put(b, off, len); + } else { + outputBuffer(); + os.write(b, off, len); + } + } + + @Override + public void flush() throws IOException { + if (finished) { + return; + } + outputBuffer(); + os.flush(); + } + + @Override + public void close() throws IOException { + finish(); + os.close(); + } + + // Output the buffer if it contains any data. + private void outputBuffer() throws IOException { + if (count > 0) { + VarInt.encode(count, os); + // We are using a heap based buffer and not a direct buffer so it is safe to access + // the underlying array. + os.write(buffer.array(), buffer.arrayOffset(), buffer.position()); + buffer.clear(); + // The buffer has been flushed so we must write to the underlying stream until + // we learn of the next element. We reset the count to zero marking that we should + // not use the buffer. + count = 0; + } + } +} + diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BufferedElementCountingOutputStreamTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BufferedElementCountingOutputStreamTest.java new file mode 100644 index 0000000000000..af2f4425507f4 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BufferedElementCountingOutputStreamTest.java @@ -0,0 +1,205 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder; +import com.google.cloud.dataflow.sdk.coders.Coder.Context; +import com.google.common.collect.ImmutableList; + +import org.hamcrest.collection.IsIterableContainingInOrder; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +/** + * Tests for {@link BufferedElementCountingOutputStream}. + */ +@RunWith(JUnit4.class) +public class BufferedElementCountingOutputStreamTest { + @Rule public final ExpectedException expectedException = ExpectedException.none(); + private static final int BUFFER_SIZE = 8; + + @Test + public void testEmptyValues() throws Exception { + testValues(Collections.emptyList()); + } + + @Test + public void testSingleValue() throws Exception { + testValues(toBytes("abc")); + } + + @Test + public void testSingleValueGreaterThanBuffer() throws Exception { + testValues(toBytes("abcdefghijklmnopqrstuvwxyz")); + } + + @Test + public void testMultipleValuesLessThanBuffer() throws Exception { + testValues(toBytes("a", "b", "c")); + } + + @Test + public void testMultipleValuesThatBecomeGreaterThanBuffer() throws Exception { + testValues(toBytes("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", + "m", "n", "o", "p", "q", "r", "s", "t", "u", "v", "w", "x", "y", "z")); + } + + @Test + public void testMultipleRandomSizedValues() throws Exception { + Random r = new Random(234589023580234890L); + byte[] randomData = new byte[r.nextInt(18)]; + for (int i = 0; i < 1000; ++i) { + List bytes = new ArrayList<>(); + for (int j = 0; j < 100; ++j) { + r.nextBytes(randomData); + bytes.add(randomData); + } + testValues(bytes); + } + } + + @Test + public void testFlushInMiddleOfElement() throws Exception { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + BufferedElementCountingOutputStream os = new BufferedElementCountingOutputStream(bos); + os.markElementStart(); + os.write(1); + os.flush(); + os.write(2); + os.close(); + assertArrayEquals(new byte[]{ 1, 1, 2, 0 }, bos.toByteArray()); + } + + @Test + public void testFlushInMiddleOfElementUsingByteArrays() throws Exception { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + BufferedElementCountingOutputStream os = new BufferedElementCountingOutputStream(bos); + os.markElementStart(); + os.write(new byte[]{ 1 }); + os.flush(); + os.write(new byte[]{ 2 }); + os.close(); + assertArrayEquals(new byte[]{ 1, 1, 2, 0 }, bos.toByteArray()); + } + + @Test + public void testFlushingWhenFinishedIsNoOp() throws Exception { + BufferedElementCountingOutputStream os = testValues(toBytes("a")); + os.flush(); + os.flush(); + os.flush(); + } + + @Test + public void testFinishingWhenFinishedIsNoOp() throws Exception { + BufferedElementCountingOutputStream os = testValues(toBytes("a")); + os.finish(); + os.finish(); + os.finish(); + } + + @Test + public void testClosingFinishesTheStream() throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + BufferedElementCountingOutputStream os = createAndWriteValues(toBytes("abcdefghij"), baos); + os.close(); + verifyValues(toBytes("abcdefghij"), new ByteArrayInputStream(baos.toByteArray())); + } + + @Test + public void testAddingElementWhenFinishedThrows() throws Exception { + expectedException.expect(IOException.class); + expectedException.expectMessage("Stream has been finished."); + testValues(toBytes("a")).markElementStart(); + } + + @Test + public void testWritingByteWhenFinishedThrows() throws Exception { + expectedException.expect(IOException.class); + expectedException.expectMessage("Stream has been finished."); + testValues(toBytes("a")).write(1); + } + + @Test + public void testWritingBytesWhenFinishedThrows() throws Exception { + expectedException.expect(IOException.class); + expectedException.expectMessage("Stream has been finished."); + testValues(toBytes("a")).write("b".getBytes()); + } + + private List toBytes(String ... values) { + ImmutableList.Builder builder = ImmutableList.builder(); + for (String value : values) { + builder.add(value.getBytes()); + } + return builder.build(); + } + + private BufferedElementCountingOutputStream + testValues(List expectedValues) throws Exception { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + BufferedElementCountingOutputStream os = createAndWriteValues(expectedValues, baos); + os.finish(); + verifyValues(expectedValues, new ByteArrayInputStream(baos.toByteArray())); + return os; + } + + private void verifyValues(List expectedValues, InputStream is) throws Exception { + List values = new ArrayList<>(); + long count; + do { + count = VarInt.decodeLong(is); + for (int i = 0; i < count; ++i) { + values.add(ByteArrayCoder.of().decode(is, Context.NESTED)); + } + } while(count > 0); + + if (expectedValues.isEmpty()) { + assertTrue(values.isEmpty()); + } else { + assertThat(values, IsIterableContainingInOrder.contains(expectedValues.toArray())); + } + } + + private BufferedElementCountingOutputStream + createAndWriteValues(List values, OutputStream output) throws Exception { + BufferedElementCountingOutputStream os = + new BufferedElementCountingOutputStream(output, BUFFER_SIZE); + + for (byte[] value : values) { + os.markElementStart(); + ByteArrayCoder.of().encode(value, os, Context.NESTED); + } + return os; + } +} + From e975c13029a222b37efb1cce0a63a6d9315392f1 Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 23 Nov 2015 08:36:18 -0800 Subject: [PATCH 1187/1541] Fire a pane on AfterAll merge if it was not finished in all windows Previously, when windows merged, an AfterAll trigger would think it was ALREADY_FINISHED if all of its subtriggers were ALREADY_FINISHED. But some of the subtriggers might have been finished in different windows, and the combined AfterAll never fired (hence finished) in any window. With this change, that firing occurs. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108508082 --- .../sdk/transforms/windowing/AfterAll.java | 10 ++++++- .../transforms/windowing/AfterAllTest.java | 26 +++++++++++++++++++ 2 files changed, 35 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAll.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAll.java index ed4f7ff77eb2d..a542bbf679a92 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAll.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAll.java @@ -84,7 +84,15 @@ public MergeResult onMerge(OnMergeContext c) throws Exception { fired |= result.isFire(); } - return fired ? MergeResult.FIRE_AND_FINISH : MergeResult.ALREADY_FINISHED; + // When we reach this point, we know all subtriggers are finished, possibly already, + // possibly because of firing right now. So this trigger is finished and the decision + // is whether to fire. If no subtrigger wants to fire and the root trigger was already + // finished in some window, then there is no need to fire. + if (!fired && c.trigger().finishedInAnyMergingWindow()) { + return MergeResult.ALREADY_FINISHED; + } else { + return MergeResult.FIRE_AND_FINISH; + } } @Override diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java index 9edef5ea2b9d3..87a00db9b55e9 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java @@ -178,6 +178,32 @@ public void testOnMergeFires() throws Exception { new IntervalWindow(new Instant(1), new Instant(22))); } + @Test + public void testOnMergeFiresNotAlreadyFinished() throws Exception { + setUp(Sessions.withGapDuration(Duration.millis(10))); + + when(mockTrigger1.onElement(Mockito..OnElementContext>any())) + .thenReturn(TriggerResult.CONTINUE); + when(mockTrigger2.onElement(Mockito..OnElementContext>any())) + .thenReturn(TriggerResult.CONTINUE); + tester.injectElements( + TimestampedValue.of(1, new Instant(1)), + TimestampedValue.of(5, new Instant(12))); + + when(mockTrigger1.onMerge(Mockito..OnMergeContext>any())) + .thenReturn(MergeResult.ALREADY_FINISHED); + when(mockTrigger2.onMerge(Mockito..OnMergeContext>any())) + .thenReturn(MergeResult.ALREADY_FINISHED); + tester.injectElements( + TimestampedValue.of(12, new Instant(5))); + + assertThat(tester.extractOutput(), Matchers.contains( + isSingleWindowedValue(Matchers.containsInAnyOrder(1, 5, 12), 1, 1, 22))); + assertTrue(tester.isMarkedFinished(new IntervalWindow(new Instant(1), new Instant(22)))); + tester.assertHasOnlyGlobalAndFinishedSetsFor( + new IntervalWindow(new Instant(1), new Instant(22))); + } + @Test public void testFireDeadline() throws Exception { BoundedWindow window = new IntervalWindow(new Instant(0), new Instant(10)); From 34f4dcc608a6f11b0f85615a407792df0ab075f1 Mon Sep 17 00:00:00 2001 From: dhalperi Date: Mon, 23 Nov 2015 12:15:18 -0800 Subject: [PATCH 1188/1541] Fix a worker crash when writing to intermediate files The check `coder instanceof ValueOnlyWindowedValueCoder` is not enough to guarantee that this is a user-requested AvroSink at the pipeline end, because ValueOnlyWindowedValueCoder is also used at the beginning of a pipeline to read from user sources, which may also be materialized. We were crashing when materializing immediately after a non-Avro source, e.g., TextIO used as a side input. This failed because we tried to pass a StringUtf8Coder to AvroSink. The updated check will catch the crashing case, but will still use AvroSink when materializing a source's immediate output and the Source uses an AvroCoder. However, this is safe since it will be treated by the reader as if it was coming directly from an AvroSource, and the window and timestamp will be re-applied correctly. Also add comments. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108535669 --- .../sdk/runners/worker/AvroReaderFactory.java | 11 ++---- .../dataflow/sdk/runners/worker/AvroSink.java | 6 +-- .../sdk/runners/worker/AvroSinkFactory.java | 39 ++++++++++++++++++- 3 files changed, 43 insertions(+), 13 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactory.java index cc8e87efa957c..82fd6fe027d32 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderFactory.java @@ -55,15 +55,10 @@ Reader create(CloudObject spec, Coder coder, PipelineOptions options) thro Long startOffset = getLong(spec, PropertyNames.START_OFFSET, null); Long endOffset = getLong(spec, PropertyNames.END_OFFSET, null); - // If the coder is a ValueOnlyWindowedValueCoder, the source is a user source. Otherwise, - // the coder is a FullWindowedValueCoder and the source is a materialized PCollection. - if (coder instanceof ValueOnlyWindowedValueCoder) { + // See AvroSinkFactory#create for an explanation of this logic. + if (coder instanceof ValueOnlyWindowedValueCoder + && ((ValueOnlyWindowedValueCoder) coder).getValueCoder() instanceof AvroCoder) { ValueOnlyWindowedValueCoder valueCoder = (ValueOnlyWindowedValueCoder) coder; - if (!(valueCoder.getValueCoder() instanceof AvroCoder)) { - throw new IllegalArgumentException( - "AvroReader requires an AvroCoder, but the instance given was " - + valueCoder.getValueCoder()); - } return new AvroReader<>( filename, startOffset, endOffset, (AvroCoder) valueCoder.getValueCoder(), options); } else { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSink.java index 9ecf1c3cdebe2..b101a2b0fcded 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSink.java @@ -16,14 +16,13 @@ package com.google.cloud.dataflow.sdk.runners.worker; -import static com.google.cloud.dataflow.sdk.util.WindowedValue.ValueOnlyWindowedValueCoder; - import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.IOChannelUtils; import com.google.cloud.dataflow.sdk.util.MimeTypes; import com.google.cloud.dataflow.sdk.util.ShardingWritableByteChannel; import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue.ValueOnlyWindowedValueCoder; import com.google.cloud.dataflow.sdk.util.common.worker.Sink; import org.apache.avro.Schema; @@ -57,7 +56,8 @@ public AvroSink(String filename, ValueOnlyWindowedValueCoder coder) { public AvroSink(String filenamePrefix, String shardFormat, String filenameSuffix, int shardCount, ValueOnlyWindowedValueCoder coder) { if (!(coder.getValueCoder() instanceof AvroCoder)) { - throw new IllegalArgumentException("AvroSink requires an AvroCoder"); + throw new IllegalArgumentException(String.format( + "AvroSink requires an AvroCoder, not a %s", coder.getValueCoder().getClass())); } this.filenamePrefix = filenamePrefix; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java index fcc7f2d48e0e5..fadd31a38fbff 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroSinkFactory.java @@ -18,6 +18,7 @@ import static com.google.cloud.dataflow.sdk.util.Structs.getString; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.util.CloudObject; @@ -49,8 +50,42 @@ static Sink create(CloudObject spec, Coder coder) throws Exception { String filename = getString(spec, PropertyNames.FILENAME); - if (coder instanceof ValueOnlyWindowedValueCoder) { - return (Sink) new AvroSink(filename, (ValueOnlyWindowedValueCoder) coder); + // Avro sinks are used both for outputting user data at the end of a pipeline and for + // materializing PCollections as intermediate results. It is important to distinguish these + // two cases because one requires only the values (outputting with AvroSink) and one requires + // the values along with their window and timestamp (materializing intermediate results with + // AvroByteSink). + // + // The logic we would like is "use AvroSink when writing at the end of a pipeline; use + // AvroByteSink for materialized results". + // + // ValueOnlyWindowedValueCoder is used to decode/encode the values read from a Source, and used + // to encode the values written to a Sink. FullWindowedValueCoder is used as the coder between + // other edges in a Dataflow pipeline graph. + // + // Checking that the provided coder is an instance of ValueOnlyWindowedValueCoder is almost + // enough to identify a user's AvroSink at the end of a pipeline, but it does not eliminate the + // case when we are materializing immediately after reading from a Source. If this was the + // entire check to decide to use AvroSink, there could be a crash when we materialized the + // output of a Source that does not use AvroCoder, such as TextIO with StringUtf8Coder. + // + // Adding the additional test that the inner value coder is an AvroCoder will eliminate the + // TextIO case but will leave sources that, like AvroSource, use AvroCoder to represent their + // values. This fixes the potential crash, but still would use AvroSink for intermediate + // results immediately after such a Source. + // + // Luckily, using AvroSink in these cases is safe. Though AvroSink will only encode the value, + // and will drop the associated timestamp and window, the dropped values were applied by + // ValueOnlyWindowedValueCoder and will be reapplied by the same when the file is re-read by + // later in the pipeline. + // + // Otherwise, this is definitely a materialized result and we should use the AvroByteSink to + // include the window and timestamp. + // + // See AvroReaderFactory#create for the accompanying reader logic. + if (coder instanceof ValueOnlyWindowedValueCoder + && ((ValueOnlyWindowedValueCoder) coder).getValueCoder() instanceof AvroCoder) { + return new AvroSink(filename, (ValueOnlyWindowedValueCoder) coder); } else { return new AvroByteSink<>(filename, coder); } From dd85e088c7680f75d64bc18b50dbe550611a212a Mon Sep 17 00:00:00 2001 From: lcwik Date: Mon, 23 Nov 2015 14:20:47 -0800 Subject: [PATCH 1189/1541] Enable application default credentials for Cloud Shell ----Release Notes---- Updated core Google dependencies from 1.20.0 from 1.21.0. [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108548408 --- examples/pom.xml | 12 +++++----- .../resources/archetype-resources/pom.xml | 16 +++++++------- pom.xml | 10 ++++----- sdk/pom.xml | 22 +++++++------------ 4 files changed, 27 insertions(+), 33 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 6b16defd203ee..bd02ea8f9bc83 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -371,7 +371,7 @@ ${google-clients.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -385,7 +385,7 @@ ${dataflow.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -399,7 +399,7 @@ ${bigquery.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -413,7 +413,7 @@ ${google-clients.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -433,7 +433,7 @@ ${datastore.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -447,7 +447,7 @@ ${pubsub.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 diff --git a/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml index bed413eaa04ce..85310878d7994 100644 --- a/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml +++ b/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml @@ -100,10 +100,10 @@ com.google.api-client google-api-client - 1.20.0 + 1.21.0 + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -115,10 +115,10 @@ com.google.apis google-api-services-bigquery - v2-rev238-1.20.0 + v2-rev248-1.21.0 + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -129,10 +129,10 @@ com.google.http-client google-http-client - 1.20.0 + 1.21.0 + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -143,10 +143,10 @@ com.google.apis google-api-services-pubsub - v1-rev3-1.20.0 + v1-rev7-1.21.0 + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 diff --git a/pom.xml b/pom.xml index ea75ee107c6c7..f352b8435fa91 100644 --- a/pom.xml +++ b/pom.xml @@ -68,19 +68,19 @@ 1.7.7 - v2-rev238-1.20.0 - v1b3-rev10-1.20.0 + v2-rev248-1.21.0 + v1b3-rev12-1.21.0 v1beta2-rev1-3.0.2 - 1.20.0 + 1.21.0 18.0 1.3 2.4.2 2.4 4.11 2.5.0 - v1-rev3-1.20.0 + v1-rev7-1.21.0 1.7.7 - v1-rev25-1.19.1 + v1-rev53-1.21.0 pom diff --git a/sdk/pom.xml b/sdk/pom.xml index e04a64fd1861d..4514e2121ff4b 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -419,7 +419,7 @@ ${dataflow.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -451,7 +451,7 @@ ${bigquery.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -465,7 +465,7 @@ ${pubsub.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -479,7 +479,7 @@ ${storage.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -493,7 +493,7 @@ ${google-clients.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -501,19 +501,13 @@ - - com.google.http-client - google-http-client - ${google-clients.version} - - com.google.oauth-client google-oauth-client-java6 ${google-clients.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -527,7 +521,7 @@ ${google-clients.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 @@ -541,7 +535,7 @@ ${datastore.version} + in by a transitive dependency of google-api-client --> com.google.guava guava-jdk5 From 470b7e430dbb4210a960942945360256091e7ab5 Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 23 Nov 2015 15:04:25 -0800 Subject: [PATCH 1190/1541] Merge all subtriggers in AfterAll. Previously, not all subtriggers were given an opportunity to merge their state. Now they are each give such an opportunity. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108553113 --- .../sdk/transforms/windowing/AfterAll.java | 24 +++++++++---------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAll.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAll.java index a542bbf679a92..e76c14b77a86b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAll.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAll.java @@ -72,23 +72,21 @@ public TriggerResult onElement(OnElementContext c) throws Exception { @Override public MergeResult onMerge(OnMergeContext c) throws Exception { // CONTINUE if merging returns CONTINUE for at least one sub-trigger - // FIRE_AND_FINISH if merging returns FIRE or FIRE_AND_FINISH for at least one sub-trigger - // *and* FIRE, FIRE_AND_FINISH, or FINISH for all other sub-triggers. - // FINISH if merging returns FINISH for all sub-triggers. - boolean fired = false; + // ALREADY_FINISHED if merging returns ALREADY_FINISHED for all sub-triggers and this + // trigger itself was already finished in some window. + // FIRE_AND_FINISH otherwise: It means this trigger is ready to fire (because all subtriggers + // are satisfied) but has never fired as a whole. + boolean anyContinue = true; + boolean alreadyFinished = true; for (ExecutableTrigger subTrigger : c.trigger().subTriggers()) { MergeResult result = subTrigger.invokeMerge(c); - if (MergeResult.CONTINUE.equals(result)) { - return MergeResult.CONTINUE; - } - fired |= result.isFire(); + anyContinue |= !(result.isFire() && result.isFinish()); + alreadyFinished &= !result.isFire() && result.isFinish(); } - // When we reach this point, we know all subtriggers are finished, possibly already, - // possibly because of firing right now. So this trigger is finished and the decision - // is whether to fire. If no subtrigger wants to fire and the root trigger was already - // finished in some window, then there is no need to fire. - if (!fired && c.trigger().finishedInAnyMergingWindow()) { + if (anyContinue) { + return MergeResult.CONTINUE; + } else if (alreadyFinished && c.trigger().finishedInAnyMergingWindow()) { return MergeResult.ALREADY_FINISHED; } else { return MergeResult.FIRE_AND_FINISH; From a3e38ef748cd555a0de7b2c96b732307c315d5f6 Mon Sep 17 00:00:00 2001 From: dhalperi Date: Tue, 24 Nov 2015 09:52:37 -0800 Subject: [PATCH 1191/1541] PubSubIO: add, test, document support for RFC3339 timestamps Fixes GoogleCloudPlatform/DataflowJavaSDK#65 ----Release Notes---- PubSubIO now accepts RFC 3339 timestamps in addition to milliseconds since Unix epoch. [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108621625 --- .../cloud/dataflow/sdk/io/PubsubIO.java | 82 +++++++-- .../cloud/dataflow/sdk/io/PubsubIOTest.java | 157 ++++++++++++++++++ 2 files changed, 222 insertions(+), 17 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index 99c802964baf7..653b31f059e4d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -16,6 +16,11 @@ package com.google.cloud.dataflow.sdk.io; +import static com.google.common.base.MoreObjects.firstNonNull; +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.api.client.util.Clock; +import com.google.api.client.util.DateTime; import com.google.api.services.pubsub.Pubsub; import com.google.api.services.pubsub.model.AcknowledgeRequest; import com.google.api.services.pubsub.model.PublishRequest; @@ -43,7 +48,9 @@ import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded; import com.google.cloud.dataflow.sdk.values.PDone; import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; import org.joda.time.Duration; import org.joda.time.Instant; @@ -129,6 +136,48 @@ private static void validatePubsubName(String name) { } } + /** + * Returns the {@link Instant} that corresponds to the timestamp in the supplied + * {@link PubsubMessage} under the specified {@code ink label}. See + * {@link PubsubIO.Read#timestampLabel(String)} for details about how these messages are + * parsed. + * + *

The {@link Clock} parameter is used to virtualize time for testing. + * + * @throws IllegalArgumentException if the timestamp label is provided, but there is no + * corresponding attribute in the message or the value provided is not a valid timestamp + * string. + * @see PubsubIO.Read#timestampLabel(String) + */ + @VisibleForTesting + protected static Instant assignMessageTimestamp( + PubsubMessage message, @Nullable String label, Clock clock) { + if (label == null) { + return new Instant(clock.currentTimeMillis()); + } + + // Extract message attributes, defaulting to empty map if null. + Map attributes = firstNonNull( + message.getAttributes(), ImmutableMap.of()); + + String timestampStr = attributes.get(label); + checkArgument(timestampStr != null && !timestampStr.isEmpty(), + "PubSub message is missing a timestamp in label: %s", label); + + long millisSinceEpoch; + try { + // Try parsing as milliseconds since epoch. Note there is no way to parse a string in + // RFC 3339 format here. + // Expected IllegalArgumentException if parsing fails; we use that to fall back to RFC 3339. + millisSinceEpoch = Long.parseLong(timestampStr); + } catch (IllegalArgumentException e) { + // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an IllegalArgumentException + // if parsing fails, and the caller should handle. + millisSinceEpoch = DateTime.parseRfc3339(timestampStr).getValue(); + } + return new Instant(millisSinceEpoch); + } + /** * Class representing a Cloud Pub/Sub Subscription. */ @@ -387,10 +436,18 @@ public static Bound subscription(String subscription) { /** * Creates and returns a transform reading from Cloud Pub/Sub where record timestamps are * expected to be provided as Pub/Sub message attributes. The {@code timestampLabel} - * parameter specifies the name of the attribute that contains the timestamp. The value of the - * attribute should be a numerical value representing the number of milliseconds since the Unix - * epoch. For example, if using the Joda time classes, - * {@link Instant#getMillis()} returns the correct value for this label. + * parameter specifies the name of the attribute that contains the timestamp. + * + *

The timestamp value is expected to be represented in the attribute as either: + * + *

    + *
  • a numerical value representing the number of milliseconds since the Unix epoch. For + * example, if using the Joda time classes, {@link Instant#getMillis()} returns the correct + * value for this attribute. + *
  • a String in RFC 3339 format. For example, {@code 2015-10-29T23:41:41.123Z}. The + * sub-second component of the timestamp is optional, and digits beyond the first three + * (i.e., time units smaller than milliseconds) will be ignored. + *
* *

If {@code timestampLabel} is not provided, the system will generate record timestamps * the first time it sees each record. All windowing will be done relative to these timestamps. @@ -402,6 +459,8 @@ public static Bound subscription(String subscription) { * *

Note that the system can guarantee that no late data will ever be seen when it assigns * timestamps by arrival time (i.e. {@code timestampLabel} is not provided). + * + * @see RFC 3339 */ public static Bound timestampLabel(String timestampLabel) { return new Bound<>(DEFAULT_PUBSUB_CODER).timestampLabel(timestampLabel); @@ -734,20 +793,9 @@ public void processElement(ProcessContext c) throws IOException { } for (PubsubMessage message : messages) { - Instant timestamp; - if (getTimestampLabel() == null) { - timestamp = Instant.now(); - } else { - if (message.getAttributes() == null - || !message.getAttributes().containsKey(getTimestampLabel())) { - throw new RuntimeException( - "Message from pubsub missing timestamp label: " + getTimestampLabel()); - } - timestamp = - new Instant(Long.parseLong(message.getAttributes().get(getTimestampLabel()))); - } c.outputWithTimestamp( - CoderUtils.decodeFromByteArray(getCoder(), message.decodeData()), timestamp); + CoderUtils.decodeFromByteArray(getCoder(), message.decodeData()), + assignMessageTimestamp(message, getTimestampLabel(), Clock.SYSTEM)); } } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/PubsubIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/PubsubIOTest.java index ad0fcafdcd2d9..8e7ad29bbe244 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/PubsubIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/PubsubIOTest.java @@ -18,12 +18,21 @@ import static org.junit.Assert.assertEquals; +import com.google.api.client.testing.http.FixedClock; +import com.google.api.client.util.Clock; +import com.google.api.services.pubsub.model.PubsubMessage; + +import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import java.util.HashMap; + +import javax.annotation.Nullable; + /** * Tests for PubsubIO Read and Write transforms. */ @@ -73,4 +82,152 @@ public void testTopicValidationTooLong() throws Exception { .append("1111111111111111111111111111111111111111111111111111111111111111111111111111") .toString()); } + + /** + * Helper function that creates a {@link PubsubMessage} with the given timestamp registered as + * an attribute with the specified label. + * + *

If {@code label} is {@code null}, then the attributes are {@code null}. + * + *

Else, if {@code timestamp} is {@code null}, then attributes are present but have no key for + * the label. + */ + private static PubsubMessage messageWithTimestamp( + @Nullable String label, @Nullable String timestamp) { + PubsubMessage message = new PubsubMessage(); + if (label == null) { + message.setAttributes(null); + return message; + } + + message.setAttributes(new HashMap()); + + if (timestamp == null) { + return message; + } + + message.getAttributes().put(label, timestamp); + return message; + } + + /** + * Helper function that parses the given string to a timestamp through the PubSubIO plumbing. + */ + private static Instant parseTimestamp(@Nullable String timestamp) { + PubsubMessage message = messageWithTimestamp("mylabel", timestamp); + return PubsubIO.assignMessageTimestamp(message, "mylabel", Clock.SYSTEM); + } + + @Test + public void noTimestampLabelReturnsNow() { + final long time = 987654321L; + Instant timestamp = PubsubIO.assignMessageTimestamp( + messageWithTimestamp(null, null), null, new FixedClock(time)); + + assertEquals(new Instant(time), timestamp); + } + + @Test + public void timestampLabelWithNullAttributesThrowsError() { + PubsubMessage message = messageWithTimestamp(null, null); + thrown.expect(RuntimeException.class); + thrown.expectMessage("PubSub message is missing a timestamp in label: myLabel"); + + PubsubIO.assignMessageTimestamp(message, "myLabel", Clock.SYSTEM); + } + + @Test + public void timestampLabelSetWithMissingAttributeThrowsError() { + PubsubMessage message = messageWithTimestamp("notMyLabel", "ignored"); + thrown.expect(RuntimeException.class); + thrown.expectMessage("PubSub message is missing a timestamp in label: myLabel"); + + PubsubIO.assignMessageTimestamp(message, "myLabel", Clock.SYSTEM); + } + + @Test + public void timestampLabelParsesMillisecondsSinceEpoch() { + Long millis = 1446162101123L; + assertEquals(new Instant(millis), parseTimestamp(millis.toString())); + } + + @Test + public void timestampLabelParsesRfc3339Seconds() { + String rfc3339 = "2015-10-29T23:41:41Z"; + assertEquals(Instant.parse(rfc3339), parseTimestamp(rfc3339)); + } + + @Test + public void timestampLabelParsesRfc3339Tenths() { + String rfc3339tenths = "2015-10-29T23:41:41.1Z"; + assertEquals(Instant.parse(rfc3339tenths), parseTimestamp(rfc3339tenths)); + } + + @Test + public void timestampLabelParsesRfc3339Hundredths() { + String rfc3339hundredths = "2015-10-29T23:41:41.12Z"; + assertEquals(Instant.parse(rfc3339hundredths), parseTimestamp(rfc3339hundredths)); + } + + @Test + public void timestampLabelParsesRfc3339Millis() { + String rfc3339millis = "2015-10-29T23:41:41.123Z"; + assertEquals(Instant.parse(rfc3339millis), parseTimestamp(rfc3339millis)); + } + + @Test + public void timestampLabelParsesRfc3339Micros() { + String rfc3339micros = "2015-10-29T23:41:41.123456Z"; + assertEquals(Instant.parse(rfc3339micros), parseTimestamp(rfc3339micros)); + // Note: micros part 456/1000 is dropped. + assertEquals(Instant.parse("2015-10-29T23:41:41.123Z"), parseTimestamp(rfc3339micros)); + } + + @Test + public void timestampLabelParsesRfc3339MicrosRounding() { + String rfc3339micros = "2015-10-29T23:41:41.123999Z"; + assertEquals(Instant.parse(rfc3339micros), parseTimestamp(rfc3339micros)); + // Note: micros part 999/1000 is dropped, not rounded up. + assertEquals(Instant.parse("2015-10-29T23:41:41.123Z"), parseTimestamp(rfc3339micros)); + } + + @Test + public void timestampLabelWithInvalidFormatThrowsError() { + thrown.expect(NumberFormatException.class); + parseTimestamp("not-a-timestamp"); + } + + @Test + public void timestampLabelWithInvalidFormat2ThrowsError() { + thrown.expect(NumberFormatException.class); + parseTimestamp("null"); + } + + @Test + public void timestampLabelWithInvalidFormat3ThrowsError() { + thrown.expect(NumberFormatException.class); + parseTimestamp("2015-10"); + } + + @Test + public void timestampLabelParsesRfc3339WithSmallYear() { + // Google and JodaTime agree on dates after 1582-10-15, when the Gregorian Calendar was adopted + // This is therefore a "small year" until this difference is reconciled. + String rfc3339SmallYear = "1582-10-15T01:23:45.123Z"; + assertEquals(Instant.parse(rfc3339SmallYear), parseTimestamp(rfc3339SmallYear)); + } + + @Test + public void timestampLabelParsesRfc3339WithLargeYear() { + // Year 9999 in range. + String rfc3339LargeYear = "9999-10-29T23:41:41.123999Z"; + assertEquals(Instant.parse(rfc3339LargeYear), parseTimestamp(rfc3339LargeYear)); + } + + @Test + public void timestampLabelRfc3339WithTooLargeYearThrowsError() { + thrown.expect(NumberFormatException.class); + // Year 10000 out of range. + parseTimestamp("10000-10-29T23:41:41.123999Z"); + } } From 5b6ed0544a4b81a6f28c6b33384c45f357fdc909 Mon Sep 17 00:00:00 2001 From: klk Date: Wed, 8 Jul 2015 10:51:40 -0700 Subject: [PATCH 1192/1541] Add customized output timestamp for GroupByKey result This addresses a problem whereby the output watermark may be held longer than neeeded/desired, depending on how combined data is going to be used. By default, in a GroupByKey (and window) the collection of grouped values for a window receives a timestamp of the minimum non-late data, and the output watermark never advances past this time until it is output. This ensures that timestamps extracted from the collection data are never late relative to the watermark. There are situations where this is not desirable. For example, with session windows the minimum non-late timestamp for a window may be indefinitely far in the past, hence the output watermark may be held up indefinitely, preventing later shorter-lived sessions from being closed and processed. A related issue affects sliding windows but a one-off solution is already in place. After this experimental addition, the behavior may be customized by providing an OutputTimeFn to Window.into(...).withOutputTimeFn(...). There are three policies provided in OutputTimeFns: - output at the earliest non-late input timestamp - output at the latest non-late input timestamp - output at the end of the window Either of the latest non-late input timestamp or the end of window allow session windows to close in a timely manner. Start at the javadoc for OutputTimeFn for a full description. ----Release Notes---- - Added customized output timestamps for GroupByKey results. This allows, in particular, setting output timestamps to a later time in order to allow the output watermark to progress, rather than being held at the earliest timestamp of any buffered input. [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108627978 --- .../sdk/annotations/Experimental.java | 5 +- .../worker/StreamingSideInputDoFnRunner.java | 3 +- .../worker/WindmillStateInternals.java | 106 +++- .../sdk/testing/WindowFnTestUtils.java | 90 ++- .../transforms/windowing/CalendarWindows.java | 6 +- .../transforms/windowing/OutputTimeFn.java | 327 +++++++++++ .../transforms/windowing/OutputTimeFns.java | 168 ++++++ .../windowing/PartitioningWindowFn.java | 4 +- .../sdk/transforms/windowing/Sessions.java | 14 +- .../transforms/windowing/SlidingWindows.java | 25 +- .../sdk/transforms/windowing/Window.java | 62 +- .../sdk/transforms/windowing/WindowFn.java | 88 ++- .../GroupAlsoByWindowsAndCombineDoFn.java | 86 +-- .../sdk/util/GroupAlsoByWindowsDoFn.java | 18 +- .../GroupAlsoByWindowsViaIteratorsDoFn.java | 14 +- .../sdk/util/ReduceFnContextFactory.java | 5 +- .../dataflow/sdk/util/ReduceFnRunner.java | 3 + .../dataflow/sdk/util/WatermarkHold.java | 58 +- .../dataflow/sdk/util/WindowingStrategy.java | 54 +- .../util/state/InMemoryStateInternals.java | 45 +- .../state/MergedWatermarkStateInternal.java | 57 +- .../sdk/util/state/MergingStateInternals.java | 22 +- .../sdk/util/state/StateInternals.java | 16 +- .../dataflow/sdk/util/state/StateTag.java | 12 +- .../dataflow/sdk/util/state/StateTags.java | 26 +- .../util/state/WatermarkStateInternal.java | 23 +- .../worker/WindmillStateInternalsTest.java | 254 +++++++-- .../sdk/transforms/GroupByKeyTest.java | 59 ++ .../transforms/windowing/SessionsTest.java | 48 +- .../sdk/transforms/windowing/WindowTest.java | 64 ++- .../GroupAlsoByWindowsAndCombineDoFnTest.java | 14 +- .../util/GroupAlsoByWindowsProperties.java | 528 +++++++++++++----- ...roupAlsoByWindowsViaIteratorsDoFnTest.java | 22 + ...pAlsoByWindowsViaOutputBufferDoFnTest.java | 44 ++ .../dataflow/sdk/util/TriggerTester.java | 8 +- .../state/InMemoryStateInternalsTest.java | 118 +++- .../dataflow/sdk/util/state/StateTagTest.java | 15 +- 37 files changed, 2112 insertions(+), 399 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OutputTimeFn.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OutputTimeFns.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/annotations/Experimental.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/annotations/Experimental.java index a75a5c9980a28..f094442ec3c08 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/annotations/Experimental.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/annotations/Experimental.java @@ -69,6 +69,9 @@ public enum Kind { CODER_ENCODING_ID, /** State-related experimental APIs. */ - STATE + STATE, + + /** Experimental APIs related to customizing the output time for computed values. */ + OUTPUT_TIME } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunner.java index 4e4874bf8b878..85e79b6438073 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunner.java @@ -107,7 +107,8 @@ public StreamingSideInputDoFnRunner( this.elementsAddr = StateTags.makeSystemTagInternal(StateTags.bag("elem", WindowedValue.getFullCoder(doFnInfo.getInputCoder(), windowFn.windowCoder()))); this.watermarkHoldingAddr = - StateTags.makeSystemTagInternal(StateTags.watermarkStateInternal("hold")); + StateTags.makeSystemTagInternal(StateTags.watermarkStateInternal("hold", + doFnInfo.getWindowingStrategy().getOutputTimeFn())); this.blockedMap = stepContext.stateInternals().state(StateNamespaces.global(), blockedMapAddr) .get().read(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java index c91644d77aa72..6f0b44c4c4559 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java @@ -18,6 +18,8 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; import com.google.cloud.dataflow.sdk.util.state.BagState; import com.google.cloud.dataflow.sdk.util.state.CombiningValueStateInternal; @@ -64,10 +66,15 @@ public BagState bindBag(StateTag> address, Coder elemCoder } @Override - public WatermarkStateInternal bindWatermark( - StateTag address) { + public WatermarkStateInternal bindWatermark( + StateTag address, + OutputTimeFn outputTimeFn) { return new WindmillWatermarkState( - encodeKey(namespace, address), stateFamily, reader, scopedReadStateSupplier); + encodeKey(namespace, address), + stateFamily, + reader, + scopedReadStateSupplier, + outputTimeFn); } @Override @@ -350,6 +357,7 @@ public void persist(Windmill.WorkItemCommitRequest.Builder commitBuilder) throws private static class WindmillWatermarkState implements WatermarkStateInternal, WindmillState { + private final OutputTimeFn outputTimeFn; private final ByteString stateKey; private final String stateFamily; private final WindmillStateReader reader; @@ -358,12 +366,17 @@ private static class WindmillWatermarkState implements WatermarkStateInternal, W private boolean cleared = false; private Instant localAdditions = null; - private WindmillWatermarkState(ByteString stateKey, String stateFamily, - WindmillStateReader reader, Supplier readStateSupplier) { + private WindmillWatermarkState( + ByteString stateKey, + String stateFamily, + WindmillStateReader reader, + Supplier readStateSupplier, + OutputTimeFn outputTimeFn) { this.stateKey = stateKey; this.stateFamily = stateFamily; this.reader = reader; this.readStateSupplier = readStateSupplier; + this.outputTimeFn = outputTimeFn; } @Override @@ -372,6 +385,15 @@ public void clear() { localAdditions = null; } + /** + * {@inheritDoc} + * + *

Does nothing. There is only one hold and it is not extraneous. + * See {@link MergedWatermarkStateInternal} for a nontrivial implementation. + */ + @Override + public void releaseExtraneousHolds() { } + @Override public StateContents get() { // If we clear after calling get() but before calling read(), technically we didn't need the @@ -386,11 +408,9 @@ public StateContents get() { public Instant read() { Instant value = localAdditions; if (!cleared) { - try (StateSampler.ScopedState scope = readStateSupplier.get()) { + try (StateSampler.ScopedState scope = readStateSupplier.get()) { Instant persisted = persistedData.get(); - if (value == null || (persisted != null && persisted.isBefore(value))) { - value = persisted; - } + value = (value == null) ? persisted : outputTimeFn.combine(value, persisted); } catch (InterruptedException | ExecutionException e) { throw new RuntimeException("Unable to read state", e); } @@ -422,28 +442,80 @@ public Boolean read() { } @Override - public void add(Instant watermarkHold) { - if (localAdditions == null || watermarkHold.isBefore(localAdditions)) { - localAdditions = watermarkHold; - } + public void add(Instant outputTime) { + localAdditions = (localAdditions == null) ? outputTime + : outputTimeFn.combine(outputTime, localAdditions); } @Override public void persist(Windmill.WorkItemCommitRequest.Builder commitBuilder) { - // If we do a delete, we need to have done a read - if (cleared) { - reader.watermarkFuture(stateKey, stateFamily); + if (!cleared && localAdditions == null) { + // Nothing to do + return; + } else if (cleared && localAdditions == null) { + // Just clearing the persisted state; blind delete commitBuilder.addWatermarkHoldsBuilder() .setTag(stateKey) .setStateFamily(stateFamily) .setReset(true); + + } else if (cleared && localAdditions != null) { + // Since we cleared before adding, we can do a blind overwrite of persisted state + commitBuilder.addWatermarkHoldsBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .setReset(true) + .addTimestamps(TimeUnit.MILLISECONDS.toMicros(localAdditions.getMillis())); + } else if (!cleared && localAdditions != null){ + // Otherwise, we need to combine the local additions with the already persisted data + combineWithPersisted(commitBuilder); + } else { + throw new IllegalStateException("Unreachable condition"); } + } - if (localAdditions != null) { + /** + * Combines local additions with persisted data and mutates the {@code commitBuilder} + * to write the result. + */ + private void combineWithPersisted(Windmill.WorkItemCommitRequest.Builder commitBuilder) { + boolean windmillCanCombine = false; + + // If the combined output time depends only on the window, then we are just blindly adding + // the same value that may or may not already be present. This depends on the state only being + // used for one window. + windmillCanCombine |= outputTimeFn.dependsOnlyOnWindow(); + + // If the combined output time depends only on the earliest input timestamp, then because + // assignOutputTime is monotonic, the hold only depends on the earliest output timestamp + // (which is the value submitted as a watermark hold). The only way holds for later inputs + // can be redundant is if the are later (or equal) to the earliest. So taking the MIN + // implicitly, as Windmill does, has the desired behavior. + windmillCanCombine |= outputTimeFn.dependsOnlyOnEarliestInputTimestamp(); + + if (windmillCanCombine) { + // We do a blind write and let Windmill take the MIN commitBuilder.addWatermarkHoldsBuilder() .setTag(stateKey) .setStateFamily(stateFamily) .addTimestamps(TimeUnit.MILLISECONDS.toMicros(localAdditions.getMillis())); + } else { + // The non-fast path does a read-modify-write + Instant priorHold; + try { + priorHold = reader.watermarkFuture(stateKey, stateFamily).get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Unable to read state", e); + } + + Instant combinedHold = (priorHold == null) ? localAdditions + : outputTimeFn.combine(priorHold, localAdditions); + + commitBuilder.addWatermarkHoldsBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .setReset(true) + .addTimestamps(TimeUnit.MILLISECONDS.toMicros(combinedHold.getMillis())); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowFnTestUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowFnTestUtils.java index 300e1c5ff11e4..dc0baf52b81de 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowFnTestUtils.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/WindowFnTestUtils.java @@ -16,13 +16,19 @@ package com.google.cloud.dataflow.sdk.testing; +import static org.hamcrest.Matchers.greaterThan; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import org.joda.time.Instant; +import org.joda.time.ReadableInstant; import java.util.ArrayList; import java.util.Collection; @@ -34,6 +40,8 @@ import java.util.Map; import java.util.Set; +import javax.annotation.Nullable; + /** * A utility class for testing {@link WindowFn}s. */ @@ -51,7 +59,6 @@ public static Set set(long... timestamps) { return result; } - /** * Runs the {@link WindowFn} over the provided input, returning a map * of windows to the timestamps in those windows. @@ -193,7 +200,7 @@ public static void validateNonInterferingOutputTime Instant instant = new Instant(timestamp); for (W window : windows) { - Instant outputTimestamp = windowFn.getOutputTime(instant, window); + Instant outputTimestamp = windowFn.getOutputTimeFn().assignOutputTime(instant, window); assertFalse("getOutputTime must be greater than or equal to input timestamp", outputTimestamp.isBefore(instant)); assertFalse("getOutputTime must be less than or equal to the max timestamp", @@ -203,8 +210,9 @@ public static void validateNonInterferingOutputTime /** * Assigns the given {@code timestamp} to windows using the specified {@code windowFn}, and - * verifies that result of {@code windowFn.getOutputTimestamp} for later windows (as defined by - * {@code maxTimestamp} won't prevent the watermark from passing the end of earlier windows. + * verifies that result of {@link WindowFn#getOutputTime windowFn.getOutputTime} for later windows + * (as defined by {@code maxTimestamp} won't prevent the watermark from passing the end of earlier + * windows. * *

This verifies that overlapping windows don't interfere at all. Depending on the * {@code windowFn} this may be stricter than desired. @@ -223,7 +231,7 @@ public int compare(BoundedWindow o1, BoundedWindow o2) { Instant instant = new Instant(timestamp); Instant endOfPrevious = null; for (W window : sortedWindows) { - Instant outputTimestamp = windowFn.getOutputTime(instant, window); + Instant outputTimestamp = windowFn.getOutputTimeFn().assignOutputTime(instant, window); if (endOfPrevious == null) { // If this is the first window, the output timestamp can be anything, as long as it is in // the valid range. @@ -242,4 +250,76 @@ public int compare(BoundedWindow o1, BoundedWindow o2) { endOfPrevious = window.maxTimestamp(); } } + + /** + * Verifies that later-ending merged windows from any of the timestamps hold up output of + * earlier-ending windows, using the provided {@link WindowFn} and {@link OutputTimeFn}. + * + *

Given a list of lists of timestamps, where each list is expected to merge into a single + * window with end times in ascending order, assigns and merges windows for each list (as though + * each were a separate key/user session). Then maps each timestamp in the list according to + * {@link OutputTimeFn#assignOutputTime outputTimeFn.assignOutputTime()} and + * {@link OutputTimeFn#combine outputTimeFn.combine()}. + * + *

Verifies that a overlapping windows do not hold each other up via the watermark. + */ + public static + void validateGetOutputTimestamps( + WindowFn windowFn, + OutputTimeFn outputTimeFn, + List> timestampsPerWindow) throws Exception { + + // Assign windows to each timestamp, then merge them, storing the merged windows in + // a list in corresponding order to timestampsPerWindow + final List windows = new ArrayList<>(); + for (List timestampsForWindow : timestampsPerWindow) { + final Set windowsToMerge = new HashSet<>(); + + for (long timestamp : timestampsForWindow) { + windowsToMerge.addAll( + WindowFnTestUtils.assignedWindows(windowFn, timestamp)); + } + + windowFn.mergeWindows(windowFn.new MergeContext() { + @Override + public Collection windows() { + return windowsToMerge; + } + + @Override + public void merge(Collection toBeMerged, W mergeResult) throws Exception { + windows.add(mergeResult); + } + }); + } + + // Map every list of input timestamps to an output timestamp + final List combinedOutputTimestamps = new ArrayList<>(); + for (int i = 0; i < timestampsPerWindow.size(); ++i) { + List timestampsForWindow = timestampsPerWindow.get(i); + W window = windows.get(i); + + List outputInstants = new ArrayList<>(); + for (long inputTimestamp : timestampsForWindow) { + outputInstants.add(outputTimeFn.assignOutputTime(new Instant(inputTimestamp), window)); + } + + combinedOutputTimestamps.add(OutputTimeFns.combineOutputTimes(outputTimeFn, outputInstants)); + } + + // Consider windows in increasing order of max timestamp; ensure the output timestamp is after + // the max timestamp of the previous + @Nullable W earlierEndingWindow = null; + for (int i = 0; i < windows.size(); ++i) { + W window = windows.get(i); + ReadableInstant outputTimestamp = combinedOutputTimestamps.get(i); + + if (earlierEndingWindow != null) { + assertThat(outputTimestamp, + greaterThan((ReadableInstant) earlierEndingWindow.maxTimestamp())); + } + + earlierEndingWindow = window; + } + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java index 689ebf247e480..de5140f2a5d60 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java @@ -131,7 +131,7 @@ public Coder windowCoder() { } @Override - public boolean isCompatible(WindowFn other) { + public boolean isCompatible(WindowFn other) { if (!(other instanceof DaysWindows)) { return false; } @@ -217,7 +217,7 @@ public Coder windowCoder() { } @Override - public boolean isCompatible(WindowFn other) { + public boolean isCompatible(WindowFn other) { if (!(other instanceof MonthsWindows)) { return false; } @@ -312,7 +312,7 @@ public Coder windowCoder() { } @Override - public boolean isCompatible(WindowFn other) { + public boolean isCompatible(WindowFn other) { if (!(other instanceof YearsWindows)) { return false; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OutputTimeFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OutputTimeFn.java new file mode 100644 index 0000000000000..f97cd8589f0e1 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OutputTimeFn.java @@ -0,0 +1,327 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import com.google.cloud.dataflow.sdk.annotations.Experimental; +import com.google.common.collect.Ordering; + +import org.joda.time.Instant; + +import java.io.Serializable; +import java.util.Objects; + +/** + * (Experimental) A function from timestamps of input values to the timestamp for a + * computed value. + * + *

The function is represented via three components: + *

    + *
  1. {@link #assignOutputTime} calculates an output timestamp for any input + * value in a particular window.
  2. + *
  3. The output timestamps for all non-late input values within a window are combined + * according to {@link #combine combine()}, a commutative and associative operation on + * the output timestamps.
  4. + *
  5. The output timestamp when windows merge is provided by {@link #merge merge()}.
  6. + *
+ * + *

To implement this interface, extend {@link OutputTimeFn.Defaults} or + * {@link OutputTimeFn.DependsOnlyOnWindow} or your implementation may be impacted when the + * interface is enlarged. This interface will only be enlarged in ways that are + * backwards-compatible for consumers. The base classes will only be changed in ways that + * are backwards-compatible for implementors as well. + * + *

Note that as long as the interface remains experimental, we may choose to change it in + * arbitrary backwards incompatible ways if it is indicated by the experiment. + * + * @param the type of window. Contravariant: methods accepting any subtype of + * {@code OutputTimeFn} should use the parameter type {@code OutputTimeFn}. + */ +@Experimental(Experimental.Kind.OUTPUT_TIME) +public interface OutputTimeFn extends Serializable { + + /** + * Returns the output timestamp to use for data depending on the given + * {@code inputTimestamp} in the specified {@code window}. + * + * + *

The result of this method must be between {@code inputTimestamp} and + * {@code window.maxTimestamp()} (inclusive on both sides). + * + *

This function must be monotonic across input timestamps. Specifically, if {@code A < B}, + * then {@code assignOutputTime(A, window) <= assignOutputTime(B, window)}. + * + *

For a {@link WindowFn} that doesn't produce overlapping windows, this can (and typically + * should) just return {@code inputTimestamp}. In the presence of overlapping windows, it is + * suggested that the result in later overlapping windows is past the end of earlier windows + * so that the later windows don't prevent the watermark from + * progressing past the end of the earlier window. + * + *

See the overview of {@link OutputTimeFn} for the consistency properties required + * between {@link #assignOutputTime}, {@link #combine}, and {@link #merge}. + */ + Instant assignOutputTime(Instant inputTimestamp, W window); + + /** + * Combines the given output times, which must be from the same window, into an output time + * for a computed value. + * + *

    + *
  • {@code combine} must be commutative: {@code combine(a, b).equals(combine(b, a))}.
  • + *
  • {@code combine} must be associative: + * {@code combine(a, combine(b, c)).equals(combine(combine(a, b), c))}.
  • + *
+ */ + Instant combine(Instant outputTime, Instant otherOutputTime); + + /** + * Merges the given output times, presumed to be combined output times for windows that + * are merging, into an output time for the {@code resultWindow}. + * + *

When windows {@code w1} and {@code w2} merge to become a new window {@code w1plus2}, + * then {@link #merge} must be implemented such that the output time is the same as + * if all timestamps were assigned in {@code w1plus2}. Formally: + * + *

{@code fn.merge(w, fn.assignOutputTime(t1, w1), fn.assignOutputTime(t2, w2))} + * + *

must be equal to + * + *

{@code fn.combine(fn.assignOutputTime(t1, w1plus2), fn.assignOutputTime(t2, w1plus2))} + * + *

If the assigned time depends only on the window, the correct implementation of + * {@link #merge merge()} necessarily returns the result of + * {@link #assignOutputTime assignOutputTime(t1, w1plus2)} + * (which equals {@link #assignOutputTime assignOutputTime(t2, w1plus2)}. + * Defaults for this case are provided by {@link DependsOnlyOnWindow}. + * + *

For many other {@link OutputTimeFn} implementations, such as taking the earliest or latest + * timestamp, this will be the same as {@link #combine combine()}. Defaults for this + * case are provided by {@link Default}. + */ + Instant merge(W intoWindow, Iterable mergingTimestamps); + + /** + * Returns {@code true} if the result of combination of many output timestamps actually depends + * only on the earliest. + * + *

This may allow optimizations when it is very efficient to retrieve the earliest timestamp + * to be combined. + */ + boolean dependsOnlyOnEarliestInputTimestamp(); + + /** + * Returns {@code true} if the result does not depend on what outputs were combined but only + * the window they are in. The canonical example is if all timestamps are sure to + * be the end of the window. + * + *

This may allow optimizations, since it is typically very efficient to retrieve the window + * and combining output timestamps is not necessary. + * + *

If the assigned output time for an implementation depends only on the window, consider + * extending {@link DependsOnlyOnWindow}, which returns {@code true} here and also provides + * a framework for easily implementing a correct {@link #merge}, {@link #combine} and + * {@link #assignOutputTime}. + */ + boolean dependsOnlyOnWindow(); + + /** + * Please extend {@link Defaults} or {@link DependsOnlyOnWindow} if you want guaranteed + * compilation compatibility; this interface may be enlarged in consumer-compatible ways. + */ + void pleaseExtendBaseClassesForCompilationCompatibility(); + + /** + * (Experimental) Default method implementations for {@link OutputTimeFn} where the + * output time depends on the input element timestamps and possibly the window. + * + *

To complete an implementation, override {@link #assignOutputTime}, at a minimum. + * + *

By default, {@link #combine} and {@link #merge} return the earliest timestamp of their + * inputs. + */ + public abstract static class Defaults implements OutputTimeFn { + + /** + * {@inheritDoc} + * + * @return the earlier of the two timestamps. + */ + @Override + public Instant combine(Instant outputTimestamp, Instant otherOutputTimestamp) { + return Ordering.natural().min(outputTimestamp, otherOutputTimestamp); + } + + /** + * {@inheritDoc} + * + * @return the result of {@link #combine combine(outputTimstamp, otherOutputTimestamp)}, + * by default. + */ + @Override + public Instant merge(W resultWindow, Iterable mergingTimestamps) { + return OutputTimeFns.combineOutputTimes(this, mergingTimestamps); + } + + /** + * {@inheritDoc} + * + * @return {@code false}. An {@link OutputTimeFn} that depends only on the window should extend + * {@link DependsOnlyOnWindow}. + */ + @Override + public final boolean dependsOnlyOnWindow() { + return false; + } + + /** + * {@inheritDoc} + * + * @return {@code true} by default. + */ + @Override + public boolean dependsOnlyOnEarliestInputTimestamp() { + return false; + } + + /** + * {@inheritDoc} + * + * @return {@code true} if the two {@link OutputTimeFn} instances have the same class, by + * default. + */ + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + + return this.getClass().equals(other.getClass()); + } + + @Override + public int hashCode() { + return Objects.hash(getClass()); + } + + /** + * This base class provides compilation compatibility when {@link OutputTimeFn} is enlarged. + */ + @Override + public void pleaseExtendBaseClassesForCompilationCompatibility() { } + } + + /** + * (Experimental) Default method implementations for {@link OutputTimeFn} when the + * output time depends only on the window. + * + *

To complete an implementation, override {@link #assignOutputTime(BoundedWindow)}. + */ + public abstract static class DependsOnlyOnWindow + implements OutputTimeFn { + + /** + * Returns the output timestamp to use for data in the specified {@code window}. + * + *

Note that the result of this method must be between the maximum possible input timestamp + * in {@code window} and {@code window.maxTimestamp()} (inclusive on both sides). + * + *

For example, using {@code Sessions.withGapDuration(gapDuration)}, we know that all input + * timestamps must lie at least {@code gapDuration} from the end of the session, so + * {@code window.maxTimestamp() - gapDuration} is an acceptable assigned timestamp. + * + * @see #assignOutputTime(Instant, BoundedWindow) + */ + protected abstract Instant assignOutputTime(W window); + + /** + * {@inheritDoc} + * + * @return the result of {#link assignOutputTime(BoundedWindow) assignOutputTime(window)}. + */ + @Override + public final Instant assignOutputTime(Instant timestamp, W window) { + return assignOutputTime(window); + } + + /** + * {@inheritDoc} + * + * @return the same timestamp as both argument timestamps, which are necessarily equal. + */ + @Override + public final Instant combine(Instant outputTimestamp, Instant otherOutputTimestamp) { + return outputTimestamp; + } + + /** + * {@inheritDoc} + * + * @return the result of + * {@link #assignOutputTime(BoundedWindow) assignOutputTime(resultWindow)}. + */ + @Override + public final Instant merge(W resultWindow, Iterable mergingTimestamps) { + return assignOutputTime(resultWindow); + } + + /** + * {@inheritDoc} + * + * @return {@code true}. + */ + @Override + public final boolean dependsOnlyOnWindow() { + return true; + } + + /** + * {@inheritDoc} + * + * @return {@code true}. Since the output time depends only on the window, it can + * certainly be ascertained given a single input timestamp. + */ + @Override + public final boolean dependsOnlyOnEarliestInputTimestamp() { + return true; + } + + /** + * {@inheritDoc} + * + * @return {@code true} if the two {@link OutputTimeFn} instances have the same class, by + * default. + */ + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + + return this.getClass().equals(other.getClass()); + } + + @Override + public int hashCode() { + return Objects.hash(getClass()); + } + + /** + * This base class provides compilation compatibility when {@link OutputTimeFn} is enlarged. + */ + @Override + public void pleaseExtendBaseClassesForCompilationCompatibility() { } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OutputTimeFns.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OutputTimeFns.java new file mode 100644 index 0000000000000..dcc0f5b7b9c2e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OutputTimeFns.java @@ -0,0 +1,168 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.cloud.dataflow.sdk.annotations.Experimental; +import com.google.common.collect.Iterables; +import com.google.common.collect.Ordering; + +import org.joda.time.Instant; + +import javax.annotation.Nullable; + +/** + * (Experimental) Static utility methods and provided implementations for + * {@link OutputTimeFn}. + */ +@Experimental(Experimental.Kind.OUTPUT_TIME) +public class OutputTimeFns { + /** + * The policy of outputting at the earliest of the input timestamps for non-late input data + * that led to a computed value. + * + *

For example, suppose v1 through vn are all on-time + * elements being aggregated via some function {@code f} into + * {@code f}(v1, ..., vn. When emitted, the output + * timestamp of the result will be the earliest of the event time timestamps + * + *

If data arrives late, it has no effect on the output timestamp. + */ + public static OutputTimeFn outputAtEarliestInputTimestamp() { + return new OutputAtEarliestInputTimestamp(); + } + + /** + * The policy of holding the watermark to the latest of the input timestamps + * for non-late input data that led to a computed value. + * + *

For example, suppose v1 through vn are all on-time + * elements being aggregated via some function {@code f} into + * {@code f}(v1, ..., vn. When emitted, the output + * timestamp of the result will be the latest of the event time timestamps + * + *

If data arrives late, it has no effect on the output timestamp. + */ + public static OutputTimeFn outputAtLatestInputTimestamp() { + return new OutputAtLatestInputTimestamp(); + } + + /** + * The policy of outputting with timestamps at the end of the window. + * + *

Note that this output timestamp depends only on the window. See + * {#link dependsOnlyOnWindow()}. + * + *

When windows merge, instead of using {@link OutputTimeFn#combine} to obtain an output + * timestamp for the results in the new window, it is mandatory to obtain a new output + * timestamp from {@link OutputTimeFn#assignOutputTime} with the new window and an arbitrary + * timestamp (because it is guaranteed that the timestamp is irrelevant). + * + *

For non-merging window functions, this {@link OutputTimeFn} works transparently. + */ + public static OutputTimeFn outputAtEndOfWindow() { + return new OutputAtEndOfWindow(); + } + + /** + * Applies the given {@link OutputTimeFn} to the given output times, obtaining + * the output time for a value computed. See {@link OutputTimeFn#combine} for + * a full specification. + * + * @throws IllegalArgumentException if {@code outputTimes} is empty. + */ + public static Instant combineOutputTimes( + OutputTimeFn outputTimeFn, Iterable outputTimes) { + checkArgument( + !Iterables.isEmpty(outputTimes), + "Collection of output times must not be empty in %s.combineOutputTimes", + OutputTimeFns.class.getName()); + + @Nullable + Instant combinedOutputTime = null; + for (Instant outputTime : outputTimes) { + combinedOutputTime = + combinedOutputTime == null + ? outputTime : outputTimeFn.combine(combinedOutputTime, outputTime); + } + return combinedOutputTime; + } + + /** + * See {@link #outputAtEarliestInputTimestamp}. + */ + private static class OutputAtEarliestInputTimestamp extends OutputTimeFn.Defaults { + @Override + public Instant assignOutputTime(Instant inputTimestamp, BoundedWindow window) { + return inputTimestamp; + } + + @Override + public Instant combine(Instant outputTime, Instant otherOutputTime) { + return Ordering.natural().min(outputTime, otherOutputTime); + } + + /** + * {@inheritDoc} + * + * @return {@code true}. The result of any combine will be the earliest input timestamp. + */ + @Override + public boolean dependsOnlyOnEarliestInputTimestamp() { + return true; + } + } + + /** + * See {@link #outputAtLatestInputTimestamp}. + */ + private static class OutputAtLatestInputTimestamp extends OutputTimeFn.Defaults { + @Override + public Instant assignOutputTime(Instant inputTimestamp, BoundedWindow window) { + return inputTimestamp; + } + + @Override + public Instant combine(Instant outputTime, Instant otherOutputTime) { + return Ordering.natural().max(outputTime, otherOutputTime); + } + + /** + * {@inheritDoc} + * + * @return {@code false}. + */ + @Override + public boolean dependsOnlyOnEarliestInputTimestamp() { + return false; + } + } + + private static class OutputAtEndOfWindow extends OutputTimeFn.DependsOnlyOnWindow { + + /** + *{@inheritDoc} + * + *@return {@code window.maxTimestamp()}. + */ + @Override + protected Instant assignOutputTime(BoundedWindow window) { + return window.maxTimestamp(); + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowFn.java index 7be1ebffd3a23..bea0285b61a82 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PartitioningWindowFn.java @@ -22,8 +22,8 @@ import java.util.Collection; /** - * A {@link WindowFn} that places each value into exactly one window - * based on its timestamp and never merges windows. + * A {@link WindowFn} that places each value into exactly one window based on its timestamp and + * never merges windows. * * @param type of elements being windowed * @param window type diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java index c419f7e792eca..da137c1f47f1c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java @@ -16,10 +16,11 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; +import com.google.cloud.dataflow.sdk.annotations.Experimental; +import com.google.cloud.dataflow.sdk.annotations.Experimental.Kind; import com.google.cloud.dataflow.sdk.coders.Coder; import org.joda.time.Duration; -import org.joda.time.Instant; import java.util.Arrays; import java.util.Collection; @@ -85,13 +86,14 @@ public IntervalWindow getSideInputWindow(BoundedWindow window) { throw new UnsupportedOperationException("Sessions is not allowed in side inputs"); } - public Duration getGapDuration() { - return gapDuration; + @Experimental(Kind.OUTPUT_TIME) + @Override + public OutputTimeFn getOutputTimeFn() { + return OutputTimeFns.outputAtEarliestInputTimestamp(); } - @Override - public Instant getOutputTime(Instant inputTimestamp, IntervalWindow window) { - return inputTimestamp; + public Duration getGapDuration() { + return gapDuration; } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java index 3b4c2d71d7051..b0066d6124eba 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java @@ -16,6 +16,8 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; +import com.google.cloud.dataflow.sdk.annotations.Experimental; +import com.google.cloud.dataflow.sdk.annotations.Experimental.Kind; import com.google.cloud.dataflow.sdk.coders.Coder; import org.joda.time.Duration; @@ -170,17 +172,28 @@ public Duration getOffset() { } /** - * Ensure that later sliding windows have an output time that is past the end of earlier windows. + * Ensures that later sliding windows have an output time that is past the end of earlier windows. * *

If this is the earliest sliding window containing {@code inputTimestamp}, that's fine. * Otherwise, we pick the earliest time that doesn't overlap with earlier windows. */ + @Experimental(Kind.OUTPUT_TIME) @Override - public Instant getOutputTime(Instant inputTimestamp, IntervalWindow window) { - Instant startOfLastSegment = window.maxTimestamp().minus(period); - return startOfLastSegment.isBefore(inputTimestamp) - ? inputTimestamp - : startOfLastSegment.plus(1); + public OutputTimeFn getOutputTimeFn() { + return new OutputTimeFn.Defaults() { + @Override + public Instant assignOutputTime(Instant inputTimestamp, BoundedWindow window) { + Instant startOfLastSegment = window.maxTimestamp().minus(period); + return startOfLastSegment.isBefore(inputTimestamp) + ? inputTimestamp + : startOfLastSegment.plus(1); + } + + @Override + public boolean dependsOnlyOnEarliestInputTimestamp() { + return true; + } + }; } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java index a88fab1ce3832..531392b551b92 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java @@ -371,21 +371,23 @@ public static class Bound extends PTransform, PCollection> @Nullable private final AccumulationMode mode; @Nullable private final Duration allowedLateness; @Nullable private final ClosingBehavior closingBehavior; + @Nullable private final OutputTimeFn outputTimeFn; private Bound(String name, @Nullable WindowFn windowFn, @Nullable Trigger trigger, @Nullable AccumulationMode mode, @Nullable Duration allowedLateness, - ClosingBehavior behavior) { + ClosingBehavior behavior, @Nullable OutputTimeFn outputTimeFn) { super(name); this.windowFn = windowFn; this.trigger = trigger; this.mode = mode; this.allowedLateness = allowedLateness; this.closingBehavior = behavior; + this.outputTimeFn = outputTimeFn; } private Bound(String name) { - this(name, null, null, null, null, null); + this(name, null, null, null, null, null, null); } /** @@ -402,7 +404,8 @@ private Bound into(WindowFn windowFn) { throw new IllegalArgumentException("Window coders must be deterministic.", e); } - return new Bound<>(name, windowFn, trigger, mode, allowedLateness, closingBehavior); + return new Bound<>( + name, windowFn, trigger, mode, allowedLateness, closingBehavior, outputTimeFn); } /** @@ -415,7 +418,8 @@ private Bound into(WindowFn windowFn) { * explanation. */ public Bound named(String name) { - return new Bound<>(name, windowFn, trigger, mode, allowedLateness, closingBehavior); + return new Bound<>( + name, windowFn, trigger, mode, allowedLateness, closingBehavior, outputTimeFn); } /** @@ -432,7 +436,13 @@ public Bound named(String name) { @Experimental(Kind.TRIGGER) public Bound triggering(TriggerBuilder trigger) { return new Bound( - name, windowFn, trigger.buildTrigger(), mode, allowedLateness, closingBehavior); + name, + windowFn, + trigger.buildTrigger(), + mode, + allowedLateness, + closingBehavior, + outputTimeFn); } /** @@ -444,9 +454,14 @@ public Bound triggering(TriggerBuilder trigger) { */ @Experimental(Kind.TRIGGER) public Bound discardingFiredPanes() { - return new Bound(name, - windowFn, trigger, AccumulationMode.DISCARDING_FIRED_PANES, - allowedLateness, closingBehavior); + return new Bound( + name, + windowFn, + trigger, + AccumulationMode.DISCARDING_FIRED_PANES, + allowedLateness, + closingBehavior, + outputTimeFn); } /** @@ -458,9 +473,14 @@ public Bound discardingFiredPanes() { */ @Experimental(Kind.TRIGGER) public Bound accumulatingFiredPanes() { - return new Bound(name, - windowFn, trigger, AccumulationMode.ACCUMULATING_FIRED_PANES, - allowedLateness, closingBehavior); + return new Bound( + name, + windowFn, + trigger, + AccumulationMode.ACCUMULATING_FIRED_PANES, + allowedLateness, + closingBehavior, + outputTimeFn); } /** @@ -478,7 +498,18 @@ public Bound accumulatingFiredPanes() { */ @Experimental(Kind.TRIGGER) public Bound withAllowedLateness(Duration allowedLateness) { - return new Bound(name, windowFn, trigger, mode, allowedLateness, closingBehavior); + return new Bound( + name, windowFn, trigger, mode, allowedLateness, closingBehavior, outputTimeFn); + } + + /** + * (Experimental) Override the default {@link OutputTimeFn}, to control + * the output timestamp of values output from a {@link GroupByKey} operation. + */ + @Experimental(Kind.OUTPUT_TIME) + public Bound withOutputTimeFn(OutputTimeFn outputTimeFn) { + return new Bound( + name, windowFn, trigger, mode, allowedLateness, closingBehavior, outputTimeFn); } /** @@ -493,9 +524,11 @@ public Bound withAllowedLateness(Duration allowedLateness) { */ @Experimental(Kind.TRIGGER) public Bound withAllowedLateness(Duration allowedLateness, ClosingBehavior behavior) { - return new Bound(name, windowFn, trigger, mode, allowedLateness, behavior); + return new Bound(name, windowFn, trigger, mode, allowedLateness, behavior, outputTimeFn); } + // Rawtype cast of OutputTimeFn cannot be eliminated with intermediate variable, as it is + // casting between wildcards private WindowingStrategy getOutputStrategy(WindowingStrategy inputStrategy) { WindowingStrategy result = inputStrategy; if (windowFn != null) { @@ -513,6 +546,9 @@ public Bound withAllowedLateness(Duration allowedLateness, ClosingBehavior be if (closingBehavior != null) { result = result.withClosingBehavior(closingBehavior); } + if (outputTimeFn != null) { + result = result.withOutputTimeFn(outputTimeFn); + } return result; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java index a685345e563ee..0423cfb5eadda 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java @@ -16,7 +16,11 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; +import com.google.cloud.dataflow.sdk.annotations.Experimental; +import com.google.cloud.dataflow.sdk.annotations.Experimental.Kind; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.util.WindowingStrategy; +import com.google.common.collect.Ordering; import org.joda.time.Instant; @@ -128,26 +132,35 @@ public abstract void merge(Collection toBeMerged, W mergeResult) public abstract W getSideInputWindow(final BoundedWindow window); /** - * Returns the output timestamp to use for data depending on the given {@code inputTimestamp} - * in the specified {@code window}. + * @deprecated Implement {@link #getOutputTimeFn} to return either the appropriate + * {@link ElementaryOutputTimeFn} or a custom {@link OutputTimeFn} extending + * {@link OutputTimeFn.Defaults}. + */ + @Deprecated + @Experimental(Kind.OUTPUT_TIME) + public Instant getOutputTime(Instant inputTimestamp, W window) { + return getOutputTimeFn().assignOutputTime(inputTimestamp, window); + } + + /** + * Provides a default implementation for {@link WindowingStrategy#getOutputTimeFn()}. + * See the full specification there. * - *

The result must be between {@code inputTimestamp} and {@code window.maxTimestamp()} - * (inclusive on both sides). If this {@link WindowFn} doesn't produce overlapping windows, - * this can (and typically should) just return {@code inputTimestamp}. If this does produce - * overlapping windows, it is suggested that the result in later overlapping windows is - * past the end of earlier windows so that the later windows don't prevent the watermark from - * progressing past the end of the earlier window. + *

If this {@link WindowFn} doesn't produce overlapping windows, this need not (and probably + * should not) override any of the default implementations in {@link OutputTimeFn.Defaults}. * - *

Each {@code KV>} produced from a {@code GroupByKey} will be output at a - * timestamp that is the minimum of {@code getOutputTime} applied to the timestamp of all of - * the non-late {@code KV} that were used as input to the {@code GroupByKey}. The watermark - * is also prevented from advancing past this minimum timestamp until after the - * {@code KV>} has been output. + *

If this {@link WindowFn} does produce overlapping windows that can be predicted here, it is + * suggested that the result in later overlapping windows is past the end of earlier windows so + * that the later windows don't prevent the watermark from progressing past the end of the earlier + * window. * - *

This function should be monotonic across input timestamps. Specifically, if {@code A < B}, - * then {@code getOutputTime(A, window) <= getOutputTime(B, window)}. + *

For example, a timestamp in a sliding window should be moved past the beginning of the next + * sliding window. See {@link SlidingWindows#getOutputTimeFn}. */ - public abstract Instant getOutputTime(Instant inputTimestamp, W window); + @Experimental(Kind.OUTPUT_TIME) + public OutputTimeFn getOutputTimeFn() { + return new OutputAtEarliestAssignedTimestamp<>(this); + } /** * Returns true if this {@code WindowFn} never needs to merge any windows. @@ -162,4 +175,47 @@ public boolean isNonMerging() { public boolean assignsToSingleWindow() { return false; } + + /** + * A compatibility adapter that will return the assigned timestamps according to the + * {@link WindowFn}, which was the prior policy. Specifying the assigned output timestamps + * on the {@link WindowFn} is now deprecated. + */ + private static class OutputAtEarliestAssignedTimestamp + extends OutputTimeFn.Defaults { + + private final WindowFn windowFn; + + public OutputAtEarliestAssignedTimestamp(WindowFn windowFn) { + this.windowFn = windowFn; + } + + /** + * {@inheritDoc} + * + * @return the result of {@link WindowFn#getOutputTime windowFn.getOutputTime()}. + */ + @Override + @SuppressWarnings("deprecation") // this is an adapter for the deprecated behavior + public Instant assignOutputTime(Instant timestamp, W window) { + return windowFn.getOutputTime(timestamp, window); + } + + @Override + public Instant combine(Instant outputTime, Instant otherOutputTime) { + return Ordering.natural().min(outputTime, otherOutputTime); + } + + /** + * {@inheritDoc} + * + * @return {@code true}. When the {@link OutputTimeFn} is not overridden by {@link WindowFn} + * or {@link WindowingStrategy}, the minimum output timestamp is taken, which depends + * only on the minimum input timestamp by monotonicity of {@link #assignOutputTime}. + */ + @Override + public boolean dependsOnlyOnEarliestInputTimestamp() { + return true; + } + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsAndCombineDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsAndCombineDoFn.java index 2f1b49c2a6fa9..2fc6c650e2e19 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsAndCombineDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsAndCombineDoFn.java @@ -16,6 +16,7 @@ package com.google.cloud.dataflow.sdk.util; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; @@ -25,11 +26,11 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.joda.time.Instant; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Comparator; @@ -61,8 +62,8 @@ public static boolean isSupported(WindowingStrategy strategy) { // Right now, we support ACCUMULATING_FIRED_PANES because it is the same as // DISCARDING_FIRED_PANES. In Batch mode there is no late data so the default // trigger (after watermark) will only fire once. - if (!strategy.getMode().equals(AccumulationMode.DISCARDING_FIRED_PANES) - && !strategy.getMode().equals(AccumulationMode.ACCUMULATING_FIRED_PANES)) { + if (!(strategy.getMode().equals(AccumulationMode.DISCARDING_FIRED_PANES) + || strategy.getMode().equals(AccumulationMode.ACCUMULATING_FIRED_PANES))) { return false; } @@ -70,17 +71,24 @@ public static boolean isSupported(WindowingStrategy strategy) { } private final KeyedCombineFn combineFn; - private WindowFn windowFn; + private WindowingStrategy windowingStrategy; public GroupAlsoByWindowsAndCombineDoFn( - WindowFn windowFn, KeyedCombineFn combineFn) { + WindowingStrategy strategy, + KeyedCombineFn combineFn) { + + checkArgument(GroupAlsoByWindowsAndCombineDoFn.isSupported(strategy), + "%s does not support non-default triggering, " + + "found in windowing strategy: %s", + getClass(), + strategy); this.combineFn = combineFn; // To make a MergeContext that is compatible with the type of windowFn, we need to remove // the wildcard from the element type. @SuppressWarnings("unchecked") - WindowFn objectWindowFn = (WindowFn) windowFn; - this.windowFn = objectWindowFn; + WindowingStrategy objectWindowingStrategy = (WindowingStrategy) strategy; + this.windowingStrategy = objectWindowingStrategy; } @@ -98,9 +106,10 @@ public int compare(BoundedWindow w1, BoundedWindow w2) { }); final Map accumulators = Maps.newHashMap(); - final Map minTimestamps = Maps.newHashMap(); + final Map accumulatorOutputTimestamps = Maps.newHashMap(); - WindowFn.MergeContext mergeContext = windowFn.new MergeContext() { + WindowFn.MergeContext mergeContext = + windowingStrategy.getWindowFn().new MergeContext() { @Override public Collection windows() { return liveWindows; @@ -108,20 +117,18 @@ public Collection windows() { @Override public void merge(Collection toBeMerged, W mergeResult) throws Exception { - List accumsToBeMerged = new ArrayList<>(toBeMerged.size()); - Instant minTimestamp = null; + List accumsToBeMerged = Lists.newArrayListWithCapacity(toBeMerged.size()); + List timestampsToBeMerged = Lists.newArrayListWithCapacity(toBeMerged.size()); for (W window : toBeMerged) { accumsToBeMerged.add(accumulators.remove(window)); - - Instant timestampToBeMerged = minTimestamps.remove(window); - if (minTimestamp == null - || (timestampToBeMerged != null && timestampToBeMerged.isBefore(minTimestamp))) { - minTimestamp = timestampToBeMerged; - } + timestampsToBeMerged.add(accumulatorOutputTimestamps.remove(window)); } liveWindows.removeAll(toBeMerged); - minTimestamps.put(mergeResult, minTimestamp); + + Instant mergedOutputTimestamp = + windowingStrategy.getOutputTimeFn().merge(mergeResult, timestampsToBeMerged); + accumulatorOutputTimestamps.put(mergeResult, mergedOutputTimestamp); liveWindows.add(mergeResult); accumulators.put(mergeResult, combineFn.mergeAccumulators(key, accumsToBeMerged)); } @@ -132,48 +139,55 @@ public void merge(Collection toBeMerged, W mergeResult) throws Exception { @SuppressWarnings("unchecked") Collection windows = (Collection) e.getWindows(); - for (W w : windows) { - Instant timestamp = minTimestamps.get(w); - if (timestamp == null || timestamp.compareTo(e.getTimestamp()) > 0) { - minTimestamps.put(w, e.getTimestamp()); + for (W window : windows) { + Instant outputTime = + windowingStrategy.getOutputTimeFn().assignOutputTime(e.getTimestamp(), window); + Instant accumulatorOutputTime = accumulatorOutputTimestamps.get(window); + if (accumulatorOutputTime == null) { + accumulatorOutputTimestamps.put(window, outputTime); } else { - minTimestamps.put(w, timestamp); + accumulatorOutputTimestamps.put(window, + windowingStrategy.getOutputTimeFn().combine(outputTime, accumulatorOutputTime)); } - AccumT accum = accumulators.get(w); - checkState((timestamp == null && accum == null) || (timestamp != null && accum != null)); + AccumT accum = accumulators.get(window); + checkState((accumulatorOutputTime == null && accum == null) + || (accumulatorOutputTime != null && accum != null), + "accumulator and accumulatorOutputTime should both be null or both be non-null"); if (accum == null) { accum = combineFn.createAccumulator(key); - liveWindows.add(w); + liveWindows.add(window); } accum = combineFn.addInput(key, accum, e.getValue()); - accumulators.put(w, accum); + accumulators.put(window, accum); } - windowFn.mergeWindows(mergeContext); + windowingStrategy.getWindowFn().mergeWindows(mergeContext); while (!liveWindows.isEmpty() && liveWindows.peek().maxTimestamp().isBefore(e.getTimestamp())) { - closeWindow(key, liveWindows.poll(), accumulators, minTimestamps, c); + closeWindow(key, liveWindows.poll(), accumulators, accumulatorOutputTimestamps, c); } } // To have gotten here, we've either not had any elements added, or we've only run merge // and then closed windows. We don't need to retry merging. while (!liveWindows.isEmpty()) { - closeWindow(key, liveWindows.poll(), accumulators, minTimestamps, c); + closeWindow(key, liveWindows.poll(), accumulators, accumulatorOutputTimestamps, c); } } private void closeWindow( - K key, W w, Map accumulators, Map minTimestamps, ProcessContext c) { - AccumT accum = accumulators.remove(w); - Instant timestamp = minTimestamps.remove(w); + K key, W window, Map accumulators, + Map accumulatorOutputTimes, + ProcessContext context) { + AccumT accum = accumulators.remove(window); + Instant timestamp = accumulatorOutputTimes.remove(window); checkState(accum != null && timestamp != null); - c.windowingInternals().outputWindowedValue( + context.windowingInternals().outputWindowedValue( KV.of(key, combineFn.extractOutput(key, accum)), - windowFn.getOutputTime(timestamp, w), - Arrays.asList(w), + timestamp, + Arrays.asList(window), PaneInfo.ON_TIME_AND_ONLY_FIRING); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java index 678bd80a43b5a..4ed180b67df30 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsDoFn.java @@ -16,11 +16,12 @@ package com.google.cloud.dataflow.sdk.util; +import static com.google.common.base.Preconditions.checkNotNull; + import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.values.KV; -import com.google.common.base.Preconditions; /** * DoFn that merges windows and groups elements in those windows, optionally @@ -44,13 +45,11 @@ public abstract class GroupAlsoByWindowsDoFn GroupAlsoByWindowsDoFn, W> createForIterable(WindowingStrategy windowingStrategy, Coder inputCoder) { - @SuppressWarnings("unchecked") - WindowingStrategy noWildcard = (WindowingStrategy) windowingStrategy; return GroupAlsoByWindowsViaIteratorsDoFn.isSupported(windowingStrategy) ? new GroupAlsoByWindowsViaIteratorsDoFn(windowingStrategy) : new GroupAlsoByWindowsViaOutputBufferDoFn<>( - noWildcard, + windowingStrategy, SystemReduceFn.buffering(inputCoder)); } @@ -63,14 +62,15 @@ public abstract class GroupAlsoByWindowsDoFn windowingStrategy, final AppliedCombineFn combineFn, final Coder keyCoder) { - Preconditions.checkNotNull(combineFn); - @SuppressWarnings("unchecked") - WindowingStrategy noWildcard = (WindowingStrategy) windowingStrategy; + checkNotNull(combineFn); + return GroupAlsoByWindowsAndCombineDoFn.isSupported(windowingStrategy) - ? new GroupAlsoByWindowsAndCombineDoFn<>(noWildcard.getWindowFn(), combineFn.getFn()) + ? new GroupAlsoByWindowsAndCombineDoFn<>( + windowingStrategy, + combineFn.getFn()) : new GroupAlsoByWindowsViaOutputBufferDoFn<>( - noWildcard, + windowingStrategy, SystemReduceFn.combining(keyCoder, combineFn)); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaIteratorsDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaIteratorsDoFn.java index c5800afe6bb9d..400824017cbcb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaIteratorsDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaIteratorsDoFn.java @@ -63,11 +63,16 @@ public static boolean isSupported(WindowingStrategy strategy) { return false; } + // It must be possible to compute the output timestamp of a pane from the input timestamp + // of the element with the earliest input timestamp. + if (!strategy.getOutputTimeFn().dependsOnlyOnEarliestInputTimestamp()) { + return false; + } // Right now, we support ACCUMULATING_FIRED_PANES because it is the same as // DISCARDING_FIRED_PANES. In Batch mode there is no late data so the default // trigger (after watermark) will only fire once. - if (!strategy.getMode().equals(AccumulationMode.DISCARDING_FIRED_PANES) - && !strategy.getMode().equals(AccumulationMode.ACCUMULATING_FIRED_PANES)) { + if (!(strategy.getMode().equals(AccumulationMode.DISCARDING_FIRED_PANES) + || strategy.getMode().equals(AccumulationMode.ACCUMULATING_FIRED_PANES))) { return false; } @@ -76,7 +81,8 @@ public static boolean isSupported(WindowingStrategy strategy) { public GroupAlsoByWindowsViaIteratorsDoFn(WindowingStrategy strategy) { checkArgument(GroupAlsoByWindowsViaIteratorsDoFn.isSupported(strategy), - "%s does not support merging or non-default triggering, " + "%s does not support merging, non-default triggering, " + + "or any OutputTimeFn where dependsOnlyOnEarliest() is false, " + "found in windowing strategy: %s", getClass(), strategy); @@ -118,7 +124,7 @@ public void processElement(ProcessContext c) throws Exception { windows.put(window.maxTimestamp(), window); c.windowingInternals().outputWindowedValue( KV.of(key, (Iterable) new WindowReiterable(iterator, window)), - strategy.getWindowFn().getOutputTime(e.getTimestamp(), typedWindow), + strategy.getOutputTimeFn().assignOutputTime(e.getTimestamp(), typedWindow), Arrays.asList(window), PaneInfo.ON_TIME_AND_ONLY_FIRING); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnContextFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnContextFactory.java index 63d8ecb144733..df4a853599334 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnContextFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnContextFactory.java @@ -165,7 +165,7 @@ public StorageT access(StateTag address) { sourceNamespaces.add(namespaceFor(sourceWindow)); } - return stateInternals.mergedState(sourceNamespaces, namespace, address); + return stateInternals.mergedState(sourceNamespaces, namespace, address, window); } } @@ -211,7 +211,8 @@ public StorageT access(StateTag address) { mergingNamespaces.add(delegate.namespaceFor(mergingWindow)); } - return delegate.stateInternals.mergedState(mergingNamespaces, delegate.namespace, address); + return delegate.stateInternals.mergedState( + mergingNamespaces, delegate.namespace, address, delegate.window()); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java index 3052420f8a0ab..d539c84ddc066 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java @@ -381,6 +381,9 @@ public TriggerResult onMerge( throw wrapMaybeUserException(e); } + // Merge the watermark hold + watermarkHold.mergeHolds(resultContext); + // Have the trigger merge state as needed, and handle the result. TriggerResult triggerResult; try { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java index 669ba4fba4201..d480369b52e65 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java @@ -16,6 +16,7 @@ package com.google.cloud.dataflow.sdk.util; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; import com.google.cloud.dataflow.sdk.transforms.windowing.Window.ClosingBehavior; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.state.StateContents; @@ -38,17 +39,24 @@ public class WatermarkHold implements Serializable { /** Watermark hold used for the actual data-based hold. */ - @VisibleForTesting static final StateTag DATA_HOLD_TAG = - StateTags.makeSystemTagInternal(StateTags.watermarkStateInternal("hold")); + @VisibleForTesting static final String DATA_HOLD_ID = "hold"; + @VisibleForTesting static StateTag watermarkHoldTagForOutputTimeFn( + OutputTimeFn outputTimeFn) { + return StateTags.makeSystemTagInternal( + StateTags.watermarkStateInternal(DATA_HOLD_ID, outputTimeFn)); + } private final WindowingStrategy windowingStrategy; + private final StateTag watermarkHoldTag; public WatermarkHold(WindowingStrategy windowingStrategy) { this.windowingStrategy = windowingStrategy; + this.watermarkHoldTag = watermarkHoldTagForOutputTimeFn(windowingStrategy.getOutputTimeFn()); } /** - * Update the watermark hold to include the timestamp of the value in {@code c}. + * Update the watermark hold to include the appropriate output timestamp for the value in + * {@code c}. * *

If the value was not late, then the input watermark must be less than the timestamp, and we * can use {@link WindowFn#getOutputTime} to determine the appropriate output time. @@ -59,20 +67,40 @@ public WatermarkHold(WindowingStrategy windowingStrategy) { * dropped. */ public void addHold(ReduceFn.ProcessValueContext c, boolean isLate) { - Instant holdTo = isLate + Instant outputTime = isLate ? c.window().maxTimestamp().plus(windowingStrategy.getAllowedLateness()) - : windowingStrategy.getWindowFn().getOutputTime(c.timestamp(), c.window()); - c.state().access(DATA_HOLD_TAG).add(holdTo); + : windowingStrategy.getOutputTimeFn().assignOutputTime(c.timestamp(), c.window()); + c.state().access(watermarkHoldTag).add(outputTime); + } + + /** + * Updates the watermark hold when windows merge. For example, if the new window implies + * a later watermark hold, then earlier holds may be released. + */ + public void mergeHolds(final ReduceFn.OnMergeContext c) { + // If the output hold depends only on the window, then there may not be a hold in place + // for the new merged window, so add one. + if (windowingStrategy.getOutputTimeFn().dependsOnlyOnWindow()) { + Instant arbitraryTimestamp = new Instant(0); + c.state().access(watermarkHoldTag).add( + windowingStrategy.getOutputTimeFn().assignOutputTime( + arbitraryTimestamp, + c.window())); + } + + c.state().accessAcrossMergedWindows(watermarkHoldTag).releaseExtraneousHolds(); } /** - * Get information from the watermark hold for outputting. + * Returns the combined timestamp at which the output watermark was being held and releases + * the hold. * - *

The output timestamp is the minimum of getOutputTimestamp applied to the non-late elements - * that arrived in the current pane. + *

The returned timestamp is the output timestamp according to the {@link OutputTimeFn} + * from the windowing strategy of this {@link WatermarkHold}, combined across all the non-late + * elements in the current pane. */ public StateContents extractAndRelease(final ReduceFn.Context c) { - final WatermarkStateInternal dataHold = c.state().accessAcrossMergedWindows(DATA_HOLD_TAG); + final WatermarkStateInternal dataHold = c.state().accessAcrossMergedWindows(watermarkHoldTag); final StateContents holdFuture = dataHold.get(); return new StateContents() { @Override @@ -82,7 +110,7 @@ public Instant read() { hold = c.window().maxTimestamp(); } - // Clear the bag (to release the watermark) + // Clear the underlying state to allow the output watermark to progress. dataHold.clear(); return hold; @@ -91,18 +119,18 @@ public Instant read() { } public void holdForOnTime(final ReduceFn.Context c) { - c.state().accessAcrossMergedWindows(DATA_HOLD_TAG).add(c.window().maxTimestamp()); + c.state().accessAcrossMergedWindows(watermarkHoldTag).add(c.window().maxTimestamp()); } public void holdForFinal(final ReduceFn.Context c) { if (c.windowingStrategy().getClosingBehavior() == ClosingBehavior.FIRE_ALWAYS) { - c.state().accessAcrossMergedWindows(DATA_HOLD_TAG) + c.state().accessAcrossMergedWindows(watermarkHoldTag) .add(c.window().maxTimestamp().plus(c.windowingStrategy().getAllowedLateness())); } } public void releaseOnTime(final ReduceFn.Context c) { - c.state().accessAcrossMergedWindows(DATA_HOLD_TAG).clear(); + c.state().accessAcrossMergedWindows(watermarkHoldTag).clear(); if (c.windowingStrategy().getClosingBehavior() == ClosingBehavior.FIRE_ALWAYS && c.windowingStrategy().getAllowedLateness().isLongerThan(Duration.ZERO)) { @@ -111,6 +139,6 @@ public void releaseOnTime(final ReduceFn.Context c) { } public void releaseFinal(final ReduceFn.Context c) { - c.state().accessAcrossMergedWindows(DATA_HOLD_TAG).clear(); + c.state().accessAcrossMergedWindows(watermarkHoldTag).clear(); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowingStrategy.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowingStrategy.java index a452dc58a840f..c167b8c0cdffe 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowingStrategy.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowingStrategy.java @@ -16,13 +16,16 @@ package com.google.cloud.dataflow.sdk.util; +import com.google.cloud.dataflow.sdk.annotations.Experimental; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger; import com.google.cloud.dataflow.sdk.transforms.windowing.Window.ClosingBehavior; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; +import com.google.common.base.MoreObjects; import org.joda.time.Duration; @@ -52,6 +55,7 @@ public enum AccumulationMode { private static final WindowingStrategy DEFAULT = of(new GlobalWindows()); private final WindowFn windowFn; + private final OutputTimeFn outputTimeFn; private final ExecutableTrigger trigger; private final AccumulationMode mode; private final Duration allowedLateness; @@ -59,12 +63,14 @@ public enum AccumulationMode { private final boolean triggerSpecified; private final boolean modeSpecified; private final boolean allowedLatenessSpecified; + private final boolean outputTimeFnSpecified; private WindowingStrategy( WindowFn windowFn, ExecutableTrigger trigger, boolean triggerSpecified, AccumulationMode mode, boolean modeSpecified, Duration allowedLateness, boolean allowedLatenessSpecified, + OutputTimeFn outputTimeFn, boolean outputTimeFnSpecified, ClosingBehavior closingBehavior) { this.windowFn = windowFn; this.trigger = trigger; @@ -74,6 +80,8 @@ private WindowingStrategy( this.allowedLateness = allowedLateness; this.allowedLatenessSpecified = allowedLatenessSpecified; this.closingBehavior = closingBehavior; + this.outputTimeFn = outputTimeFn; + this.outputTimeFnSpecified = outputTimeFnSpecified; } /** @@ -89,6 +97,7 @@ public static WindowingStrategy of( ExecutableTrigger.create(DefaultTrigger.of()), false, AccumulationMode.DISCARDING_FIRED_PANES, false, DEFAULT_ALLOWED_LATENESS, false, + windowFn.getOutputTimeFn(), false, ClosingBehavior.FIRE_IF_NON_EMPTY); } @@ -124,6 +133,14 @@ public ClosingBehavior getClosingBehavior() { return closingBehavior; } + public OutputTimeFn getOutputTimeFn() { + return outputTimeFn; + } + + public boolean isOutputTimeFnSpecified() { + return outputTimeFnSpecified; + } + /** * Returns a {@link WindowingStrategy} identical to {@code this} but with the trigger set to * {@code wildcardTrigger}. @@ -136,6 +153,7 @@ public WindowingStrategy withTrigger(Trigger wildcardTrigger) { ExecutableTrigger.create(typedTrigger), true, mode, modeSpecified, allowedLateness, allowedLatenessSpecified, + outputTimeFn, outputTimeFnSpecified, closingBehavior); } @@ -149,6 +167,7 @@ public WindowingStrategy withMode(AccumulationMode mode) { trigger, triggerSpecified, mode, true, allowedLateness, allowedLatenessSpecified, + outputTimeFn, outputTimeFnSpecified, closingBehavior); } @@ -159,11 +178,18 @@ public WindowingStrategy withMode(AccumulationMode mode) { public WindowingStrategy withWindowFn(WindowFn wildcardWindowFn) { @SuppressWarnings("unchecked") WindowFn typedWindowFn = (WindowFn) wildcardWindowFn; + + // The onus of type correctness falls on the callee. + @SuppressWarnings("unchecked") + OutputTimeFn newOutputTimeFn = (OutputTimeFn) + (outputTimeFnSpecified ? outputTimeFn : typedWindowFn.getOutputTimeFn()); + return new WindowingStrategy( typedWindowFn, trigger, triggerSpecified, mode, modeSpecified, allowedLateness, allowedLatenessSpecified, + newOutputTimeFn, outputTimeFnSpecified, closingBehavior); } @@ -177,6 +203,7 @@ public WindowingStrategy withAllowedLateness(Duration allowedLateness) { trigger, triggerSpecified, mode, modeSpecified, allowedLateness, true, + outputTimeFn, outputTimeFnSpecified, closingBehavior); } @@ -186,15 +213,34 @@ public WindowingStrategy withClosingBehavior(ClosingBehavior closingBehavi trigger, triggerSpecified, mode, modeSpecified, allowedLateness, allowedLatenessSpecified, + outputTimeFn, outputTimeFnSpecified, + closingBehavior); + } + + @Experimental(Experimental.Kind.OUTPUT_TIME) + public WindowingStrategy withOutputTimeFn(OutputTimeFn outputTimeFn) { + + @SuppressWarnings("unchecked") + OutputTimeFn typedOutputTimeFn = (OutputTimeFn) outputTimeFn; + + return new WindowingStrategy( + windowFn, + trigger, triggerSpecified, + mode, modeSpecified, + allowedLateness, allowedLatenessSpecified, + typedOutputTimeFn, true, closingBehavior); } @Override public String toString() { - return String.format("%s, %s, %s", - StringUtils.approximateSimpleName(windowFn.getClass()), - trigger.toString(), - mode.toString()); + return MoreObjects.toStringHelper(this) + .add("windowFn", windowFn) + .add("allowedLateness", allowedLateness) + .add("trigger", trigger) + .add("accumulationMode", mode) + .add("outputTimeFn", outputTimeFn) + .toString(); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternals.java index a710dd050cb5c..1af60ad209461 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternals.java @@ -15,8 +15,12 @@ */ package com.google.cloud.dataflow.sdk.util.state; +import com.google.cloud.dataflow.sdk.annotations.Experimental; +import com.google.cloud.dataflow.sdk.annotations.Experimental.Kind; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; import com.google.cloud.dataflow.sdk.util.state.StateTag.StateBinder; import org.joda.time.Instant; @@ -30,6 +34,7 @@ * In-memory implementation of {@link StateInternals}. Used in {@code BatchModeExecutionContext} * and for running tests that need state. */ +@Experimental(Kind.STATE) public class InMemoryStateInternals extends MergingStateInternals { private interface InMemoryState { boolean isEmptyForTesting(); @@ -58,8 +63,10 @@ public BagState bindBag(final StateTag> address, Coder ele } @Override - public WatermarkStateInternal bindWatermark(StateTag address) { - return new WatermarkStateInternalImplementation(); + public WatermarkStateInternal bindWatermark( + StateTag address, + OutputTimeFn outputTimeFn) { + return new WatermarkStateInternalImplementation(outputTimeFn); } }; } @@ -118,35 +125,49 @@ public boolean isEmptyForTesting() { private final class WatermarkStateInternalImplementation implements WatermarkStateInternal, InMemoryState { - private Instant minimumHold = null; + + private final OutputTimeFn outputTimeFn; + private Instant combinedHold = null; + + public WatermarkStateInternalImplementation(OutputTimeFn outputTimeFn) { + this.outputTimeFn = outputTimeFn; + } @Override public void clear() { // Even though we're clearing we can't remove this from the in-memory state map, since // other users may already have a handle on this WatermarkBagInternal. - minimumHold = null; + combinedHold = null; } + /** + * {@inheritDoc} + * + *

Does nothing. There is only one hold and it is not extraneous. + * See {@link MergedWatermarkStateInternal} for a nontrivial implementation. + */ + @Override + public void releaseExtraneousHolds() { } + @Override public StateContents get() { return new StateContents() { @Override public Instant read() { - return minimumHold; + return combinedHold; } }; } @Override - public void add(Instant watermarkHold) { - if (minimumHold == null || minimumHold.isAfter(watermarkHold)) { - minimumHold = watermarkHold; - } + public void add(Instant outputTime) { + combinedHold = combinedHold == null ? outputTime + : outputTimeFn.combine(combinedHold, outputTime); } @Override public boolean isEmptyForTesting() { - return minimumHold == null; + return combinedHold == null; } @Override @@ -154,14 +175,14 @@ public StateContents isEmpty() { return new StateContents() { @Override public Boolean read() { - return minimumHold == null; + return combinedHold == null; } }; } @Override public String toString() { - return Objects.toString(minimumHold); + return Objects.toString(combinedHold); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergedWatermarkStateInternal.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergedWatermarkStateInternal.java index e2af4db164edf..ec9c492417681 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergedWatermarkStateInternal.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergedWatermarkStateInternal.java @@ -15,6 +15,11 @@ */ package com.google.cloud.dataflow.sdk.util.state; + +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; +import com.google.common.collect.Lists; + import org.joda.time.Instant; import java.util.ArrayList; @@ -25,15 +30,22 @@ * Implementation of {@link WatermarkStateInternal} reading from multiple sources and writing to a * single result. */ -class MergedWatermarkStateInternal implements WatermarkStateInternal { +class MergedWatermarkStateInternal implements WatermarkStateInternal { private final Collection sources; private final WatermarkStateInternal result; + private final OutputTimeFn outputTimeFn; + private final W resultWindow; public MergedWatermarkStateInternal( - Collection sources, WatermarkStateInternal result) { + Collection sources, + WatermarkStateInternal result, + W resultWindow, + OutputTimeFn outputTimeFn) { this.sources = sources; this.result = result; + this.resultWindow = resultWindow; + this.outputTimeFn = outputTimeFn; } @Override @@ -45,12 +57,17 @@ public void clear() { } @Override - public void add(Instant watermarkHold) { - result.add(watermarkHold); + public void add(Instant outputTimestamp) { + result.add(outputTimestamp); } @Override public StateContents get() { + // Short circuit if output times depend only on the window, hence are all equal. + if (outputTimeFn.dependsOnlyOnWindow()) { + return result.get(); + } + // Get the underlying StateContents's right away. final List> reads = new ArrayList<>(sources.size()); for (WatermarkStateInternal source : sources) { @@ -61,21 +78,23 @@ public StateContents get() { return new StateContents() { @Override public Instant read() { - Instant minimum = null; + List outputTimesToMerge = Lists.newArrayListWithCapacity(sources.size()); for (StateContents read : reads) { - Instant input = read.read(); - if (minimum == null || (input != null && minimum.isAfter(input))) { - minimum = input; + Instant sourceOutputTime = read.read(); + if (sourceOutputTime != null) { + outputTimesToMerge.add(sourceOutputTime); } } - // Also, compact the state - if (minimum != null) { + if (outputTimesToMerge.isEmpty()) { + return null; + } else { + // Also, compact the state clear(); - add(minimum); + Instant mergedOutputTime = outputTimeFn.merge(resultWindow, outputTimesToMerge); + add(mergedOutputTime); + return mergedOutputTime; } - - return minimum; } }; } @@ -101,4 +120,16 @@ public Boolean read() { } }; } + + @Override + public void releaseExtraneousHolds() { + if (outputTimeFn.dependsOnlyOnEarliestInputTimestamp()) { + // No need to do anything; the merged watermark state will hold to the earliest + // due to semantics of watermark holds. + } else { + // In all other cases, get() implements the necessary combining logic, and actually + // performs compaction that releases the watermark. + get().read(); + } + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergingStateInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergingStateInternals.java index a54b95dd8fc7c..c6a7ae17e3c56 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergingStateInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergingStateInternals.java @@ -15,8 +15,12 @@ */ package com.google.cloud.dataflow.sdk.util.state; +import com.google.cloud.dataflow.sdk.annotations.Experimental; +import com.google.cloud.dataflow.sdk.annotations.Experimental.Kind; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; import com.google.cloud.dataflow.sdk.util.state.StateTag.StateBinder; import java.util.ArrayList; @@ -26,12 +30,15 @@ * Abstract implementation of {@link StateInternals} that provides {@link #mergedState} in terms of * {@link #state}. */ +@Experimental(Kind.STATE) public abstract class MergingStateInternals implements StateInternals { @Override public > T mergedState( final Iterable sourceNamespaces, - final StateNamespace resultNamespace, StateTag address) { + final StateNamespace resultNamespace, + StateTag address, + final BoundedWindow resultWindow) { return address.bind(new StateBinder() { @Override public ValueState bindValue(StateTag> address, Coder coder) { @@ -73,8 +80,9 @@ CombiningValueStateInternal bindCombiningValue( } @Override - public WatermarkStateInternal bindWatermark( - StateTag address) { + public WatermarkStateInternal bindWatermark( + StateTag address, + OutputTimeFn outputTimeFn) { List sources = new ArrayList<>(); for (StateNamespace sourceNamespace : sourceNamespaces) { // Skip adding the result namespace for now. @@ -84,7 +92,13 @@ public WatermarkStateInternal bindWatermark( } WatermarkStateInternal result = state(resultNamespace, address); sources.add(result); - return new MergedWatermarkStateInternal(sources, result); + + // It is the responsibility of the SDK to only pass allowed result windows. + @SuppressWarnings("unchecked") + W typedResultWindow = (W) resultWindow; + + return new MergedWatermarkStateInternal( + sources, result, typedResultWindow, outputTimeFn); } }); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateInternals.java index a65ef87617682..f8038823b762e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateInternals.java @@ -15,13 +15,18 @@ */ package com.google.cloud.dataflow.sdk.util.state; +import com.google.cloud.dataflow.sdk.annotations.Experimental; +import com.google.cloud.dataflow.sdk.annotations.Experimental.Kind; +import com.google.cloud.dataflow.sdk.transforms.GroupByKey; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; + /** * {@code StateInternals} describes the functionality a runner needs to provide for the * State API to be supported. * *

The SDK will only use this after elements have been partitioned by key. For instance, after a - * {@code GroupByKey} operation. The runner implementation must ensure that any writes using - * {@code StaeIntetrnals} are implicitly scoped to the key being processed and the specific step + * {@link GroupByKey} operation. The runner implementation must ensure that any writes using + * {@link StateInternals} are implicitly scoped to the key being processed and the specific step * accessing state. * *

The runner implementation must also ensure that any writes to the associated state objects @@ -31,6 +36,7 @@ *

This is a low-level API intended for use by the Dataflow SDK. It should not be * used directly, and is highly likely to change. */ +@Experimental(Kind.STATE) public interface StateInternals { /** @@ -48,6 +54,8 @@ public interface StateInternals { * {@code sourceNamespaces} and {@code resultNamespace}. */ > T mergedState( - Iterable sourceNamespaces, StateNamespace resultNamespace, - StateTag address); + Iterable sourceNamespaces, + StateNamespace resultNamespace, + StateTag address, + BoundedWindow resultWindow); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTag.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTag.java index 6cf6e3b13011a..2419bb8ed6007 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTag.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTag.java @@ -20,6 +20,8 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; import com.google.cloud.dataflow.sdk.transforms.GroupByKey; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; import java.io.Serializable; @@ -50,7 +52,15 @@ public interface StateBinder { StateTag> address, Coder accumCoder, CombineFn combineFn); - WatermarkStateInternal bindWatermark(StateTag address); + /** + * Bind to a watermark {@link StateTag}. + * + *

This accepts the {@link OutputTimeFn} that dictates how watermark hold timestamps + * added to the returned {@link WatermarkStateInternal} are to be combined. + */ + WatermarkStateInternal bindWatermark( + StateTag address, + OutputTimeFn outputTimeFn); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java index 87f3ec0b62a21..8fe1681c3c970 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java @@ -21,6 +21,8 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderRegistry; import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; import com.google.common.base.MoreObjects; import java.io.Serializable; @@ -113,8 +115,9 @@ public static StateTag> bag(String id, Coder elemCoder) { /** * Create a state tag for holding the watermark. */ - public static StateTag watermarkStateInternal(String id) { - return new WatermarkStateTagInternal(new StructuredId(id)); + public static StateTag watermarkStateInternal( + String id, OutputTimeFn outputTimeFn) { + return new WatermarkStateTagInternal(new StructuredId(id), outputTimeFn); } /** @@ -349,15 +352,23 @@ protected StateTag> asKind(StateKind kind) { } } - private static class WatermarkStateTagInternal extends StateTagBase { + private static class WatermarkStateTagInternal + extends StateTagBase { - private WatermarkStateTagInternal(StructuredId id) { + /** + * When multiple output times are added to hold the watermark, this determines how they are + * combined, and also the behavior when merging windows. + */ + private final OutputTimeFn outputTimeFn; + + private WatermarkStateTagInternal(StructuredId id, OutputTimeFn outputTimeFn) { super(id); + this.outputTimeFn = outputTimeFn; } @Override public WatermarkStateInternal bind(StateBinder visitor) { - return visitor.bindWatermark(this); + return visitor.bindWatermark(this, outputTimeFn); } @Override @@ -371,7 +382,8 @@ public boolean equals(Object obj) { } WatermarkStateTagInternal that = (WatermarkStateTagInternal) obj; - return Objects.equals(this.id, that.id); + return Objects.equals(this.id, that.id) + && Objects.equals(this.outputTimeFn, that.outputTimeFn); } @Override @@ -381,7 +393,7 @@ public int hashCode() { @Override protected StateTag asKind(StateKind kind) { - return new WatermarkStateTagInternal(id.asKind(kind)); + return new WatermarkStateTagInternal(id.asKind(kind), outputTimeFn); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/WatermarkStateInternal.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/WatermarkStateInternal.java index 6d26746831441..29ed90b0b8141 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/WatermarkStateInternal.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/WatermarkStateInternal.java @@ -17,14 +17,29 @@ import com.google.cloud.dataflow.sdk.annotations.Experimental; import com.google.cloud.dataflow.sdk.annotations.Experimental.Kind; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns; import org.joda.time.Instant; /** - * State for holding up the watermark to the minimum of input {@code Instant}s. + * A {@link State} accepting and aggregating output timestamps, which determines + * the time to which the output watermark must be held. * - *

This is intended for internal use only. The watermark will be held up based on the - * values that are added and only released when items are cleared. + *

For internal use only. This API may change at any time. */ @Experimental(Kind.STATE) -public interface WatermarkStateInternal extends MergeableState {} +public interface WatermarkStateInternal extends MergeableState { + + /** + * Release any holds that have become extraneous so they do not prevent progress of the + * output watermark. + * + *

For example, when using {@link OutputTimeFns#outputAtEndOfWindow()}, there will be holds + * in place at the end of every initial window that merges into the result window. These holds + * need to be released. It is implementation-dependent how (or whether) this happens. + * + *

This method is permitted to be "best effort" but should always try to release holds + * as far as possible to allow the output watermark to make progress. + */ + void releaseExtraneousHolds(); +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java index 6b00db9ee9f6b..fe80f90a175fd 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java @@ -28,6 +28,7 @@ import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill.TagList; import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill.TagValue; import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; import com.google.cloud.dataflow.sdk.util.state.BagState; @@ -40,6 +41,7 @@ import com.google.cloud.dataflow.sdk.util.state.ValueState; import com.google.cloud.dataflow.sdk.util.state.WatermarkStateInternal; import com.google.common.base.Supplier; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.SettableFuture; import com.google.protobuf.ByteString; @@ -61,7 +63,6 @@ */ @RunWith(JUnit4.class) public class WindmillStateInternalsTest { - private static final StateNamespace NAMESPACE = new StateNamespaceForTest("ns"); private static final String STATE_FAMILY = "family"; @@ -93,8 +94,7 @@ public void setUp() { underTest = new WindmillStateInternals(STATE_FAMILY, true, mockReader, readStateSupplier); } - private void waitAndSet( - final SettableFuture future, final T value, final long millis) { + private void waitAndSet(final SettableFuture future, final T value, final long millis) { new Thread(new Runnable() { @Override public void run() { @@ -292,11 +292,11 @@ public void testCombiningAddBeforeRead() throws Exception { value.add(5); value.add(6); - waitAndSet(future, Arrays.asList(new int[]{8}, new int[]{10}), 200); + waitAndSet(future, Arrays.asList(new int[] {8}, new int[] {10}), 200); assertThat(result.read(), Matchers.equalTo(29)); // That get "compressed" the combiner. So, the underlying future should change: - future.set(Arrays.asList(new int[]{29})); + future.set(Arrays.asList(new int[] {29})); value.add(2); assertThat(result.read(), Matchers.equalTo(31)); @@ -330,7 +330,7 @@ public void testCombiningIsEmpty() throws Exception { StateContents result = value.isEmpty(); Mockito.verify(mockReader).listFuture(key(NAMESPACE, "combining"), STATE_FAMILY, accumCoder); - waitAndSet(future, Arrays.asList(new int[]{29}), 200); + waitAndSet(future, Arrays.asList(new int[] {29}), 200); assertThat(result.read(), Matchers.is(false)); } @@ -364,9 +364,10 @@ public void testCombiningAddPersist() throws Exception { TagList listUpdates = commitBuilder.getListUpdates(0); assertEquals(key(NAMESPACE, "combining"), listUpdates.getTag()); assertEquals(1, listUpdates.getValuesCount()); - assertEquals(11, - CoderUtils.decodeFromByteArray(accumCoder, - listUpdates.getValues(0).getData().substring(1).toByteArray())[0]); + assertEquals( + 11, + CoderUtils.decodeFromByteArray( + accumCoder, listUpdates.getValues(0).getData().substring(1).toByteArray())[0]); // Blind adds should not need to read the future. Mockito.verify(mockReader).startBatchAndBlock(); @@ -395,9 +396,10 @@ public void testCombiningClearPersist() throws Exception { TagList listUpdates = commitBuilder.getListUpdates(1); assertEquals(key(NAMESPACE, "combining"), listUpdates.getTag()); assertEquals(1, listUpdates.getValuesCount()); - assertEquals(11, - CoderUtils.decodeFromByteArray(accumCoder, - listUpdates.getValues(0).getData().substring(1).toByteArray())[0]); + assertEquals( + 11, + CoderUtils.decodeFromByteArray( + accumCoder, listUpdates.getValues(0).getData().substring(1).toByteArray())[0]); // Blind adds should not need to read the future. Mockito.verify(mockReader).listFuture(key(NAMESPACE, "combining"), STATE_FAMILY, accumCoder); @@ -406,8 +408,9 @@ public void testCombiningClearPersist() throws Exception { } @Test - public void testWatermarkAddBeforeRead() throws Exception { - StateTag addr = StateTags.watermarkStateInternal("watermark"); + public void testWatermarkAddBeforeReadEarliest() throws Exception { + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); SettableFuture future = SettableFuture.create(); @@ -428,9 +431,59 @@ public void testWatermarkAddBeforeRead() throws Exception { Mockito.verifyNoMoreInteractions(mockReader); } + @Test + public void testWatermarkAddBeforeReadLatest() throws Exception { + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtLatestInputTimestamp()); + WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); + + SettableFuture future = SettableFuture.create(); + when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)).thenReturn(future); + + StateContents result = bag.get(); + + bag.add(new Instant(3000)); + waitAndSet(future, new Instant(2000), 200); + assertThat(result.read(), Matchers.equalTo(new Instant(3000))); + + Mockito.verify(mockReader).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); + Mockito.verifyNoMoreInteractions(mockReader); + + // Adding another value doesn't create another future, but does update the result. + bag.add(new Instant(3000)); + assertThat(result.read(), Matchers.equalTo(new Instant(3000))); + Mockito.verifyNoMoreInteractions(mockReader); + } + + @Test + public void testWatermarkAddBeforeReadEndOfWindow() throws Exception { + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtEndOfWindow()); + WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); + + SettableFuture future = SettableFuture.create(); + when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)).thenReturn(future); + + StateContents result = bag.get(); + + bag.add(new Instant(3000)); + waitAndSet(future, new Instant(3000), 200); + assertThat(result.read(), Matchers.equalTo(new Instant(3000))); + + Mockito.verify(mockReader).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); + Mockito.verifyNoMoreInteractions(mockReader); + + // Adding another value doesn't create another future, but does update the result. + bag.add(new Instant(3000)); + assertThat(result.read(), Matchers.equalTo(new Instant(3000))); + Mockito.verifyNoMoreInteractions(mockReader); + } + @Test public void testWatermarkClearBeforeRead() throws Exception { - StateTag addr = StateTags.watermarkStateInternal("watermark"); + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); + WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); bag.clear(); @@ -445,7 +498,8 @@ public void testWatermarkClearBeforeRead() throws Exception { @Test public void testWatermarkIsEmptyWindmillHasData() throws Exception { - StateTag addr = StateTags.watermarkStateInternal("watermark"); + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); SettableFuture future = SettableFuture.create(); @@ -459,7 +513,8 @@ public void testWatermarkIsEmptyWindmillHasData() throws Exception { @Test public void testWatermarkIsEmpty() throws Exception { - StateTag addr = StateTags.watermarkStateInternal("watermark"); + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); SettableFuture future = SettableFuture.create(); @@ -473,7 +528,8 @@ public void testWatermarkIsEmpty() throws Exception { @Test public void testWatermarkIsEmptyAfterClear() throws Exception { - StateTag addr = StateTags.watermarkStateInternal("watermark"); + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); bag.clear(); @@ -486,8 +542,9 @@ public void testWatermarkIsEmptyAfterClear() throws Exception { } @Test - public void testWatermarkPersist() throws Exception { - StateTag addr = StateTags.watermarkStateInternal("watermark"); + public void testWatermarkPersistEarliest() throws Exception { + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); bag.add(new Instant(1000)); @@ -508,9 +565,118 @@ public void testWatermarkPersist() throws Exception { Mockito.verifyNoMoreInteractions(mockReader); } + @Test + public void testWatermarkPersistLatestEmpty() throws Exception { + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtLatestInputTimestamp()); + WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); + + bag.add(new Instant(1000)); + bag.add(new Instant(2000)); + + when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)) + .thenReturn(Futures.immediateFuture(null)); + + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + underTest.persist(commitBuilder); + + assertEquals(1, commitBuilder.getWatermarkHoldsCount()); + + Windmill.WatermarkHold watermarkHold = commitBuilder.getWatermarkHolds(0); + assertEquals(key(NAMESPACE, "watermark"), watermarkHold.getTag()); + assertEquals(TimeUnit.MILLISECONDS.toMicros(2000), watermarkHold.getTimestamps(0)); + + // Blind adds should not need to read the future. + Mockito.verify(mockReader).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); + Mockito.verify(mockReader).startBatchAndBlock(); + Mockito.verifyNoMoreInteractions(mockReader); + } + + @Test + public void testWatermarkPersistLatestWindmillWins() throws Exception { + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtLatestInputTimestamp()); + WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); + + bag.add(new Instant(1000)); + bag.add(new Instant(2000)); + + when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)) + .thenReturn(Futures.immediateFuture(new Instant(4000))); + + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + underTest.persist(commitBuilder); + + assertEquals(1, commitBuilder.getWatermarkHoldsCount()); + + Windmill.WatermarkHold watermarkHold = commitBuilder.getWatermarkHolds(0); + assertEquals(key(NAMESPACE, "watermark"), watermarkHold.getTag()); + assertEquals(TimeUnit.MILLISECONDS.toMicros(4000), watermarkHold.getTimestamps(0)); + + // Blind adds should not need to read the future. + Mockito.verify(mockReader).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); + Mockito.verify(mockReader).startBatchAndBlock(); + Mockito.verifyNoMoreInteractions(mockReader); + } + + @Test + public void testWatermarkPersistLatestLocalAdditionsWin() throws Exception { + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtLatestInputTimestamp()); + WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); + + bag.add(new Instant(1000)); + bag.add(new Instant(2000)); + + when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)) + .thenReturn(Futures.immediateFuture(new Instant(500))); + + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + underTest.persist(commitBuilder); + + assertEquals(1, commitBuilder.getWatermarkHoldsCount()); + + Windmill.WatermarkHold watermarkHold = commitBuilder.getWatermarkHolds(0); + assertEquals(key(NAMESPACE, "watermark"), watermarkHold.getTag()); + assertEquals(TimeUnit.MILLISECONDS.toMicros(2000), watermarkHold.getTimestamps(0)); + + // Blind adds should not need to read the future. + Mockito.verify(mockReader).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); + Mockito.verify(mockReader).startBatchAndBlock(); + Mockito.verifyNoMoreInteractions(mockReader); + } + + @Test + public void testWatermarkPersistEndOfWindow() throws Exception { + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtEndOfWindow()); + WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); + + bag.add(new Instant(2000)); + bag.add(new Instant(2000)); + + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + underTest.persist(commitBuilder); + + assertEquals(1, commitBuilder.getWatermarkHoldsCount()); + + Windmill.WatermarkHold watermarkHold = commitBuilder.getWatermarkHolds(0); + assertEquals(key(NAMESPACE, "watermark"), watermarkHold.getTag()); + assertEquals(TimeUnit.MILLISECONDS.toMicros(2000), watermarkHold.getTimestamps(0)); + + // Blind adds should not need to read the future. + Mockito.verify(mockReader).startBatchAndBlock(); + Mockito.verifyNoMoreInteractions(mockReader); + } + @Test public void testWatermarkClearPersist() throws Exception { - StateTag addr = StateTags.watermarkStateInternal("watermark"); + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); bag.add(new Instant(500)); @@ -522,26 +688,24 @@ public void testWatermarkClearPersist() throws Exception { Windmill.WorkItemCommitRequest.newBuilder(); underTest.persist(commitBuilder); - assertEquals(2, commitBuilder.getWatermarkHoldsCount()); - - Windmill.WatermarkHold clear = commitBuilder.getWatermarkHolds(0); - assertEquals(key(NAMESPACE, "watermark"), clear.getTag()); - assertEquals(0, clear.getTimestampsCount()); + assertEquals(1, commitBuilder.getWatermarkHoldsCount()); - Windmill.WatermarkHold update = commitBuilder.getWatermarkHolds(1); - assertEquals(key(NAMESPACE, "watermark"), update.getTag()); - assertEquals(1, update.getTimestampsCount()); - assertEquals(TimeUnit.MILLISECONDS.toMicros(1000), update.getTimestamps(0)); + Windmill.WatermarkHold clearAndUpdate = commitBuilder.getWatermarkHolds(0); + assertEquals(key(NAMESPACE, "watermark"), clearAndUpdate.getTag()); + assertEquals(1, clearAndUpdate.getTimestampsCount()); + assertEquals(key(NAMESPACE, "watermark"), clearAndUpdate.getTag()); + assertEquals(1, clearAndUpdate.getTimestampsCount()); + assertEquals(TimeUnit.MILLISECONDS.toMicros(1000), clearAndUpdate.getTimestamps(0)); // Clearing requires reading the future. - Mockito.verify(mockReader).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @Test public void testWatermarkPersistEmpty() throws Exception { - StateTag addr = StateTags.watermarkStateInternal("watermark"); + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); bag.add(new Instant(500)); @@ -556,15 +720,37 @@ public void testWatermarkPersistEmpty() throws Exception { } @Test - public void testWatermarkNoStateFamilies() throws Exception { + public void testWatermarkNoStateFamiliesEarliest() throws Exception { underTest = new WindmillStateInternals(STATE_FAMILY, false, mockReader, readStateSupplier); - StateTag addr = StateTags.watermarkStateInternal("watermark"); + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); bag.get(); Mockito.verify(mockReader).watermarkFuture(key(STATE_FAMILY, NAMESPACE, "watermark"), ""); } + @Test + public void testWatermarkNoStateFamiliesLatest() throws Exception { + underTest = new WindmillStateInternals(STATE_FAMILY, false, mockReader, readStateSupplier); + + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtLatestInputTimestamp()); + WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); + bag.get(); + Mockito.verify(mockReader).watermarkFuture(key(STATE_FAMILY, NAMESPACE, "watermark"), ""); + } + + @Test + public void testWatermarkNoStateFamiliesEndOfWindow() throws Exception { + underTest = new WindmillStateInternals(STATE_FAMILY, false, mockReader, readStateSupplier); + + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtLatestInputTimestamp()); + WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); + bag.get(); + Mockito.verify(mockReader).watermarkFuture(key(STATE_FAMILY, NAMESPACE, "watermark"), ""); + } @Test public void testValueSetBeforeRead() throws Exception { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java index b48051bd3d4b0..90aefa01de40a 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.transforms; import static com.google.cloud.dataflow.sdk.TestUtils.KvMatcher.isKv; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertThat; @@ -36,6 +37,7 @@ import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.util.NoopPathValidator; @@ -43,8 +45,10 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PBegin; import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TimestampedValue; import org.joda.time.Duration; +import org.joda.time.Instant; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -361,6 +365,61 @@ public PCollection> apply(PBegin input) { input.apply("GroupByKey", GroupByKey.create()); } + /** + * Tests that when two elements are combined via a GroupByKey their output timestamp agrees + * with the windowing function customized to actually be the same as the default, the earlier of + * the two values. + */ + @Test + public void testOutputTimeFnEarliest() { + Pipeline pipeline = TestPipeline.create(); + + pipeline.apply( + Create.timestamped( + TimestampedValue.of(KV.of(0, "hello"), new Instant(0)), + TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10)))) + .apply(Window.>into(FixedWindows.of(Duration.standardMinutes(10))) + .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())) + .apply(GroupByKey.create()) + .apply(ParDo.of(new AssertTimestamp(new Instant(0)))); + + pipeline.run(); + } + + + /** + * Tests that when two elements are combined via a GroupByKey their output timestamp agrees + * with the windowing function customized to use the latest value. + */ + @Test + public void testOutputTimeFnLatest() { + Pipeline pipeline = TestPipeline.create(); + + pipeline.apply( + Create.timestamped( + TimestampedValue.of(KV.of(0, "hello"), new Instant(0)), + TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10)))) + .apply(Window.>into(FixedWindows.of(Duration.standardMinutes(10))) + .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp())) + .apply(GroupByKey.create()) + .apply(ParDo.of(new AssertTimestamp(new Instant(10)))); + + pipeline.run(); + } + + private static class AssertTimestamp extends DoFn, Void> { + private final Instant timestamp; + + public AssertTimestamp(Instant timestamp) { + this.timestamp = timestamp; + } + + @Override + public void processElement(ProcessContext c) throws Exception { + assertThat(c.timestamp(), equalTo(timestamp)); + } + } + @Test public void testGroupByKeyGetName() { Assert.assertEquals("GroupByKey", GroupByKey.create().getName()); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java index 8da9b1cbcc3a3..91049cd9d9d8f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java @@ -18,10 +18,15 @@ import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.runWindowFn; import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.set; + +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.containsString; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils; +import com.google.common.collect.ImmutableList; import org.joda.time.Duration; import org.joda.time.Instant; @@ -31,6 +36,7 @@ import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; @@ -100,11 +106,51 @@ public void testEquality() { Sessions.withGapDuration(new Duration(20)))); } + /** + * Validates that the output timestamp for aggregate data falls within the acceptable range. + */ @Test public void testValidOutputTimes() throws Exception { for (long timestamp : Arrays.asList(200, 800, 700)) { WindowFnTestUtils.validateGetOutputTimestamp( - Sessions.withGapDuration(new Duration(500)), timestamp); + Sessions.withGapDuration(Duration.millis(500)), timestamp); + } + } + + /** + * Test to confirm that {@link Sessions} with the default {@link OutputTimeFn} holds up the + * watermark potentially indefinitely. + */ + @Test + public void testInvalidOutputAtEarliest() throws Exception { + try { + WindowFnTestUtils.validateGetOutputTimestamps( + Sessions.withGapDuration(Duration.millis(10)), + OutputTimeFns.outputAtEarliestInputTimestamp(), + ImmutableList.of( + (List) ImmutableList.of(1L, 3L), + (List) ImmutableList.of(0L, 5L, 10L, 15L, 20L))); + } catch (AssertionError exc) { + assertThat( + exc.getMessage(), + // These are the non-volatile pieces of the error message that a timestamp + // was not greater than what it should be. + allOf(containsString("a value greater than"), containsString("was less than"))); } } + + /** + * When a user explicitly requests per-key aggregate values have their derived timestamp to be + * the end of the window (instead of the earliest possible), the session here should not hold + * each other up, even though they overlap. + */ + @Test + public void testValidOutputAtEndTimes() throws Exception { + WindowFnTestUtils.validateGetOutputTimestamps( + Sessions.withGapDuration(Duration.millis(10)), + OutputTimeFns.outputAtEndOfWindow(), + ImmutableList.of( + (List) ImmutableList.of(1L, 3L), + (List) ImmutableList.of(0L, 5L, 10L, 15L, 20L))); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowTest.java index d22e7b896ca68..ac0e15886374f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowTest.java @@ -16,20 +16,29 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; +import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.when; +import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.GroupByKey; +import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.util.WindowingStrategy; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.TimestampedValue; import org.hamcrest.Matchers; import org.joda.time.Duration; +import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -37,14 +46,16 @@ import org.junit.runners.JUnit4; import org.mockito.Mockito; +import java.io.Serializable; + /** * Tests for {@link Window}. */ @RunWith(JUnit4.class) -public class WindowTest { +public class WindowTest implements Serializable { @Rule - public ExpectedException thrown = ExpectedException.none(); + public transient ExpectedException thrown = ExpectedException.none(); @Test public void testWindowIntoSetWindowfn() { @@ -159,4 +170,53 @@ public void testMissingLateness() { .apply("Window", Window.into(fixed10)) .apply("Trigger", Window.triggering(trigger)); } + + /** + * Tests that when two elements are combined via a GroupByKey their output timestamp agrees + * with the windowing function default, the earlier of the two values. + */ + @Test + public void testOutputTimeFnDefault() { + Pipeline pipeline = TestPipeline.create(); + + pipeline.apply( + Create.timestamped( + TimestampedValue.of(KV.of(0, "hello"), new Instant(0)), + TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10)))) + .apply(Window.>into(FixedWindows.of(Duration.standardMinutes(10)))) + .apply(GroupByKey.create()) + .apply(ParDo.of(new DoFn>, Void>() { + @Override + public void processElement(ProcessContext c) throws Exception { + assertThat(c.timestamp(), equalTo(new Instant(0))); + } + })); + + pipeline.run(); + } + + /** + * Tests that when two elements are combined via a GroupByKey their output timestamp agrees + * with the windowing function customized to use the end of the window. + */ + @Test + public void testOutputTimeFnEndOfWindow() { + Pipeline pipeline = TestPipeline.create(); + + pipeline.apply( + Create.timestamped( + TimestampedValue.of(KV.of(0, "hello"), new Instant(0)), + TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10)))) + .apply(Window.>into(FixedWindows.of(Duration.standardMinutes(10))) + .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow())) + .apply(GroupByKey.create()) + .apply(ParDo.of(new DoFn>, Void>() { + @Override + public void processElement(ProcessContext c) throws Exception { + assertThat(c.timestamp(), equalTo(new Instant(10 * 60 * 1000 - 1))); + } + })); + + pipeline.run(); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsAndCombineDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsAndCombineDoFnTest.java index 6c9cfa61f0d1a..11df7e19d099e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsAndCombineDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsAndCombineDoFnTest.java @@ -20,7 +20,6 @@ import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; import com.google.cloud.dataflow.sdk.transforms.Sum; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsProperties.GroupAlsoByWindowsDoFnFactory; import org.junit.Test; @@ -45,11 +44,8 @@ public GABWAndCombineDoFnFactory( public GroupAlsoByWindowsDoFn forStrategy(WindowingStrategy windowingStrategy) { - @SuppressWarnings("unchecked") - WindowFn windowFn = (WindowFn) windowingStrategy.getWindowFn(); - return new GroupAlsoByWindowsAndCombineDoFn( - windowFn, + windowingStrategy, keyedCombineFn); } } @@ -78,4 +74,12 @@ public void testCombinesIntoSessions() throws Exception { combineFn); } + @Test + public void testCombinesIntoSessionsWithEndOfWindowTimestamp() throws Exception { + CombineFn combineFn = new Sum.SumLongFn(); + + GroupAlsoByWindowsProperties.combinesElementsPerSessionWithEndOfWindowTimestamp( + new GABWAndCombineDoFnFactory<>(combineFn.asKeyedFn()), + combineFn); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsProperties.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsProperties.java index d725e0caf3c74..10af67d68b9b7 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsProperties.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsProperties.java @@ -19,7 +19,6 @@ import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import com.google.cloud.dataflow.sdk.TestUtils.KvMatcher; @@ -29,6 +28,8 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; @@ -36,12 +37,15 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import org.joda.time.Duration; import org.joda.time.Instant; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; import java.util.List; /** @@ -74,52 +78,29 @@ public static void emptyInputEmptyOutput( WindowingStrategy windowingStrategy = WindowingStrategy.of(FixedWindows.of(Duration.millis(10))); - TupleTag> outputTag = new TupleTag<>(); - DoFnRunner.ListOutputManager outputManager = new DoFnRunner.ListOutputManager(); - - DoFnRunner> runner = - makeRunner( - gabwFactory.forStrategy(windowingStrategy), - windowingStrategy, - outputTag, - outputManager); - runner.startBundle(); + List result = runGABW( + gabwFactory, + windowingStrategy, + (K) null, // key should never be used + Collections.>emptyList()); - runner.finishBundle(); - - List>> result = outputManager.getOutput(outputTag); - - assertEquals(0, result.size()); + assertThat(result.size(), equalTo(0)); } /** * Tests that for a simple sequence of elements on the same key, the given GABW implementation * correctly groups them according to fixed windows. - * - *

The notable specialized property of this input is that each element occurs in a single - * window. */ public static void groupsElementsIntoFixedWindows( GroupAlsoByWindowsDoFnFactory> gabwFactory) throws Exception { - TupleTag>> outputTag = new TupleTag<>(); - DoFnRunner.ListOutputManager outputManager = new DoFnRunner.ListOutputManager(); WindowingStrategy windowingStrategy = WindowingStrategy.of(FixedWindows.of(Duration.millis(10))); - DoFnRunner>>, KV>> runner = - makeRunner( - gabwFactory.forStrategy(windowingStrategy), - windowingStrategy, - outputTag, - outputManager); - - runner.startBundle(); - - runner.processElement(WindowedValue.valueInEmptyWindows( - KV.of("k", (Iterable>) Arrays.asList( + List>>> result = + runGABW(gabwFactory, windowingStrategy, "key", WindowedValue.of( "v1", new Instant(1), @@ -134,24 +115,18 @@ public static void groupsElementsIntoFixedWindows( "v3", new Instant(13), Arrays.asList(window(10, 20)), - PaneInfo.NO_FIRING))))); + PaneInfo.NO_FIRING)); - runner.finishBundle(); - - List>>> result = outputManager.getOutput(outputTag); - - assertEquals(2, result.size()); + assertThat(result.size(), equalTo(2)); WindowedValue>> item0 = result.get(0); - assertEquals("k", item0.getValue().getKey()); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); - assertEquals(new Instant(1), item0.getTimestamp()); + assertThat(item0.getTimestamp(), equalTo(new Instant(1))); assertThat(item0.getWindows(), contains(window(0, 10))); WindowedValue>> item1 = result.get(1); - assertEquals("k", item1.getValue().getKey()); assertThat(item1.getValue().getValue(), contains("v3")); - assertEquals(new Instant(13), item1.getTimestamp()); + assertThat(item1.getTimestamp(), equalTo(new Instant(13))); assertThat(item1.getWindows(), contains(window(10, 20))); } @@ -166,22 +141,11 @@ public static void groupsElementsIntoSlidingWindows( GroupAlsoByWindowsDoFnFactory> gabwFactory) throws Exception { - TupleTag>> outputTag = new TupleTag<>(); - DoFnRunner.ListOutputManager outputManager = new DoFnRunner.ListOutputManager(); WindowingStrategy windowingStrategy = WindowingStrategy.of( SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10))); - DoFnRunner>>, KV>> runner = - makeRunner( - gabwFactory.forStrategy(windowingStrategy), - windowingStrategy, - outputTag, - outputManager); - - runner.startBundle(); - - runner.processElement(WindowedValue.valueInEmptyWindows( - KV.of("k", (Iterable>) Arrays.asList( + List>>> result = + runGABW(gabwFactory, windowingStrategy, "key", WindowedValue.of( "v1", new Instant(5), @@ -191,32 +155,25 @@ public static void groupsElementsIntoSlidingWindows( "v2", new Instant(15), Arrays.asList(window(0, 20), window(10, 30)), - PaneInfo.NO_FIRING))))); - - runner.finishBundle(); + PaneInfo.NO_FIRING)); - List>>> result = outputManager.getOutput(outputTag); - - assertEquals(3, result.size()); + assertThat(result.size(), equalTo(3)); WindowedValue>> item0 = result.get(0); - assertEquals("k", item0.getValue().getKey()); assertThat(item0.getValue().getValue(), contains("v1")); - assertEquals(new Instant(5), item0.getTimestamp()); + assertThat(item0.getTimestamp(), equalTo(new Instant(5))); assertThat(item0.getWindows(), contains(window(-10, 10))); WindowedValue>> item1 = result.get(1); - assertEquals("k", item1.getValue().getKey()); assertThat(item1.getValue().getValue(), containsInAnyOrder("v1", "v2")); - assertEquals(new Instant(10), item1.getTimestamp()); + assertThat(item1.getTimestamp(), equalTo(new Instant(10))); assertThat(item1.getWindows(), contains(window(0, 20))); WindowedValue>> item2 = result.get(2); - assertEquals("k", item2.getValue().getKey()); assertThat(item2.getValue().getValue(), contains("v2")); - assertEquals(new Instant(20), item2.getTimestamp()); + assertThat(item2.getTimestamp(), equalTo(new Instant(20))); assertThat(item2.getWindows(), contains(window(10, 30))); } @@ -232,22 +189,11 @@ public static void combinesElementsInSlidingWindows( CombineFn combineFn) throws Exception { - TupleTag> outputTag = new TupleTag<>(); - DoFnRunner.ListOutputManager outputManager = new DoFnRunner.ListOutputManager(); WindowingStrategy windowingStrategy = WindowingStrategy.of( SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10))); - DoFnRunner>>, KV> runner = - makeRunner( - gabwFactory.forStrategy(windowingStrategy), - windowingStrategy, - outputTag, - outputManager); - - runner.startBundle(); - - runner.processElement(WindowedValue.valueInEmptyWindows( - KV.of("k", (Iterable>) Arrays.asList( + List>> result = + runGABW(gabwFactory, windowingStrategy, "k", WindowedValue.of( 1L, new Instant(5), @@ -262,13 +208,9 @@ public static void combinesElementsInSlidingWindows( 4L, new Instant(18), Arrays.asList(window(0, 20), window(10, 30)), - PaneInfo.NO_FIRING))))); + PaneInfo.NO_FIRING)); - runner.finishBundle(); - - List>> result = outputManager.getOutput(outputTag); - - assertEquals(3, result.size()); + assertThat(result.size(), equalTo(3)); assertThat(result, contains( WindowMatchers.isSingleWindowedValue( @@ -302,22 +244,11 @@ public static void groupsIntoOverlappingNonmergingWindows( GroupAlsoByWindowsDoFnFactory> gabwFactory) throws Exception { - TupleTag>> outputTag = new TupleTag<>(); - DoFnRunner.ListOutputManager outputManager = new DoFnRunner.ListOutputManager(); WindowingStrategy windowingStrategy = WindowingStrategy.of(FixedWindows.of(Duration.millis(10))); - DoFnRunner>>, KV>> runner = - makeRunner( - gabwFactory.forStrategy(windowingStrategy), - windowingStrategy, - outputTag, - outputManager); - - runner.startBundle(); - - runner.processElement(WindowedValue.valueInEmptyWindows( - KV.of("k", (Iterable>) Arrays.asList( + List>>> result = + runGABW(gabwFactory, windowingStrategy, "key", WindowedValue.of( "v1", new Instant(1), @@ -332,25 +263,19 @@ public static void groupsIntoOverlappingNonmergingWindows( "v3", new Instant(4), Arrays.asList(window(0, 5)), - PaneInfo.NO_FIRING))))); - - runner.finishBundle(); + PaneInfo.NO_FIRING)); - List>>> result = outputManager.getOutput(outputTag); - - assertEquals(2, result.size()); + assertThat(result.size(), equalTo(2)); WindowedValue>> item0 = result.get(0); - assertEquals("k", item0.getValue().getKey()); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v3")); - assertEquals(new Instant(1), item0.getTimestamp()); + assertThat(item0.getTimestamp(), equalTo(new Instant(1))); assertThat(item0.getWindows(), contains(window(0, 5))); WindowedValue>> item1 = result.get(1); - assertEquals("k", item1.getValue().getKey()); assertThat(item1.getValue().getValue(), contains("v2")); - assertEquals(new Instant(4), item1.getTimestamp()); + assertThat(item1.getTimestamp(), equalTo(new Instant(4))); assertThat(item1.getWindows(), contains(window(1, 5))); } @@ -362,22 +287,11 @@ public static void groupsElementsInMergedSessions( GroupAlsoByWindowsDoFnFactory> gabwFactory) throws Exception { - TupleTag>> outputTag = new TupleTag<>(); - DoFnRunner.ListOutputManager outputManager = new DoFnRunner.ListOutputManager(); WindowingStrategy windowingStrategy = WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))); - DoFnRunner>>, KV>> runner = - makeRunner( - gabwFactory.forStrategy(windowingStrategy), - windowingStrategy, - outputTag, - outputManager); - - runner.startBundle(); - - runner.processElement(WindowedValue.valueInEmptyWindows( - KV.of("k", (Iterable>) Arrays.asList( + List>>> result = + runGABW(gabwFactory, windowingStrategy, "key", WindowedValue.of( "v1", new Instant(0), @@ -392,25 +306,19 @@ public static void groupsElementsInMergedSessions( "v3", new Instant(15), Arrays.asList(window(15, 25)), - PaneInfo.NO_FIRING))))); - - runner.finishBundle(); - - List>>> result = outputManager.getOutput(outputTag); + PaneInfo.NO_FIRING)); - assertEquals(2, result.size()); + assertThat(result.size(), equalTo(2)); WindowedValue>> item0 = result.get(0); - assertEquals("k", item0.getValue().getKey()); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); - assertEquals(new Instant(0), item0.getTimestamp()); + assertThat(item0.getTimestamp(), equalTo(new Instant(0))); assertThat(item0.getWindows(), contains(window(0, 15))); WindowedValue>> item1 = result.get(1); - assertEquals("k", item1.getValue().getKey()); assertThat(item1.getValue().getValue(), contains("v3")); - assertEquals(new Instant(15), item1.getTimestamp()); + assertThat(item1.getTimestamp(), equalTo(new Instant(15))); assertThat(item1.getWindows(), contains(window(15, 25))); } @@ -424,22 +332,11 @@ public static void combinesElementsPerSession( CombineFn combineFn) throws Exception { - TupleTag> outputTag = new TupleTag<>(); - DoFnRunner.ListOutputManager outputManager = new DoFnRunner.ListOutputManager(); WindowingStrategy windowingStrategy = WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))); - DoFnRunner>>, KV> runner = - makeRunner( - gabwFactory.forStrategy(windowingStrategy), - windowingStrategy, - outputTag, - outputManager); - - runner.startBundle(); - - runner.processElement(WindowedValue.valueInEmptyWindows( - KV.of("k", (Iterable>) Arrays.asList( + List>> result = + runGABW(gabwFactory, windowingStrategy, "k", WindowedValue.of( 1L, new Instant(0), @@ -454,11 +351,7 @@ public static void combinesElementsPerSession( 4L, new Instant(15), Arrays.asList(window(15, 25)), - PaneInfo.NO_FIRING))))); - - runner.finishBundle(); - - List>> result = outputManager.getOutput(outputTag); + PaneInfo.NO_FIRING)); assertThat(result, contains( WindowMatchers.isSingleWindowedValue( @@ -477,6 +370,340 @@ public static void combinesElementsPerSession( 25))); // window end } + /** + * Tests that for a simple sequence of elements on the same key, the given GABW implementation + * correctly groups them according to fixed windows and also sets the output timestamp + * according to the policy {@link OutputTimeFns#outputAtEndOfWindow()}. + */ + public static void groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp( + GroupAlsoByWindowsDoFnFactory> gabwFactory) + throws Exception { + + WindowingStrategy windowingStrategy = + WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) + .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow()); + + List>>> result = + runGABW(gabwFactory, windowingStrategy, "key", + WindowedValue.of( + "v1", + new Instant(1), + Arrays.asList(window(0, 10)), + PaneInfo.NO_FIRING), + WindowedValue.of( + "v2", + new Instant(2), + Arrays.asList(window(0, 10)), + PaneInfo.NO_FIRING), + WindowedValue.of( + "v3", + new Instant(13), + Arrays.asList(window(10, 20)), + PaneInfo.NO_FIRING)); + + assertThat(result.size(), equalTo(2)); + + WindowedValue>> item0 = result.get(0); + assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); + assertThat(item0.getTimestamp(), equalTo(window(0, 10).maxTimestamp())); + assertThat(item0.getTimestamp(), + equalTo(Iterables.getOnlyElement(item0.getWindows()).maxTimestamp())); + + WindowedValue>> item1 = result.get(1); + assertThat(item1.getValue().getValue(), contains("v3")); + assertThat(item1.getTimestamp(), equalTo(window(10, 20).maxTimestamp())); + assertThat(item1.getTimestamp(), + equalTo(Iterables.getOnlyElement(item1.getWindows()).maxTimestamp())); + } + + /** + * Tests that for a simple sequence of elements on the same key, the given GABW implementation + * correctly groups them according to fixed windows and also sets the output timestamp + * according to a custom {@link OutputTimeFn}. + */ + public static void groupsElementsIntoFixedWindowsWithCustomTimestamp( + GroupAlsoByWindowsDoFnFactory> gabwFactory) + throws Exception { + + WindowingStrategy windowingStrategy = + WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) + .withOutputTimeFn(new OutputTimeFn.Defaults() { + + @Override + public Instant assignOutputTime(Instant inputTimestamp, IntervalWindow window) { + return window.start(); + } + + @Override + public Instant combine(Instant outputTime, Instant otherOutputTime) { + return outputTime; + } + + @Override + public boolean dependsOnlyOnEarliestInputTimestamp() { + return true; + } + }); + + List>>> result = + runGABW(gabwFactory, windowingStrategy, "key", + WindowedValue.of( + "v1", + new Instant(1), + Arrays.asList(window(0, 10)), + PaneInfo.NO_FIRING), + WindowedValue.of( + "v2", + new Instant(2), + Arrays.asList(window(0, 10)), + PaneInfo.NO_FIRING), + WindowedValue.of( + "v3", + new Instant(13), + Arrays.asList(window(10, 20)), + PaneInfo.NO_FIRING)); + + assertThat(result.size(), equalTo(2)); + + WindowedValue>> item0 = result.get(0); + assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); + assertThat(item0.getWindows(), contains(window(0, 10))); + assertThat(item0.getTimestamp(), + equalTo(((IntervalWindow) Iterables.getOnlyElement(item0.getWindows())).start())); + + WindowedValue>> item1 = result.get(1); + assertThat(item1.getValue().getValue(), contains("v3")); + assertThat(item1.getWindows(), contains(window(10, 20))); + assertThat(item1.getTimestamp(), + equalTo(((IntervalWindow) Iterables.getOnlyElement(item1.getWindows())).start())); + } + + + /** + * Tests that for a simple sequence of elements on the same key, the given GABW implementation + * correctly groups them according to fixed windows and also sets the output timestamp + * according to the policy {@link OutputTimeFns#outputAtLatestInputTimestamp()}. + */ + public static void groupsElementsIntoFixedWindowsWithLatestTimestamp( + GroupAlsoByWindowsDoFnFactory> gabwFactory) + throws Exception { + + WindowingStrategy windowingStrategy = + WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) + .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp()); + + List>>> result = + runGABW(gabwFactory, windowingStrategy, "k", + WindowedValue.of( + "v1", + new Instant(1), + Arrays.asList(window(0, 10)), + PaneInfo.NO_FIRING), + WindowedValue.of( + "v2", + new Instant(2), + Arrays.asList(window(0, 10)), + PaneInfo.NO_FIRING), + WindowedValue.of( + "v3", + new Instant(13), + Arrays.asList(window(10, 20)), + PaneInfo.NO_FIRING)); + + assertThat(result.size(), equalTo(2)); + + WindowedValue>> item0 = result.get(0); + assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); + assertThat(item0.getWindows(), contains(window(0, 10))); + assertThat(item0.getTimestamp(), equalTo(new Instant(2))); + + WindowedValue>> item1 = result.get(1); + assertThat(item1.getValue().getValue(), contains("v3")); + assertThat(item1.getWindows(), contains(window(10, 20))); + assertThat(item1.getTimestamp(), equalTo(new Instant(13))); + } + + /** + * Tests that the given GABW implementation correctly groups elements into merged sessions + * with output timestamps at the end of the merged window. + */ + public static void groupsElementsInMergedSessionsWithEndOfWindowTimestamp( + GroupAlsoByWindowsDoFnFactory> gabwFactory) + throws Exception { + + WindowingStrategy windowingStrategy = + WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))) + .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow()); + + List>>> result = + runGABW(gabwFactory, windowingStrategy, "k", + WindowedValue.of( + "v1", + new Instant(0), + Arrays.asList(window(0, 10)), + PaneInfo.NO_FIRING), + WindowedValue.of( + "v2", + new Instant(5), + Arrays.asList(window(5, 15)), + PaneInfo.NO_FIRING), + WindowedValue.of( + "v3", + new Instant(15), + Arrays.asList(window(15, 25)), + PaneInfo.NO_FIRING)); + + assertThat(result.size(), equalTo(2)); + + WindowedValue>> item0 = result.get(0); + assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); + assertThat(item0.getWindows(), contains(window(0, 15))); + assertThat(item0.getTimestamp(), + equalTo(Iterables.getOnlyElement(item0.getWindows()).maxTimestamp())); + + WindowedValue>> item1 = result.get(1); + assertThat(item1.getValue().getValue(), contains("v3")); + assertThat(item1.getWindows(), contains(window(15, 25))); + assertThat(item1.getTimestamp(), + equalTo(Iterables.getOnlyElement(item1.getWindows()).maxTimestamp())); + } + + /** + * Tests that the given GABW implementation correctly groups elements into merged sessions + * with output timestamps at the end of the merged window. + */ + public static void groupsElementsInMergedSessionsWithLatestTimestamp( + GroupAlsoByWindowsDoFnFactory> gabwFactory) + throws Exception { + + WindowingStrategy windowingStrategy = + WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))) + .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp()); + + List>>> result = + runGABW(gabwFactory, windowingStrategy, "k", + WindowedValue.of( + "v1", + new Instant(0), + Arrays.asList(window(0, 10)), + PaneInfo.NO_FIRING), + WindowedValue.of( + "v2", + new Instant(5), + Arrays.asList(window(5, 15)), + PaneInfo.NO_FIRING), + WindowedValue.of( + "v3", + new Instant(15), + Arrays.asList(window(15, 25)), + PaneInfo.NO_FIRING)); + + assertThat(result.size(), equalTo(2)); + + WindowedValue>> item0 = result.get(0); + assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); + assertThat(item0.getWindows(), contains(window(0, 15))); + assertThat(item0.getTimestamp(), equalTo(new Instant(5))); + + WindowedValue>> item1 = result.get(1); + assertThat(item1.getValue().getValue(), contains("v3")); + assertThat(item1.getWindows(), contains(window(15, 25))); + assertThat(item1.getTimestamp(), equalTo(new Instant(15))); + } + + /** + * Tests that the given {@link GroupAlsoByWindowsDoFn} implementation combines elements per + * session window correctly according to the provided {@link CombineFn}. + */ + public static void combinesElementsPerSessionWithEndOfWindowTimestamp( + GroupAlsoByWindowsDoFnFactory gabwFactory, + CombineFn combineFn) + throws Exception { + + WindowingStrategy windowingStrategy = + WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))) + .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow()); + + + List>> result = + runGABW(gabwFactory, windowingStrategy, "k", + WindowedValue.of( + 1L, + new Instant(0), + Arrays.asList(window(0, 10)), + PaneInfo.NO_FIRING), + WindowedValue.of( + 2L, + new Instant(5), + Arrays.asList(window(5, 15)), + PaneInfo.NO_FIRING), + WindowedValue.of( + 4L, + new Instant(15), + Arrays.asList(window(15, 25)), + PaneInfo.NO_FIRING)); + + assertThat(result.size(), equalTo(2)); + + WindowedValue> item0 = result.get(0); + assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L)))); + assertThat(item0.getWindows(), contains(window(0, 15))); + assertThat(item0.getTimestamp(), + equalTo(Iterables.getOnlyElement(item0.getWindows()).maxTimestamp())); + + WindowedValue> item1 = result.get(1); + assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(4L)))); + assertThat(item1.getWindows(), contains(window(15, 25))); + assertThat(item1.getTimestamp(), + equalTo(Iterables.getOnlyElement(item1.getWindows()).maxTimestamp())); + } + + @SafeVarargs + private static + List>> runGABW( + GroupAlsoByWindowsDoFnFactory gabwFactory, + WindowingStrategy windowingStrategy, + K key, + WindowedValue... values) { + return runGABW(gabwFactory, windowingStrategy, key, Arrays.asList(values)); + } + + private static + List>> runGABW( + GroupAlsoByWindowsDoFnFactory gabwFactory, + WindowingStrategy windowingStrategy, + K key, + Collection> values) { + + TupleTag> outputTag = new TupleTag<>(); + DoFnRunner.ListOutputManager outputManager = new DoFnRunner.ListOutputManager(); + + DoFnRunner>>, KV> runner = + makeRunner( + gabwFactory.forStrategy(windowingStrategy), + windowingStrategy, + outputTag, + outputManager); + + runner.startBundle(); + + if (values.size() > 0) { + runner.processElement(WindowedValue.valueInEmptyWindows( + KV.of(key, (Iterable>) values))); + } + + runner.finishBundle(); + + List>> result = outputManager.getOutput(outputTag); + + // Sanity check for corruption + for (WindowedValue> elem : result) { + assertThat(elem.getValue().getKey(), equalTo(key)); + } + + return result; + } + private static DoFnRunner>>, KV> makeRunner( @@ -503,4 +730,5 @@ public static void combinesElementsPerSession( private static BoundedWindow window(long start, long end) { return new IntervalWindow(new Instant(start), new Instant(end)); } + } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaIteratorsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaIteratorsDoFnTest.java index e53a4219124f3..41341443551cc 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaIteratorsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaIteratorsDoFnTest.java @@ -69,6 +69,28 @@ public void testGroupsIntoOverlappingNonmergingWindows() throws Exception { new GABWViaIteratorsDoFnFactory()); } + @Test + public void testGroupsElementsIntoFixedWindowsWithEndOfWindowTimestamp() throws Exception { + GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp( + new GABWViaIteratorsDoFnFactory()); + } + + @Test + public void testLatestTimestampNotSupported() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("OutputTimeFn"); + thrown.expectMessage("not support"); + + GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithLatestTimestamp( + new GABWViaIteratorsDoFnFactory()); + } + + @Test + public void testGroupsElementsIntoFixedWindowsWithCustomTimestamp() throws Exception { + GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithCustomTimestamp( + new GABWViaIteratorsDoFnFactory()); + } + @Test public void testMergingNotSupported() throws Exception { thrown.expect(IllegalArgumentException.class); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java index 43ab7685963ee..6109bf744210c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java @@ -132,4 +132,48 @@ public void testCombinesIntoSessions() throws Exception { appliedFn), combineFn); } + + @Test + public void testGroupsElementsIntoFixedWindowsWithEndOfWindowTimestamp() throws Exception { + GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp( + new BufferingGABWViaOutputBufferDoFnFactory(StringUtf8Coder.of())); + } + + @Test + public void testGroupsElementsIntoFixedWindowsWithLatestTimestamp() throws Exception { + GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithLatestTimestamp( + new BufferingGABWViaOutputBufferDoFnFactory(StringUtf8Coder.of())); + } + + @Test + public void testGroupsElementsIntoFixedWindowsWithCustomTimestamp() throws Exception { + GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithCustomTimestamp( + new BufferingGABWViaOutputBufferDoFnFactory(StringUtf8Coder.of())); + } + + @Test + public void testGroupsElementsIntoSessionsWithEndOfWindowTimestamp() throws Exception { + GroupAlsoByWindowsProperties.groupsElementsInMergedSessionsWithEndOfWindowTimestamp( + new BufferingGABWViaOutputBufferDoFnFactory(StringUtf8Coder.of())); + } + + @Test + public void testGroupsElementsIntoSessionsWithLatestTimestamp() throws Exception { + GroupAlsoByWindowsProperties.groupsElementsInMergedSessionsWithLatestTimestamp( + new BufferingGABWViaOutputBufferDoFnFactory(StringUtf8Coder.of())); + } + + @Test + public void testCombinesIntoSessionsWithEndOfWindowTimestamp() throws Exception { + CombineFn combineFn = new Sum.SumLongFn(); + AppliedCombineFn appliedFn = AppliedCombineFn.withInputCoder( + combineFn.asKeyedFn(), new CoderRegistry(), + KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())); + + GroupAlsoByWindowsProperties.combinesElementsPerSessionWithEndOfWindowTimestamp( + new CombiningGABWViaOutputBufferDoFnFactory<>( + StringUtf8Coder.of(), + appliedFn), + combineFn); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerTester.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerTester.java index 531488bf5a375..f5bef0f3da40f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerTester.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerTester.java @@ -190,7 +190,8 @@ public final void assertHasOnlyGlobalAndFinishedSetsAndPaneInfoFor(W... expected ImmutableSet.>of( TriggerRunner.FINISHED_BITS_TAG, PaneInfoTracker.PANE_INFO_TAG, - WatermarkHold.DATA_HOLD_TAG)); + WatermarkHold.watermarkHoldTagForOutputTimeFn( + objectStrategy.getOutputTimeFn()))); } public final void assertHasOnlyGlobalState() { @@ -202,7 +203,10 @@ public final void assertHasOnlyGlobalState() { public final void assertHasOnlyGlobalAndPaneInfoFor(W... expectedWindows) { assertHasOnlyGlobalAndAllowedTags( ImmutableSet.copyOf(expectedWindows), - ImmutableSet.>of(PaneInfoTracker.PANE_INFO_TAG, WatermarkHold.DATA_HOLD_TAG)); + ImmutableSet.>of( + PaneInfoTracker.PANE_INFO_TAG, + WatermarkHold.watermarkHoldTagForOutputTimeFn( + objectStrategy.getOutputTimeFn()))); } /** diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternalsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternalsTest.java index 4b1e162d4e9cd..ee09e08c37ebd 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternalsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternalsTest.java @@ -22,6 +22,9 @@ import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.VarIntCoder; import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns; import org.hamcrest.Matchers; import org.joda.time.Instant; @@ -37,6 +40,8 @@ @RunWith(JUnit4.class) public class InMemoryStateInternalsTest { + private static final BoundedWindow WINDOW_1 = new IntervalWindow(new Instant(0), new Instant(10)); + private static final BoundedWindow WINDOW_3 = new IntervalWindow(new Instant(5), new Instant(20)); private static final StateNamespace NAMESPACE_1 = new StateNamespaceForTest("ns1"); private static final StateNamespace NAMESPACE_2 = new StateNamespaceForTest("ns2"); private static final StateNamespace NAMESPACE_3 = new StateNamespaceForTest("ns3"); @@ -48,8 +53,15 @@ public class InMemoryStateInternalsTest { "sumInteger", VarIntCoder.of(), new Sum.SumIntegerFn()); private static final StateTag> STRING_BAG_ADDR = StateTags.bag("stringBag", StringUtf8Coder.of()); - private static final StateTag WATERMARK_BAG_ADDR = - StateTags.watermarkStateInternal("watermark"); + private static final StateTag WATERMARK_EARLIEST_ADDR = + StateTags.watermarkStateInternal("watermark", + OutputTimeFns.outputAtEarliestInputTimestamp()); + private static final StateTag WATERMARK_LATEST_ADDR = + StateTags.watermarkStateInternal("watermark", + OutputTimeFns.outputAtLatestInputTimestamp()); + private static final StateTag WATERMARK_EOW_ADDR = + StateTags.watermarkStateInternal("watermark", + OutputTimeFns.outputAtEndOfWindow()); InMemoryStateInternals underTest = new InMemoryStateInternals(); @@ -120,7 +132,7 @@ public void testMergeBagIntoSource() throws Exception { bag1.add("!"); BagState merged = underTest.mergedState( - Arrays.asList(NAMESPACE_1, NAMESPACE_2), NAMESPACE_1, STRING_BAG_ADDR); + Arrays.asList(NAMESPACE_1, NAMESPACE_2), NAMESPACE_1, STRING_BAG_ADDR, WINDOW_1); // Reading the merged bag gets both the contents assertThat(merged.get().read(), Matchers.containsInAnyOrder("Hello", "World", "!")); @@ -142,7 +154,7 @@ public void testMergeBagIntoNewNamespace() throws Exception { bag1.add("!"); BagState merged = underTest.mergedState( - Arrays.asList(NAMESPACE_1, NAMESPACE_2), NAMESPACE_3, STRING_BAG_ADDR); + Arrays.asList(NAMESPACE_1, NAMESPACE_2), NAMESPACE_3, STRING_BAG_ADDR, WINDOW_3); // Reading the merged bag gets both the contents assertThat(merged.get().read(), Matchers.containsInAnyOrder("Hello", "World", "!")); @@ -204,7 +216,7 @@ public void testMergeCombiningValueIntoSource() throws Exception { assertThat(value2.get().read(), Matchers.equalTo(10)); CombiningValueState merged = underTest.mergedState( - Arrays.asList(NAMESPACE_1, NAMESPACE_2), NAMESPACE_1, SUM_INTEGER_ADDR); + Arrays.asList(NAMESPACE_1, NAMESPACE_2), NAMESPACE_1, SUM_INTEGER_ADDR, WINDOW_1); assertThat(value1.get().read(), Matchers.equalTo(11)); assertThat(value2.get().read(), Matchers.equalTo(10)); @@ -233,7 +245,7 @@ public void testMergeCombiningValueIntoNewNamespace() throws Exception { assertThat(value2.get().read(), Matchers.equalTo(10)); CombiningValueState merged = underTest.mergedState( - Arrays.asList(NAMESPACE_1, NAMESPACE_2), NAMESPACE_3, SUM_INTEGER_ADDR); + Arrays.asList(NAMESPACE_1, NAMESPACE_2), NAMESPACE_3, SUM_INTEGER_ADDR, WINDOW_3); assertThat(value1.get().read(), Matchers.equalTo(11)); assertThat(value2.get().read(), Matchers.equalTo(10)); @@ -253,12 +265,12 @@ public void testMergeCombiningValueIntoNewNamespace() throws Exception { } @Test - public void testWatermarkState() throws Exception { - WatermarkStateInternal value = underTest.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + public void testWatermarkEarliestState() throws Exception { + WatermarkStateInternal value = underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR); // State instances are cached, but depend on the namespace. - assertEquals(value, underTest.state(NAMESPACE_1, WATERMARK_BAG_ADDR)); - assertFalse(value.equals(underTest.state(NAMESPACE_2, WATERMARK_BAG_ADDR))); + assertEquals(value, underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR)); + assertFalse(value.equals(underTest.state(NAMESPACE_2, WATERMARK_EARLIEST_ADDR))); assertThat(value.get().read(), Matchers.nullValue()); StateContents readFuture = value.get(); @@ -276,12 +288,58 @@ public void testWatermarkState() throws Exception { value.clear(); assertThat(readFuture.read(), Matchers.equalTo(null)); - assertThat(underTest.state(NAMESPACE_1, WATERMARK_BAG_ADDR), Matchers.sameInstance(value)); + assertThat(underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR), Matchers.sameInstance(value)); + } + + @Test + public void testWatermarkLatestState() throws Exception { + WatermarkStateInternal value = underTest.state(NAMESPACE_1, WATERMARK_LATEST_ADDR); + + // State instances are cached, but depend on the namespace. + assertEquals(value, underTest.state(NAMESPACE_1, WATERMARK_LATEST_ADDR)); + assertFalse(value.equals(underTest.state(NAMESPACE_2, WATERMARK_LATEST_ADDR))); + + assertThat(value.get().read(), Matchers.nullValue()); + StateContents readFuture = value.get(); + value.add(new Instant(2000)); + assertThat(readFuture.read(), Matchers.equalTo(new Instant(2000))); + assertThat(value.get().read(), Matchers.equalTo(new Instant(2000))); + + value.add(new Instant(3000)); + assertThat(readFuture.read(), Matchers.equalTo(new Instant(3000))); + assertThat(value.get().read(), Matchers.equalTo(new Instant(3000))); + + value.add(new Instant(1000)); + assertThat(readFuture.read(), Matchers.equalTo(new Instant(3000))); + assertThat(value.get().read(), Matchers.equalTo(new Instant(3000))); + + value.clear(); + assertThat(readFuture.read(), Matchers.equalTo(null)); + assertThat(underTest.state(NAMESPACE_1, WATERMARK_LATEST_ADDR), Matchers.sameInstance(value)); + } + + @Test + public void testWatermarkEndOfWindowState() throws Exception { + WatermarkStateInternal value = underTest.state(NAMESPACE_1, WATERMARK_EOW_ADDR); + + // State instances are cached, but depend on the namespace. + assertEquals(value, underTest.state(NAMESPACE_1, WATERMARK_EOW_ADDR)); + assertFalse(value.equals(underTest.state(NAMESPACE_2, WATERMARK_EOW_ADDR))); + + assertThat(value.get().read(), Matchers.nullValue()); + StateContents readFuture = value.get(); + value.add(new Instant(2000)); + assertThat(readFuture.read(), Matchers.equalTo(new Instant(2000))); + assertThat(value.get().read(), Matchers.equalTo(new Instant(2000))); + + value.clear(); + assertThat(readFuture.read(), Matchers.equalTo(null)); + assertThat(underTest.state(NAMESPACE_1, WATERMARK_EOW_ADDR), Matchers.sameInstance(value)); } @Test public void testWatermarkStateIsEmpty() throws Exception { - WatermarkStateInternal value = underTest.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + WatermarkStateInternal value = underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR); assertThat(value.isEmpty().read(), Matchers.is(true)); StateContents readFuture = value.isEmpty(); @@ -293,9 +351,9 @@ public void testWatermarkStateIsEmpty() throws Exception { } @Test - public void testMergeWatermarkIntoSource() throws Exception { - WatermarkStateInternal value1 = underTest.state(NAMESPACE_1, WATERMARK_BAG_ADDR); - WatermarkStateInternal value2 = underTest.state(NAMESPACE_2, WATERMARK_BAG_ADDR); + public void testMergeEarliestWatermarkIntoSource() throws Exception { + WatermarkStateInternal value1 = underTest.state(NAMESPACE_1, WATERMARK_EARLIEST_ADDR); + WatermarkStateInternal value2 = underTest.state(NAMESPACE_2, WATERMARK_EARLIEST_ADDR); value1.add(new Instant(3000)); value2.add(new Instant(5000)); @@ -303,7 +361,7 @@ public void testMergeWatermarkIntoSource() throws Exception { value2.add(new Instant(2000)); WatermarkStateInternal merged = underTest.mergedState( - Arrays.asList(NAMESPACE_1, NAMESPACE_2), NAMESPACE_1, WATERMARK_BAG_ADDR); + Arrays.asList(NAMESPACE_1, NAMESPACE_2), NAMESPACE_1, WATERMARK_EARLIEST_ADDR, WINDOW_1); assertThat(value1.get().read(), Matchers.equalTo(new Instant(3000))); assertThat(value2.get().read(), Matchers.equalTo(new Instant(2000))); @@ -316,4 +374,32 @@ public void testMergeWatermarkIntoSource() throws Exception { merged.add(new Instant(1000)); assertThat(merged.get().read(), Matchers.equalTo(new Instant(1000))); } + + @Test + public void testMergeLatestWatermarkIntoSource() throws Exception { + WatermarkStateInternal value1 = underTest.state(NAMESPACE_1, WATERMARK_LATEST_ADDR); + WatermarkStateInternal value2 = underTest.state(NAMESPACE_2, WATERMARK_LATEST_ADDR); + + value1.add(new Instant(3000)); + value2.add(new Instant(5000)); + value1.add(new Instant(4000)); + value2.add(new Instant(2000)); + + WatermarkStateInternal merged = underTest.mergedState( + Arrays.asList(NAMESPACE_1, NAMESPACE_2), NAMESPACE_1, WATERMARK_LATEST_ADDR, WINDOW_1); + + assertThat(value1.get().read(), Matchers.equalTo(new Instant(4000))); + assertThat(value2.get().read(), Matchers.equalTo(new Instant(5000))); + assertThat(merged.get().read(), Matchers.equalTo(new Instant(5000))); + + // Reading the merged value compressed the old values + assertThat(value1.get().read(), Matchers.equalTo(new Instant(5000))); + assertThat(value2.get().read(), Matchers.equalTo(null)); + + merged.add(new Instant(1000)); + assertThat(merged.get().read(), Matchers.equalTo(new Instant(5000))); + + merged.add(new Instant(7000)); + assertThat(merged.get().read(), Matchers.equalTo(new Instant(7000))); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/StateTagTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/StateTagTest.java index 4705ef1b0dea9..9b0fed7d11b51 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/StateTagTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/StateTagTest.java @@ -25,6 +25,7 @@ import com.google.cloud.dataflow.sdk.transforms.Max.MaxIntegerFn; import com.google.cloud.dataflow.sdk.transforms.Min; import com.google.cloud.dataflow.sdk.transforms.Min.MinIntegerFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns; import org.junit.Test; import org.junit.runner.RunWith; @@ -35,7 +36,6 @@ */ @RunWith(JUnit4.class) public class StateTagTest { - @Test public void testValueEquality() { StateTag fooVarInt1 = StateTags.value("foo", VarIntCoder.of()); @@ -62,12 +62,19 @@ public void testBagEquality() { @Test public void testWatermarkBagEquality() { - StateTag foo1 = StateTags.watermarkStateInternal("foo"); - StateTag foo2 = StateTags.watermarkStateInternal("foo"); - StateTag bar = StateTags.watermarkStateInternal("bar"); + StateTag foo1 = StateTags.watermarkStateInternal( + "foo", OutputTimeFns.outputAtEarliestInputTimestamp()); + StateTag foo2 = StateTags.watermarkStateInternal( + "foo", OutputTimeFns.outputAtEarliestInputTimestamp()); + StateTag bar = StateTags.watermarkStateInternal( + "bar", OutputTimeFns.outputAtEarliestInputTimestamp()); + + StateTag bar2 = StateTags.watermarkStateInternal( + "bar", OutputTimeFns.outputAtLatestInputTimestamp()); assertEquals(foo1, foo2); assertNotEquals(foo1, bar); + assertNotEquals(bar, bar2); } @Test From 777004ebfbe006373b0cf2584cf291d4840e7a4a Mon Sep 17 00:00:00 2001 From: robertwb Date: Tue, 24 Nov 2015 15:51:57 -0800 Subject: [PATCH 1193/1541] Minor javadoc updates. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108656541 --- .../dataflow/sdk/transforms/Aggregator.java | 2 +- .../sdk/transforms/AggregatorRetriever.java | 2 +- .../sdk/transforms/AppliedPTransform.java | 2 ++ .../sdk/transforms/ApproximateQuantiles.java | 7 ++-- .../dataflow/sdk/transforms/Combine.java | 32 +++++++++++++------ .../cloud/dataflow/sdk/transforms/Count.java | 8 +++-- .../cloud/dataflow/sdk/transforms/Max.java | 4 +++ .../cloud/dataflow/sdk/transforms/Min.java | 4 +++ .../cloud/dataflow/sdk/transforms/Sum.java | 4 +++ .../cloud/dataflow/sdk/transforms/Top.java | 4 +++ 10 files changed, 53 insertions(+), 16 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Aggregator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Aggregator.java index 71f330c3521c1..7e56ddac0dc7e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Aggregator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Aggregator.java @@ -41,7 +41,7 @@ * myAggregator = createAggregator("myAggregator", new Sum.SumIntegerFn()); * } * - * {@literal @}Override + * @Override * public void processElement(ProcessContext c) { * myAggregator.addValue(1); * } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/AggregatorRetriever.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/AggregatorRetriever.java index 97d5367626876..4bbea85f52a0c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/AggregatorRetriever.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/AggregatorRetriever.java @@ -19,7 +19,7 @@ import java.util.Collection; /** - * A class for extracting {@link Aggregator Aggregators} from {@link DoFn DoFns}. + * An internal class for extracting {@link Aggregator Aggregators} from {@link DoFn DoFns}. */ public final class AggregatorRetriever { private AggregatorRetriever() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/AppliedPTransform.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/AppliedPTransform.java index b86d5d723bbf2..7b3d87dfcf8be 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/AppliedPTransform.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/AppliedPTransform.java @@ -25,6 +25,8 @@ * Represents the application of a {@link PTransform} to a specific input to produce * a specific output. * + *

For internal use. + * * @param transform input type * @param transform output type * @param transform type diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java index 687c51e4d2326..f1b607c182a47 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java @@ -51,10 +51,13 @@ /** * {@code PTransform}s for getting an idea of a {@code PCollection}'s - * data distribution using approximate {@code N}-tiles, either - * globally or per-key. + * data distribution using approximate {@code N}-tiles (e.g. quartiles, + * percentiles, etc.), either globally or per-key. */ public class ApproximateQuantiles { + private ApproximateQuantiles() { + // do not instantiate + } /** * Returns a {@code PTransform} that takes a {@code PCollection} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java index 835081b273adf..16d12e2350499 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -66,8 +66,14 @@ /** * {@code PTransform}s for combining {@code PCollection} elements * globally and per-key. + * + *

See the documentation + * for how to use the operations in this class. */ public class Combine { + private Combine() { + // do not instantiate + } /** * Returns a {@link Globally Combine.Globally} {@code PTransform} @@ -396,8 +402,9 @@ public AccumT compact(AccumT accumulator) { * Applies this {@code CombineFn} to a collection of input values * to produce a combined output value. * - *

Useful when testing the behavior of a {@code CombineFn} - * separately from a {@code Combine} transform. + *

Useful when using a {@code CombineFn} separately from a + * {@code Combine} transform. Does not invoke the + * {@link mergeAccumulators} operation. */ public OutputT apply(Iterable inputs) { AccumT accum = createAccumulator(); @@ -621,8 +628,7 @@ public Coder getDefaultOutputCoder(CoderRegistry registry, Coder inputCode /** * Holds a single value value of type {@code V} which may or may not be present. * - *

Used only as a private accumulator class. The type appears in public interfaces, but from - * a public perspective, it has no accessible members. + *

Used only as a private accumulator class. */ public static class Holder { private V value; @@ -683,7 +689,7 @@ public void verifyDeterministic() throws NonDeterministicException { /** * An abstract subclass of {@link CombineFn} for implementing combiners that are more - * easily expressed as binary operations on ints. + * easily and efficiently expressed as binary operations on ints. */ public abstract static class BinaryCombineIntegerFn extends CombineFn { @@ -763,7 +769,7 @@ public Counter getCounter(String name) { /** * An abstract subclass of {@link CombineFn} for implementing combiners that are more - * easily expressed as binary operations on longs. + * easily and efficiently expressed as binary operations on longs. */ public abstract static class BinaryCombineLongFn extends CombineFn { /** @@ -841,7 +847,7 @@ public Counter getCounter(String name) { /** * An abstract subclass of {@link CombineFn} for implementing combiners that are more - * easily expressed as binary operations on doubles. + * easily and efficiently expressed as binary operations on doubles. */ public abstract static class BinaryCombineDoubleFn extends CombineFn { @@ -1373,8 +1379,8 @@ public Globally named(String name) { * Returns a {@link PTransform} that produces a {@code PCollectionView} * whose elements are the result of combining elements per-window in * the input {@code PCollection}. If a value is requested from the view - * for a window that is not present, the result of calling the {@code CombineFn} - * on empty input will returned. + * for a window that is not present, the result of applying the {@code CombineFn} + * to an empty input set will be returned. */ public GloballyAsSingletonView asSingletonView() { return new GloballyAsSingletonView<>(fn, insertDefault, fanout); @@ -1382,7 +1388,8 @@ public GloballyAsSingletonView asSingletonView() { /** * Returns a {@link PTransform} identical to this, but that does not attempt to - * provide a default value in the case of empty input. + * provide a default value in the case of empty input. Required when the input + * is not globally windowed and the output is not being used as a side input. */ public Globally withoutDefaults() { return new Globally<>(name, fn, false, fanout); @@ -2125,6 +2132,11 @@ public void processElement(ProcessContext c) { return output; } + /** + * Returns the {@link CombineFn} bound to its coders. + * + *

For internal use. + */ public AppliedCombineFn getAppliedFn( CoderRegistry registry, Coder>> inputCoder) { KvCoder kvCoder = getKvCoder(inputCoder); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java index 9b29088cbe2ed..ffa11d13a3c91 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Count.java @@ -24,10 +24,14 @@ * {@code PTransorm}s to count the elements in a {@link PCollection}. * *

{@link Count#perElement()} can be used to count the number of occurrences of each - * distinct element in the PCollection. {@link Count#globally()} can - * be used to count the total number of elements in a PCollection. + * distinct element in the PCollection, {@link Count#perKey()} can be used to count the + * number of values per key, and {@link Count#globally()} can be used to count the total + * number of elements in a PCollection. */ public class Count { + private Count() { + // do not instantiate + } /** * Returns a {@link Combine.Globally} {@link PTransform} that counts the number of elements in diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java index 79d904ed57057..8678e4f33eaeb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java @@ -44,6 +44,10 @@ */ public class Max { + private Max() { + // do not instantiate + } + /** * Returns a {@code PTransform} that takes an input {@code PCollection} and returns a * {@code PCollection} whose contents is the maximum of the input {@code PCollection}'s diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java index a938cf147ce22..47ab3a0ad27d6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java @@ -44,6 +44,10 @@ */ public class Min { + private Min() { + // do not instantiate + } + /** * Returns a {@code PTransform} that takes an input {@code PCollection} and returns a * {@code PCollection} whose contents is a single value that is the minimum of the input diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java index dbd5067c24154..5b30475a9d8cd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sum.java @@ -41,6 +41,10 @@ */ public class Sum { + private Sum() { + // do not instantiate + } + /** * Returns a {@code PTransform} that takes an input * {@code PCollection} and returns a diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java index b1fe7275bfa8c..b5af914b8d66b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java @@ -49,6 +49,10 @@ */ public class Top { + private Top() { + // do not instantiate + } + /** * Returns a {@code PTransform} that takes an input * {@code PCollection} and returns a {@code PCollection>} with a From c4a6c4da410dd0de9b9e465f2fc331f3911e89b7 Mon Sep 17 00:00:00 2001 From: bchambers Date: Tue, 24 Nov 2015 15:56:37 -0800 Subject: [PATCH 1194/1541] Javadoc cleanup to BoundedSource ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108656926 --- .../cloud/dataflow/sdk/io/BoundedSource.java | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedSource.java index e7b1807556ef7..5f91ef39b863a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedSource.java @@ -48,7 +48,7 @@ */ public abstract class BoundedSource extends Source { /** - * Splits the source into bundles of approximately given size (in bytes). + * Splits the source into bundles of approximately {@code desiredBundleSizeBytes}. */ public abstract List> splitIntoBundles( long desiredBundleSizeBytes, PipelineOptions options) throws Exception; @@ -61,8 +61,8 @@ public abstract List> splitIntoBundles( public abstract long getEstimatedSizeBytes(PipelineOptions options) throws Exception; /** - * Whether this source is known to produce key/value pairs with the (encoded) keys in - * lexicographically sorted order. + * Whether this source is known to produce key/value pairs sorted by lexicographic order on + * the bytes of the encoded key. */ public abstract boolean producesSortedKeys(PipelineOptions options) throws Exception; @@ -102,8 +102,9 @@ public abstract List> splitIntoBundles( @Experimental(Experimental.Kind.SOURCE_SINK) public abstract static class BoundedReader extends Source.Reader { /** - * Returns a value in [0, 1] representing approximately what fraction of the source - * ({@link #getCurrentSource}) this reader has read so far. + * Returns a value in [0, 1] representing approximately what fraction of the + * {@link #getCurrentSource current source} this reader has read so far, or {@code null} if such + * an estimate is not available. * *

It is recommended that this method should satisfy the following properties: *

    @@ -114,13 +115,10 @@ public abstract static class BoundedReader extends Source.Reader { * *

    By default, returns null to indicate that this cannot be estimated. * - *

    Thread safety

    + *
    Thread safety
    * If {@link #splitAtFraction} is implemented, this method can be called concurrently to other * methods (including itself), and it is therefore critical for it to be implemented * in a thread-safe way. - * - * @return A value in [0, 1] representing the fraction of this reader's current input - * read so far, or {@code null} if such an estimate is not available. */ public Double getFractionConsumed() { return null; @@ -133,9 +131,10 @@ public Double getFractionConsumed() { * Tells the reader to narrow the range of the input it's going to read and give up * the remainder, so that the new range would contain approximately the given * fraction of the amount of data in the current range. + * *

    Returns a {@code BoundedSource} representing the remainder. * - *

    Detailed description

    + *
    Detailed description
    * Assuming the following sequence of calls: *
    {@code
          *   BoundedSource initial = reader.getCurrentSource();
    @@ -161,20 +160,20 @@ public Double getFractionConsumed() {
          * corresponding to the given fraction. In this case, the method MUST have no effect
          * (the reader must behave as if the method hadn't been called at all).
          *
    -     * 

    Statefulness

    + *
    Statefulness
    * Since this method (if successful) affects the reader's source, in subsequent invocations * "fraction" should be interpreted relative to the new current source. * - *

    Thread safety and blocking

    + *
    Thread safety and blocking
    * This method will be called concurrently to other methods (however there will not be multiple * concurrent invocations of this method itself), and it is critical for it to be implemented * in a thread-safe way (otherwise data loss is possible). * - *

    It is also very important that this method always completes quickly, in particular, + *

    It is also very important that this method always completes quickly. In particular, * it should not perform or wait on any blocking operations such as I/O, RPCs etc. Violating * this requirement may stall completion of the work item or even cause it to fail. * - *

    E.g. it is incorrect to make both this method and {@link #start}/{@link #advance} + *

    It is incorrect to make both this method and {@link #start}/{@link #advance} * {@code synchronized}, because those methods can perform blocking operations, and then * this method would have to wait for those calls to complete. * From 2174ac18a4d244b8274dd9b4f6a5574ac83ed425 Mon Sep 17 00:00:00 2001 From: bchambers Date: Tue, 24 Nov 2015 15:57:19 -0800 Subject: [PATCH 1195/1541] Cleanup Javadoc on BlockBasedSource ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108656990 --- .../dataflow/sdk/io/BlockBasedSource.java | 37 +++++++++++-------- 1 file changed, 22 insertions(+), 15 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BlockBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BlockBasedSource.java index f51d269d767db..62115dfbe5e69 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BlockBasedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BlockBasedSource.java @@ -30,14 +30,15 @@ * *

    {@code BlockBasedSource} should be derived from when a file format does not support efficient * seeking to a record in the file, but can support efficient seeking to a block. Alternatively, - * records in the file cannot be offset-addressed, but blocks can (i.e., it is not possible to say - * that record i starts at offset m, but it is possible to say that block j starts at offset n). + * records in the file cannot be offset-addressed, but blocks can (it is not possible to say + * that record {code i} starts at offset {@code m}, but it is possible to say that block {@code j} + * starts at offset {@code n}). * *

    The records that will be read from a {@code BlockBasedSource} that corresponds to a subrange - * of a file [startOffset, endOffset) are those records such that the record is contained in a - * block that starts at offset {@code i}, where {@code i >= startOffset} and {@code i < endOffset}. - * In other words, a record will be read from the source if it is contained in a block that begins - * within the range described by the source. + * of a file {@code [startOffset, endOffset)} are those records such that the record is contained in + * a block that starts at offset {@code i}, where {@code i >= startOffset} and + * {@code i < endOffset}. In other words, a record will be read from the source if its first byte is + * contained in a block that begins within the range described by the source. * *

    This entails that it is possible to determine the start offsets of all blocks in a file. * @@ -105,12 +106,12 @@ protected abstract static class Block { public abstract boolean readNextRecord() throws IOException; /** - * Returns the fraction of the block already consumed (i.e., not including the current record), - * if possible, as a value in [0, 1]. Successive calls to this method must be monotonically - * non-decreasing. + * Returns the fraction of the block already consumed, if possible, as a value in + * {@code [0, 1]}. It should not include the current record. Successive results from this method + * must be monotonically increasing. * - *

    If it is not possible to compute the fraction of the block consumed (e.g., the total - * number of records is unknown and record offsets are unknown), this method may return zero. + *

    If it is not possible to compute the fraction of the block consumed this method may + * return zero. For example, when the total number of records in the block is unknown. */ public abstract double getFractionOfBlockConsumed(); } @@ -118,7 +119,7 @@ protected abstract static class Block { /** * A {@code Reader} that reads records from a {@link BlockBasedSource}. If the source is a * subrange of a file, the blocks that will be read by this reader are those such that the first - * byte of the block is within the range [start, end). + * byte of the block is within the range {@code [start, end)}. */ @Experimental(Experimental.Kind.SOURCE_SINK) protected abstract static class BlockBasedReader extends FileBasedReader { @@ -143,7 +144,7 @@ protected BlockBasedReader(BlockBasedSource source) { /** * Returns the size of the current block in bytes as it is represented in the underlying file, - * if possible. This method may return 0 if the size of the current block is unknown. + * if possible. This method may return {@code 0} if the size of the current block is unknown. * *

    The size returned by this method must be such that for two successive blocks A and B, * {@code offset(A) + size(A) <= offset(B)}. If this is not satisfied, the progress reported @@ -152,8 +153,8 @@ protected BlockBasedReader(BlockBasedSource source) { * *

    This method and {@link Block#getFractionOfBlockConsumed} are used to provide an estimate * of progress within a block ({@code getCurrentBlock().getFractionOfBlockConsumed() * - * getCurrentBlockSize()}). It is acceptable for the result of this computation to be 0, but - * progress estimation will be inaccurate. + * getCurrentBlockSize()}). It is acceptable for the result of this computation to be {@code 0}, + * but progress estimation will be inaccurate. */ public abstract long getCurrentBlockSize(); @@ -183,6 +184,12 @@ protected boolean isAtSplitPoint() { return atSplitPoint; } + /** + * Reads the next record from the {@link getCurrentBlock() current block} if + * possible. Will call {@link readNextBlock()} to advance to the next block if not. + * + *

    The first record read from a block is treated as a split point. + */ @Override protected final boolean readNextRecord() throws IOException { atSplitPoint = false; From cd212b1937df737c38887d59ce38291d099f4b33 Mon Sep 17 00:00:00 2001 From: lcwik Date: Mon, 30 Nov 2015 10:42:26 -0800 Subject: [PATCH 1196/1541] Migrate to use nano time and a nano sleep in tests Use a high precision timer instead of the system clock to measure time differences to prevent things like NTP from affecting tests. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108993050 --- .../DataflowWorkProgressUpdaterTest.java | 5 +- .../runners/worker/FakeWindmillServer.java | 3 +- .../worker/WindmillStateInternalsTest.java | 3 +- .../sdk/testing/ExpectedLogsTest.java | 9 ++- .../sdk/testing/SystemNanoTimeSleeper.java | 68 +++++++++++++++++++ .../testing/SystemNanoTimeSleeperTest.java | 53 +++++++++++++++ .../IntraBundleParallelizationTest.java | 44 ++++++------ .../dataflow/sdk/util/MemoryMonitorTest.java | 6 +- .../util/common/worker/ReadOperationTest.java | 3 +- .../util/common/worker/StateSamplerTest.java | 22 +++--- 10 files changed, 177 insertions(+), 39 deletions(-) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/SystemNanoTimeSleeper.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/SystemNanoTimeSleeperTest.java diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java index b76ebba36fed3..a6d60d946ca38 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java @@ -22,6 +22,7 @@ import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateProgress; +import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; import static com.google.cloud.dataflow.sdk.util.CloudCounterUtils.extractCounter; import static com.google.cloud.dataflow.sdk.util.CloudMetricUtils.extractCloudMetric; import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; @@ -266,7 +267,7 @@ public void workProgressUpdaterSendsLastPendingUpdateWhenStopped() throws Except progressUpdater.startReportingProgress(); // The initial update should be sent after 300 msec. - Thread.sleep(50); + sleepMillis(50); verifyZeroInteractions(workUnitClient); verify(workUnitClient, timeout(350)) @@ -279,7 +280,7 @@ public void workProgressUpdaterSendsLastPendingUpdateWhenStopped() throws Except verifyNoMoreInteractions(workUnitClient); // still not yet after 50ms - Thread.sleep(50); + sleepMillis(50); verifyNoMoreInteractions(workUnitClient); // Stop the progressUpdater now, and expect the last update immediately diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/FakeWindmillServer.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/FakeWindmillServer.java index 3dcb53ffeefc3..1b609f044e2db 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/FakeWindmillServer.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/FakeWindmillServer.java @@ -16,6 +16,7 @@ package com.google.cloud.dataflow.sdk.runners.worker; +import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; import static org.junit.Assert.assertFalse; import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; @@ -76,7 +77,7 @@ public Windmill.GetDataResponse getData(Windmill.GetDataRequest request) { } else { try { // Sleep for a little bit to ensure that *-windmill-read state-sampled counters show up. - Thread.sleep(500); + sleepMillis(500); } catch (InterruptedException e) {} } return response; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java index fe80f90a175fd..b976e05051af4 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java @@ -15,6 +15,7 @@ */ package com.google.cloud.dataflow.sdk.runners.worker; +import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -99,7 +100,7 @@ private void waitAndSet(final SettableFuture future, final T value, final @Override public void run() { try { - Thread.sleep(millis); + sleepMillis(millis); } catch (InterruptedException e) { throw new RuntimeException("Interrupted before setting", e); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java index 44b0b60039c95..2dce880ffc39f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/ExpectedLogsTest.java @@ -16,6 +16,8 @@ package com.google.cloud.dataflow.sdk.testing; +import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; + import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -31,6 +33,7 @@ import java.util.concurrent.CompletionService; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; /** Tests for {@link FastNanoClockAndSleeper}. */ @RunWith(JUnit4.class) @@ -114,7 +117,8 @@ public void testLogCaptureOccursAtLowestLogLevel() throws Throwable { public void testThreadSafetyOfLogSaver() throws Throwable { CompletionService completionService = new ExecutorCompletionService<>(Executors.newCachedThreadPool()); - final long scheduledLogTime = System.currentTimeMillis() + 500L; + final long scheduledLogTime = + TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS) + 500L; List expectedStrings = new ArrayList<>(); for (int i = 0; i < 100; i++) { @@ -124,7 +128,8 @@ public void testThreadSafetyOfLogSaver() throws Throwable { @Override public Void call() throws Exception { // Have all threads started and waiting to log at about the same moment. - Thread.sleep(Math.max(1, scheduledLogTime - System.currentTimeMillis())); + sleepMillis(Math.max(1, scheduledLogTime + - TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS))); LOG.trace(expected); return null; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/SystemNanoTimeSleeper.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/SystemNanoTimeSleeper.java new file mode 100644 index 0000000000000..d8507f79b08f5 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/SystemNanoTimeSleeper.java @@ -0,0 +1,68 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import com.google.api.client.util.Sleeper; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.LockSupport; + +/** + * This class provides an expensive sleeper to deal with issues around Java's + * accuracy of {@link System#currentTimeMillis} and methods such as + * {@link Object#wait} and {@link Thread#sleep} which depend on it. This + * article goes into further detail about this issue. + * + * This {@link Sleeper} uses {@link System#nanoTime} + * as the timing source and {@link LockSupport#parkNanos} as the wait method. + * Note that usage of this sleeper may impact performance because + * of the relatively more expensive methods being invoked when compared to + * {@link Thread#sleep}. + */ +public class SystemNanoTimeSleeper implements Sleeper { + public static final Sleeper INSTANCE = new SystemNanoTimeSleeper(); + + /** Limit visibility to prevent instantiation. */ + private SystemNanoTimeSleeper() { + } + + @Override + public void sleep(long millis) throws InterruptedException { + long currentTime; + long endTime = System.nanoTime() + TimeUnit.NANOSECONDS.convert(millis, TimeUnit.MILLISECONDS); + while ((currentTime = System.nanoTime()) < endTime) { + if (Thread.interrupted()) { + throw new InterruptedException(); + } + LockSupport.parkNanos(endTime - currentTime); + } + if (Thread.interrupted()) { + throw new InterruptedException(); + } + return; + } + + /** + * Causes the currently executing thread to sleep (temporarily cease + * execution) for the specified number of milliseconds. The thread does not + * lose ownership of any monitors. + */ + public static void sleepMillis(long millis) throws InterruptedException { + INSTANCE.sleep(millis); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/SystemNanoTimeSleeperTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/SystemNanoTimeSleeperTest.java new file mode 100644 index 0000000000000..33b6b693a2977 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/SystemNanoTimeSleeperTest.java @@ -0,0 +1,53 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.testing; + +import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; +import static org.junit.Assert.assertTrue; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link SystemNanoTimeSleeper}. */ +@RunWith(JUnit4.class) +public class SystemNanoTimeSleeperTest { + @Test + public void testSleep() throws Exception { + long startTime = System.nanoTime(); + sleepMillis(100); + long endTime = System.nanoTime(); + assertTrue(endTime - startTime >= 100); + } + + @Test + public void testNegativeSleep() throws Exception { + sleepMillis(-100); + } + + @Test(expected = InterruptedException.class) + public void testInterruptionInLoop() throws Exception { + Thread.currentThread().interrupt(); + sleepMillis(0); + } + + @Test(expected = InterruptedException.class) + public void testInterruptionOutsideOfLoop() throws Exception { + Thread.currentThread().interrupt(); + sleepMillis(-100); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelizationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelizationTest.java index afefcdae15d35..a15c694fe97c8 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelizationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelizationTest.java @@ -16,23 +16,27 @@ package com.google.cloud.dataflow.sdk.transforms; +import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.testing.TestPipeline; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import java.util.ArrayList; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; /** @@ -65,7 +69,7 @@ private static class DelayFn extends DoFn { @Override public void processElement(ProcessContext c) { try { - Thread.sleep(DELAY_MS); + sleepMillis(DELAY_MS); } catch (InterruptedException e) { e.printStackTrace(); throw new RuntimeException("Interrupted"); @@ -101,7 +105,9 @@ public void processElement(ProcessContext c) { private static class ConcurrencyMeasuringFn extends DoFn { @Override public void processElement(ProcessContext c) { - synchronized (this) { + // Synchronize on the class to provide synchronous access irrespective of + // how this DoFn is called. + synchronized (ConcurrencyMeasuringFn.class) { concurrentElements++; if (concurrentElements > maxConcurrency) { maxConcurrency = concurrentElements; @@ -110,7 +116,7 @@ public void processElement(ProcessContext c) { c.output(c.element()); - synchronized (this) { + synchronized (ConcurrencyMeasuringFn.class) { concurrentElements--; } } @@ -127,11 +133,11 @@ public void testParallelization() { // The minimum is guaranteed to be >= 2x the delay interval, since no more than half the // elements can be scheduled at once. - Assert.assertThat(minDuration, + assertThat(minDuration, greaterThanOrEqualTo(2 * DelayFn.DELAY_MS)); // Also, it should take <= 8x the delay interval since we should be at least // parallelizing some of the work. - Assert.assertThat(minDuration, + assertThat(minDuration, lessThanOrEqualTo(8 * DelayFn.DELAY_MS)); } @@ -140,19 +146,19 @@ public void testExceptionHandling() { ExceptionThrowingFn fn = new ExceptionThrowingFn<>(10); try { run(100, PARALLELISM_FACTOR, fn); - Assert.fail("Expected exception to propagate"); + fail("Expected exception to propagate"); } catch (RuntimeException e) { - Assert.assertThat(e.getMessage(), containsString("Expected failure")); + assertThat(e.getMessage(), containsString("Expected failure")); } // Should have processed 10 elements, but stopped before processing all // of them. - Assert.assertThat(numProcessed.get(), + assertThat(numProcessed.get(), is(both(greaterThanOrEqualTo(10)) .and(lessThan(100)))); // The first failure should prevent the scheduling of any more elements. - Assert.assertThat(numFailures.get(), + assertThat(numFailures.get(), is(both(greaterThanOrEqualTo(1)) .and(lessThanOrEqualTo(PARALLELISM_FACTOR)))); } @@ -162,20 +168,20 @@ public void testExceptionHandlingOnLastElement() { ExceptionThrowingFn fn = new ExceptionThrowingFn<>(9); try { run(10, PARALLELISM_FACTOR, fn); - Assert.fail("Expected exception to propagate"); + fail("Expected exception to propagate"); } catch (RuntimeException e) { - Assert.assertThat(e.getMessage(), containsString("Expected failure")); + assertThat(e.getMessage(), containsString("Expected failure")); } // Should have processed 10 elements, but stopped before processing all // of them. - Assert.assertEquals(10, numProcessed.get()); - Assert.assertEquals(1, numFailures.get()); + assertEquals(10, numProcessed.get()); + assertEquals(1, numFailures.get()); } @Test public void testIntraBundleParallelizationGetName() { - Assert.assertEquals( + assertEquals( "IntraBundleParallelization", IntraBundleParallelization.of(new DelayFn()).withMaxParallelism(1).getName()); } @@ -194,14 +200,14 @@ private long run(int numElements, int maxParallelism, DoFn doF .apply(IntraBundleParallelization.of(doFn).withMaxParallelism(maxParallelism)) .apply(ParDo.of(downstream)); - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); pipeline.run(); // Downstream methods should not see parallel threads. - Assert.assertEquals(1, maxConcurrency); + assertEquals(1, maxConcurrency); - long endTime = System.currentTimeMillis(); - return endTime - startTime; + long endTime = System.nanoTime(); + return TimeUnit.MILLISECONDS.convert(endTime - startTime, TimeUnit.NANOSECONDS); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/MemoryMonitorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/MemoryMonitorTest.java index 4cf75623a54fb..c175165b6dcaa 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/MemoryMonitorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/MemoryMonitorTest.java @@ -16,6 +16,8 @@ package com.google.cloud.dataflow.sdk.util; +import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; + import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -64,10 +66,10 @@ public void setup() { @Test(timeout = 1000) public void detectGCThrashing() throws InterruptedException { - Thread.sleep(100); + sleepMillis(100); monitor.waitForResources("Test1"); provider.inGCThrashingState.set(true); - Thread.sleep(100); + sleepMillis(100); final Semaphore s = new Semaphore(0); new Thread(new Runnable() { @Override diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java index 99c296517970f..72c24de508b4d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java @@ -23,6 +23,7 @@ import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.readerProgressToCloudProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.toCloudPosition; +import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind; import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.MEAN; import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; @@ -124,7 +125,7 @@ public void testGetProgress() throws Exception { Thread thread = runReadLoopInThread(readOperation); for (int i = 0; i < 5; ++i) { - Thread.sleep(500); // Wait for the operation to start and block. + sleepMillis(500); // Wait for the operation to start and block. // Ensure that getProgress() doesn't block while the next() method is blocked. ApproximateProgress progress = readerProgressToCloudProgress(readOperation.getProgress()); long observedIndex = progress.getPosition().getRecordIndex().longValue(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java index 8291b73021eaa..f4f6752b6c504 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSamplerTest.java @@ -16,6 +16,7 @@ package com.google.cloud.dataflow.sdk.util.common.worker; +import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -59,13 +60,13 @@ public void basicTest() throws InterruptedException { try (StateSampler.ScopedState s1 = stateSampler.scopedState(state1)) { assert s1 != null; - Thread.sleep(2 * periodMs); + sleepMillis(2 * periodMs); } try (StateSampler.ScopedState s2 = stateSampler.scopedState(state2)) { assert s2 != null; - Thread.sleep(3 * periodMs); + sleepMillis(3 * periodMs); } long s1 = getCounterLongValue(counters, "test-1-msecs"); @@ -91,21 +92,21 @@ public void nestingTest() throws InterruptedException { try (StateSampler.ScopedState s1 = stateSampler.scopedState(state1)) { assert s1 != null; - Thread.sleep(2 * periodMs); + sleepMillis(2 * periodMs); try (StateSampler.ScopedState s2 = stateSampler.scopedState(state2)) { assert s2 != null; - Thread.sleep(2 * periodMs); + sleepMillis(2 * periodMs); try (StateSampler.ScopedState s3 = stateSampler.scopedState(state3)) { assert s3 != null; - Thread.sleep(2 * periodMs); + sleepMillis(2 * periodMs); } - Thread.sleep(periodMs); + sleepMillis(periodMs); } - Thread.sleep(periodMs); + sleepMillis(periodMs); } long s1 = getCounterLongValue(counters, "test-1-msecs"); @@ -127,7 +128,7 @@ public void nonScopedTest() throws InterruptedException { int state1 = stateSampler.stateForName("1", StateSampler.StateKind.USER); int previousState = stateSampler.setState(state1); - Thread.sleep(2 * periodMs); + sleepMillis(2 * periodMs); stateSampler.setState(previousState); long tolerance = periodMs; long s = getCounterLongValue(counters, "test-1-msecs"); @@ -149,14 +150,13 @@ private void noSamplingAfterCloseTestOnce() throws Exception { stateSampler.addSamplingCallback(new SamplingCallback(){ @Override public void run(int state, StateKind kind, long elapsedMs) { - lastSampledTimeStamp.set(System.currentTimeMillis()); + lastSampledTimeStamp.set(System.nanoTime()); sampleHappened.release(); } }); sampleHappened.acquire(); } - long samplerStoppedTimeStamp = System.currentTimeMillis(); - Thread.sleep(2 * periodMs); + long samplerStoppedTimeStamp = System.nanoTime(); assertThat(lastSampledTimeStamp.get(), Matchers.lessThanOrEqualTo(samplerStoppedTimeStamp)); } From a8347d186700e83edb91a3c673200abec1b1d4e7 Mon Sep 17 00:00:00 2001 From: bchambers Date: Mon, 30 Nov 2015 10:48:03 -0800 Subject: [PATCH 1197/1541] Cleanup Javadoc on CompressedSource ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=108993638 --- .../dataflow/sdk/io/CompressedSource.java | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java index 31393f1a5d8e9..5f98732608ead 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java @@ -37,15 +37,12 @@ * *

    For example, use the following to read from a gzip-compressed XML file: * - * {@code + *

     {@code
      * XmlSource mySource = XmlSource.from(...);
    - * PCollection collection = p.apply(CompressedSource.readFromSource(mySource,
    - * CompressedSource.CompressionMode.GZIP);}
    - *
    - * Or, alternatively:
    - * XmlSource mySource = XmlSource.from(...);
    - * {@code PCollection collection = p.apply(Read.from(CompressedSource.from(mySource,
    - * CompressedSource.CompressionMode.GZIP)));}
    + * PCollection collection = p.apply(Read.from(CompressedSource
    + *     .from(mySource)
    + *     .withDecompression(CompressedSource.CompressionMode.GZIP)));
    + * } 
    * *

    Default compression modes are {@link CompressionMode#GZIP} and {@link CompressionMode#BZIP2}. * User-defined compression types are supported by implementing {@link DecompressingChannelFactory}. @@ -56,9 +53,8 @@ public class CompressedSource extends FileBasedSource { /** * Factory interface for creating channels that decompress the content of an underlying channel. - * - *

    TODO: Refactor decompressing channel/stream creation and default instances to util classes. */ + // TODO: Refactor decompressing channel/stream creation and default instances to util classes. public static interface DecompressingChannelFactory extends Serializable { /** * Given a channel, create a channel that decompresses the content read from the channel. @@ -97,8 +93,8 @@ public abstract ReadableByteChannel createDecompressingChannel(ReadableByteChann private final DecompressingChannelFactory channelFactory; /** - * Creates a {@link Read} transform that reads from a {@code CompressedSource} that reads from an - * underlying {@link FileBasedSource} after decompressing it with a {@link + * Creates a {@link Read} transform that reads from that reads from the underlying + * {@link FileBasedSource} {@code sourceDelegate} after decompressing it with a {@link * DecompressingChannelFactory}. */ public static Read.Bounded readFromSource( From 435508ef2ebcd9b78b89e4cd8fcb12f656cc7638 Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 30 Nov 2015 19:53:19 -0800 Subject: [PATCH 1198/1541] Convert OutputTimeFn from interface to abstract class ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109041088 --- .../transforms/windowing/OutputTimeFn.java | 70 ++++++++----------- 1 file changed, 31 insertions(+), 39 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OutputTimeFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OutputTimeFn.java index f97cd8589f0e1..a72a9cb1eb8b5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OutputTimeFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OutputTimeFn.java @@ -38,20 +38,22 @@ *

  • The output timestamp when windows merge is provided by {@link #merge merge()}.
  • * * - *

    To implement this interface, extend {@link OutputTimeFn.Defaults} or - * {@link OutputTimeFn.DependsOnlyOnWindow} or your implementation may be impacted when the - * interface is enlarged. This interface will only be enlarged in ways that are - * backwards-compatible for consumers. The base classes will only be changed in ways that - * are backwards-compatible for implementors as well. - * - *

    Note that as long as the interface remains experimental, we may choose to change it in - * arbitrary backwards incompatible ways if it is indicated by the experiment. + *

    This abstract class cannot be subclassed directly, by design: it may grow + * in consumer-compatible ways that require mutually-exclusive default implementations. To + * create a concrete subclass, extend {@link OutputTimeFn.Defaults} or + * {@link OutputTimeFn.DependsOnlyOnWindow}. Note that as long as this class remains + * experimental, we may also choose to change it in arbitrary backwards-incompatible ways. * * @param the type of window. Contravariant: methods accepting any subtype of * {@code OutputTimeFn} should use the parameter type {@code OutputTimeFn}. */ @Experimental(Experimental.Kind.OUTPUT_TIME) -public interface OutputTimeFn extends Serializable { +public abstract class OutputTimeFn implements Serializable { + + /** + * Private constructor to prevent subclassing other than provided base classes. + */ + private OutputTimeFn() { } /** * Returns the output timestamp to use for data depending on the given @@ -73,7 +75,7 @@ public interface OutputTimeFn extends Serializable { *

    See the overview of {@link OutputTimeFn} for the consistency properties required * between {@link #assignOutputTime}, {@link #combine}, and {@link #merge}. */ - Instant assignOutputTime(Instant inputTimestamp, W window); + public abstract Instant assignOutputTime(Instant inputTimestamp, W window); /** * Combines the given output times, which must be from the same window, into an output time @@ -85,7 +87,7 @@ public interface OutputTimeFn extends Serializable { * {@code combine(a, combine(b, c)).equals(combine(combine(a, b), c))}. *

*/ - Instant combine(Instant outputTime, Instant otherOutputTime); + public abstract Instant combine(Instant outputTime, Instant otherOutputTime); /** * Merges the given output times, presumed to be combined output times for windows that @@ -111,7 +113,7 @@ public interface OutputTimeFn extends Serializable { * timestamp, this will be the same as {@link #combine combine()}. Defaults for this * case are provided by {@link Default}. */ - Instant merge(W intoWindow, Iterable mergingTimestamps); + public abstract Instant merge(W intoWindow, Iterable mergingTimestamps); /** * Returns {@code true} if the result of combination of many output timestamps actually depends @@ -120,7 +122,7 @@ public interface OutputTimeFn extends Serializable { *

This may allow optimizations when it is very efficient to retrieve the earliest timestamp * to be combined. */ - boolean dependsOnlyOnEarliestInputTimestamp(); + public abstract boolean dependsOnlyOnEarliestInputTimestamp(); /** * Returns {@code true} if the result does not depend on what outputs were combined but only @@ -135,13 +137,7 @@ public interface OutputTimeFn extends Serializable { * a framework for easily implementing a correct {@link #merge}, {@link #combine} and * {@link #assignOutputTime}. */ - boolean dependsOnlyOnWindow(); - - /** - * Please extend {@link Defaults} or {@link DependsOnlyOnWindow} if you want guaranteed - * compilation compatibility; this interface may be enlarged in consumer-compatible ways. - */ - void pleaseExtendBaseClassesForCompilationCompatibility(); + public abstract boolean dependsOnlyOnWindow(); /** * (Experimental) Default method implementations for {@link OutputTimeFn} where the @@ -152,7 +148,11 @@ public interface OutputTimeFn extends Serializable { *

By default, {@link #combine} and {@link #merge} return the earliest timestamp of their * inputs. */ - public abstract static class Defaults implements OutputTimeFn { + public abstract static class Defaults extends OutputTimeFn { + + protected Defaults() { + super(); + } /** * {@inheritDoc} @@ -186,11 +186,11 @@ public final boolean dependsOnlyOnWindow() { return false; } - /** - * {@inheritDoc} - * - * @return {@code true} by default. - */ + /** + * {@inheritDoc} + * + * @return {@code true} by default. + */ @Override public boolean dependsOnlyOnEarliestInputTimestamp() { return false; @@ -215,12 +215,6 @@ public boolean equals(Object other) { public int hashCode() { return Objects.hash(getClass()); } - - /** - * This base class provides compilation compatibility when {@link OutputTimeFn} is enlarged. - */ - @Override - public void pleaseExtendBaseClassesForCompilationCompatibility() { } } /** @@ -230,7 +224,11 @@ public void pleaseExtendBaseClassesForCompilationCompatibility() { } *

To complete an implementation, override {@link #assignOutputTime(BoundedWindow)}. */ public abstract static class DependsOnlyOnWindow - implements OutputTimeFn { + extends OutputTimeFn { + + protected DependsOnlyOnWindow() { + super(); + } /** * Returns the output timestamp to use for data in the specified {@code window}. @@ -317,11 +315,5 @@ public boolean equals(Object other) { public int hashCode() { return Objects.hash(getClass()); } - - /** - * This base class provides compilation compatibility when {@link OutputTimeFn} is enlarged. - */ - @Override - public void pleaseExtendBaseClassesForCompilationCompatibility() { } } } From c7bb5bbe7d5a6350c2ce991f06084cd378ddb69b Mon Sep 17 00:00:00 2001 From: tgroh Date: Tue, 1 Dec 2015 13:25:38 -0800 Subject: [PATCH 1199/1541] Remove Unneccessary import in CombineJava8Test ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109118933 --- .../google/cloud/dataflow/sdk/transforms/CombineJava8Test.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/CombineJava8Test.java b/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/CombineJava8Test.java index f3f04c3ae7497..b569e49c951db 100644 --- a/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/CombineJava8Test.java +++ b/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/CombineJava8Test.java @@ -16,10 +16,7 @@ package com.google.cloud.dataflow.sdk.transforms; -import static org.junit.Assert.assertEquals; - import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.values.KV; From f56b1b8d32b5df59c504517812f15b72e69961a2 Mon Sep 17 00:00:00 2001 From: lcwik Date: Tue, 1 Dec 2015 14:06:21 -0800 Subject: [PATCH 1200/1541] Fix sizing of windowed side inputs to improve caching Caching was using the entire size of the side input, not the size of just the elements. This was incorrectly reporting the size of the side input by a factor of the number of windows making the caching pointless for side inputs with a large number of windows. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109123443 --- .../worker/DataflowSideInputReader.java | 56 ++++++++++++++----- .../worker/DataflowSideInputReaderTest.java | 45 +++++++++++---- 2 files changed, 75 insertions(+), 26 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReader.java index 821bd4c6016e1..545f32bacad43 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReader.java @@ -19,15 +19,18 @@ import com.google.api.services.dataflow.model.SideInputInfo; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; -import com.google.cloud.dataflow.sdk.util.DirectSideInputReader; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.PTuple; import com.google.cloud.dataflow.sdk.util.SideInputReader; import com.google.cloud.dataflow.sdk.util.Sized; import com.google.cloud.dataflow.sdk.util.SizedSideInputReader; +import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; import java.util.HashMap; import java.util.Map; @@ -51,8 +54,8 @@ public class DataflowSideInputReader /** A byte count saved as overhead per side input, not cleared when the observer is reset. */ private final Map, Long> overheads; - /** The underlying reader, which does not keep track of sizes. */ - private final SideInputReader subReader; + /** A list of TupleTags representing the side input values. */ + private final PTuple sideInputValues; private DataflowSideInputReader( Iterable sideInputInfos, @@ -64,7 +67,7 @@ private DataflowSideInputReader( this.observers = new HashMap<>(); this.overheads = new HashMap<>(); - PTuple sideInputValues = PTuple.empty(); + PTuple sideInputValuesBeingBuilt = PTuple.empty(); for (SideInputInfo sideInputInfo : sideInputInfos) { TupleTag tag = new TupleTag<>(sideInputInfo.getTag()); ByteSizeObserver observer = new ByteSizeObserver(); @@ -73,9 +76,9 @@ private DataflowSideInputReader( overheads.put(tag, observer.getBytes()); observer.reset(); observers.put(tag, observer); - sideInputValues = sideInputValues.and(tag, sideInputValue); + sideInputValuesBeingBuilt = sideInputValuesBeingBuilt.and(tag, sideInputValue); } - this.subReader = DirectSideInputReader.of(sideInputValues); + sideInputValues = sideInputValuesBeingBuilt; } /** @@ -92,12 +95,12 @@ public static DataflowSideInputReader of( @Override public boolean contains(PCollectionView view) { - return subReader.contains(view); + return sideInputValues.has(view.getTagInternal()); } @Override public boolean isEmpty() { - return subReader.isEmpty(); + return sideInputValues.isEmpty(); } /** @@ -108,16 +111,39 @@ public boolean isEmpty() { */ @Override public Sized getSized(PCollectionView view, final BoundedWindow window) { + final TupleTag>> tag = view.getTagInternal(); + if (!sideInputValues.has(tag)) { + throw new IllegalArgumentException("calling getSideInput() with unknown view"); + } + // It is hard to estimate the size with any accuracy here, and there will be improvements // possible, but it is only required to estimate in a way so that a cache will not OOM. - T value = subReader.get(view, window); - @SuppressWarnings({"rawtypes", "unchecked"}) // irrelevant phantom type - TupleTag tag = (TupleTag) view.getTagInternal(); - ByteSizeObserver observer = observers.get(tag); + T value; long overhead = overheads.get(tag); - long bytesRead = observer.getBytes(); - observer.reset(); - return Sized.of(value, overhead + bytesRead); + final ByteSizeObserver observer = observers.get(tag); + if (view.getWindowingStrategyInternal().getWindowFn() instanceof GlobalWindows) { + value = view.fromIterableInternal(sideInputValues.get(tag)); + long bytesRead = observer.getBytes(); + observer.reset(); + return Sized.of(value, overhead + bytesRead); + } else { + final long[] sum = new long[]{ 0L }; + value = view.fromIterableInternal( + Iterables.filter(sideInputValues.get(tag), + new Predicate>() { + @Override + public boolean apply(WindowedValue element) { + boolean containsWindow = element.getWindows().contains(window); + // Only sum up the size of the elements within the window. + if (containsWindow) { + sum[0] += observer.getBytes(); + } + observer.reset(); + return containsWindow; + } + })); + return Sized.of(value, overhead + sum[0]); + } } /** diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReaderTest.java index 47a0949378765..c3ad2bbd23e6c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReaderTest.java @@ -28,6 +28,9 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.testing.PCollectionViewTesting; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.ExecutionContext; @@ -36,7 +39,9 @@ import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.joda.time.Instant; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -62,7 +67,8 @@ public class DataflowSideInputReaderTest { ImmutableList.of(1L, -43255L, 0L, 13L, 1975858L); private static final long DEFAULT_SOURCE_LENGTH = DEFAULT_SOURCE_CONTENTS.size(); - + private static final IntervalWindow OTHER_WINDOW = + new IntervalWindow(new Instant(50000L), new Instant(60000L)); private PipelineOptions options = PipelineOptionsFactory.create(); private static ExecutionContext executionContext; private SideInputInfo defaultSideInputInfo; @@ -70,15 +76,19 @@ public class DataflowSideInputReaderTest { /** * Creates a {@link Source} descriptor for reading the provided contents as a side input. - * The contents will all be placed in the {@link PCollectionViewTesting#DEFAULT_NONEMPTY_WINDOW}. * *

If the {@code PCollectionView} has an incompatible {@code Coder} or * {@code WindowingStrategy}, then results are unpredictable. */ - private final Source sourceInDefaultWindow(PCollectionView view, Iterable values) + private final Source sourceInMultipleWindows(PCollectionView view, Iterable values) throws Exception { - List> windowedValues = - ImmutableList.copyOf(PCollectionViewTesting.contentsInDefaultWindow(values)); + List> windowedValues = ImmutableList.>builder() + .addAll(PCollectionViewTesting.contentsInDefaultWindow(values)) + // We add the values twice within the other window so there are a different number + // then in the default window. + .addAll(contentsInWindow(values, OTHER_WINDOW)) + .addAll(contentsInWindow(values, OTHER_WINDOW)) + .build(); @SuppressWarnings({"unchecked", "rawtypes"}) List> componentCoders = (List) view.getCoderInternal().getCoderArguments(); @@ -109,13 +119,26 @@ private long windowedLongBytes() throws Exception { PCollectionViewTesting.valueInDefaultWindow(arbitraryLong)).length; } + /** + * Prepares {@code values} for reading as the contents of a {@link PCollectionView} side input. + */ + private static Iterable> contentsInWindow(Iterable values, + BoundedWindow window) throws Exception { + List> windowedValues = Lists.newArrayList(); + for (T value : values) { + windowedValues.add( + WindowedValue.of(value, window.maxTimestamp().minus(1), window, PaneInfo.NO_FIRING)); + } + return windowedValues; + } + @Before public void setUp() throws Exception { options = PipelineOptionsFactory.create(); executionContext = BatchModeExecutionContext.fromOptions(options); defaultSideInputInfo = SideInputUtils.createCollectionSideInputInfo( - sourceInDefaultWindow(DEFAULT_LENGTH_VIEW, DEFAULT_SOURCE_CONTENTS)); + sourceInMultipleWindows(DEFAULT_LENGTH_VIEW, DEFAULT_SOURCE_CONTENTS)); defaultSideInputInfo.setTag(DEFAULT_LENGTH_VIEW.getTagInternal().getId()); defaultSideInputReader = DataflowSideInputReader.of( @@ -136,7 +159,7 @@ public void testDataflowSideInputReaderNotEmpty() throws Exception { * {@link DataflowSideInputReader}, the read succeeds and has the right size. */ @Test - public void testDataflowSideInputReaderGoodRead() throws Exception { + public void testDataflowSideInputReaderFilteredRead() throws Exception { assertTrue(defaultSideInputReader.contains(DEFAULT_LENGTH_VIEW)); Sized sizedValue = defaultSideInputReader.getSized( DEFAULT_LENGTH_VIEW, PCollectionViewTesting.DEFAULT_NONEMPTY_WINDOW); @@ -156,7 +179,7 @@ public void testDataflowSideInputReaderRepeatedRead() throws Exception { Sized firstRead = sideInputReader.getSized( DEFAULT_LENGTH_VIEW, PCollectionViewTesting.DEFAULT_NONEMPTY_WINDOW); - // A repeated read should yield the same size + // A repeated read should yield the same size. Sized repeatedRead = sideInputReader.getSized( DEFAULT_LENGTH_VIEW, PCollectionViewTesting.DEFAULT_NONEMPTY_WINDOW); @@ -170,11 +193,11 @@ public void testDataflowSideInputReaderMiss() throws Exception { DataflowSideInputReader sideInputReader = DataflowSideInputReader.of( Collections.singletonList(defaultSideInputInfo), options, executionContext); - // Reading an empty window still yields the same size, for now + // Reading an empty window yields the size of 0 elements. Sized emptyWindowValue = sideInputReader.getSized( DEFAULT_LENGTH_VIEW, PCollectionViewTesting.DEFAULT_EMPTY_WINDOW); assertThat(emptyWindowValue.getValue(), equalTo(0L)); - assertThat(emptyWindowValue.getSize(), equalTo(DEFAULT_SOURCE_LENGTH * windowedLongBytes())); + assertThat(emptyWindowValue.getSize(), equalTo(0L)); } /** @@ -184,7 +207,7 @@ public void testDataflowSideInputReaderMiss() throws Exception { @Test public void testDataflowSideInputReaderBadRead() throws Exception { SideInputInfo sideInputInfo = SideInputUtils.createCollectionSideInputInfo( - sourceInDefaultWindow(DEFAULT_LENGTH_VIEW, DEFAULT_SOURCE_CONTENTS)); + sourceInMultipleWindows(DEFAULT_LENGTH_VIEW, DEFAULT_SOURCE_CONTENTS)); sideInputInfo.setTag("not the same tag at all"); DataflowSideInputReader sideInputReader = DataflowSideInputReader.of( From 3ebf8b5e3143821893feb67cfc24ab3a9c51ec9b Mon Sep 17 00:00:00 2001 From: tgroh Date: Tue, 1 Dec 2015 14:12:27 -0800 Subject: [PATCH 1201/1541] Improve Lambda Support in WithKeys, RemoveDuplicates Both WithKeys#of(SerializableFunction) and RemoveDuplicates#withRepresentativeValuesFn(SerializableFunction) can be passed Lambdas in Java 8, but due to erasure fail during Pipeline construction when the coder is not accessible. Add WithKeys#withKeyType(TypeDescriptor) and WithRepresentativeValues#withRepresentativeType(TypeDescriptor) to ease use of these PTransforms without explicitly providing a coder to the output PCollection. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109124138 --- .../sdk/transforms/RemoveDuplicates.java | 43 +++++++- .../dataflow/sdk/transforms/WithKeys.java | 17 ++++ .../dataflow/sdk/transforms/WithKeysTest.java | 16 +++ .../transforms/RemoveDuplicatesJava8Test.java | 99 +++++++++++++++++++ .../sdk/transforms/WithKeysJava8Test.java | 74 ++++++++++++++ 5 files changed, 244 insertions(+), 5 deletions(-) create mode 100644 sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicatesJava8Test.java create mode 100644 sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/WithKeysJava8Test.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java index cff84991365e0..8913138abb278 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicates.java @@ -18,6 +18,7 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; /** * {@code RemoveDuplicates} takes a {@code PCollection} and @@ -77,7 +78,7 @@ public static RemoveDuplicates create() { */ public static WithRepresentativeValues withRepresentativeValueFn( SerializableFunction fn) { - return new WithRepresentativeValues(fn); + return new WithRepresentativeValues(fn, null); } @Override @@ -100,18 +101,34 @@ public Void apply(Iterable iter) { .apply(Keys.create()); } - private static class WithRepresentativeValues + /** + * A {@link RemoveDuplicates} {@link PTransform} that uses a {@link SerializableFunction} to + * obtain a representative value for each input element. + * + * Construct via {@link RemoveDuplicates#withRepresentativeValueFn(SerializableFunction)}. + * + * @param the type of input and output element + * @param the type of representative values used to dedup + */ + public static class WithRepresentativeValues extends PTransform, PCollection> { - private SerializableFunction fn; + private final SerializableFunction fn; + private final TypeDescriptor representativeType; - private WithRepresentativeValues(SerializableFunction fn) { + private WithRepresentativeValues( + SerializableFunction fn, TypeDescriptor representativeType) { this.fn = fn; + this.representativeType = representativeType; } @Override public PCollection apply(PCollection in) { + WithKeys withKeys = WithKeys.of(fn); + if (representativeType != null) { + withKeys = withKeys.withKeyType(representativeType); + } return in - .apply(WithKeys.of(fn)) + .apply(withKeys) .apply(Combine.perKey( new Combine.BinaryCombineFn() { @Override @@ -121,5 +138,21 @@ public T apply(T left, T right) { })) .apply(Values.create()); } + + /** + * Return a {@code WithRepresentativeValues} {@link PTransform} that is like this one, but with + * the specified output type descriptor. + * + * Required for use of {@link RemoveDuplicates#withRepresentativeValueFn(SerializableFunction)} + * in Java 8 with a lambda as the fn. + * + * @param type a {@link TypeDescriptor} describing the representative type of this + * {@code WithRepresentativeValues} + * @return A {@code WithRepresentativeValues} {@link PTransform} that is like this one, but with + * the specified output type descriptor. + */ + public WithRepresentativeValues withRepresentativeType(TypeDescriptor type) { + return new WithRepresentativeValues<>(fn, type); + } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java index aa8559c4e631c..c06795c703849 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/WithKeys.java @@ -58,6 +58,9 @@ public class WithKeys extends PTransform, * values in the input {@code PCollection} has been paired with a * key computed from the value by invoking the given * {@code SerializableFunction}. + * + *

If using a lambda in Java 8, {@link #withKeyType(TypeDescriptor)} must + * be called on the result {@link PTransform}. */ public static WithKeys of(SerializableFunction fn) { return new WithKeys<>(fn, null); @@ -92,6 +95,20 @@ private WithKeys(SerializableFunction fn, Class keyClass) { this.keyClass = keyClass; } + /** + * Return a {@link WithKeys} that is like this one with the specified key type descriptor. + * + * For use with lambdas in Java 8, either this method must be called with an appropriate type + * descriptor or {@link PCollection#setCoder(Coder)} must be called on the output + * {@link PCollection}. + */ + public WithKeys withKeyType(TypeDescriptor keyType) { + // Safe cast + @SuppressWarnings("unchecked") + Class rawType = (Class) keyType.getRawType(); + return new WithKeys<>(fn, rawType); + } + @Override public PCollection> apply(PCollection in) { PCollection> result = diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/WithKeysTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/WithKeysTest.java index d6d0fe865d0ed..0f9abd487f931 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/WithKeysTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/WithKeysTest.java @@ -24,6 +24,7 @@ import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; import org.junit.Test; import org.junit.runner.RunWith; @@ -98,6 +99,21 @@ public void testWithKeysGetName() { assertEquals("WithKeys", WithKeys.of(100).getName()); } + @Test + public void testWithKeysWithUnneededWithKeyTypeSucceeds() { + TestPipeline p = TestPipeline.create(); + + PCollection input = + p.apply(Create.of(Arrays.asList(COLLECTION)).withCoder( + StringUtf8Coder.of())); + + PCollection> output = + input.apply(WithKeys.of(new LengthAsKey()).withKeyType(TypeDescriptor.of(Integer.class))); + DataflowAssert.that(output).containsInAnyOrder(WITH_KEYS); + + p.run(); + } + /** * Key a value by its length. */ diff --git a/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicatesJava8Test.java b/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicatesJava8Test.java new file mode 100644 index 0000000000000..d9e2180b7da74 --- /dev/null +++ b/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/RemoveDuplicatesJava8Test.java @@ -0,0 +1,99 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.not; +import static org.junit.Assert.assertThat; + +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.HashSet; +import java.util.Set; + +/** + * Java 8 tests for {@link RemoveDuplicates}. + */ +@RunWith(JUnit4.class) +public class RemoveDuplicatesJava8Test { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void withLambdaRepresentativeValuesFnAndTypeDescriptorShouldApplyFn() { + TestPipeline p = TestPipeline.create(); + + Multimap predupedContents = HashMultimap.create(); + predupedContents.put(3, "foo"); + predupedContents.put(4, "foos"); + predupedContents.put(6, "barbaz"); + predupedContents.put(6, "bazbar"); + PCollection dupes = + p.apply(Create.of("foo", "foos", "barbaz", "barbaz", "bazbar", "foo")); + PCollection deduped = + dupes.apply(RemoveDuplicates.withRepresentativeValueFn((String s) -> s.length()) + .withRepresentativeType(TypeDescriptor.of(Integer.class))); + + DataflowAssert.that(deduped).satisfies((Iterable strs) -> { + Set seenLengths = new HashSet<>(); + for (String s : strs) { + assertThat(predupedContents.values(), hasItem(s)); + assertThat(seenLengths, not(contains(s.length()))); + seenLengths.add(s.length()); + } + return null; + }); + + p.run(); + } + + @Test + public void withLambdaRepresentativeValuesFnNoTypeDescriptorShouldThrow() { + TestPipeline p = TestPipeline.create(); + + Multimap predupedContents = HashMultimap.create(); + predupedContents.put(3, "foo"); + predupedContents.put(4, "foos"); + predupedContents.put(6, "barbaz"); + predupedContents.put(6, "bazbar"); + PCollection dupes = + p.apply(Create.of("foo", "foos", "barbaz", "barbaz", "bazbar", "foo")); + + thrown.expect(IllegalStateException.class); + thrown.expectMessage("Unable to return a default Coder for RemoveRepresentativeDupes"); + thrown.expectMessage("Cannot provide a coder for type variable K"); + thrown.expectMessage("the actual type is unknown due to erasure."); + + // Thrown when applying a transform to the internal WithKeys that withRepresentativeValueFn is + // implemented with + dupes.apply("RemoveRepresentativeDupes", + RemoveDuplicates.withRepresentativeValueFn((String s) -> s.length())); + } +} + diff --git a/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/WithKeysJava8Test.java b/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/WithKeysJava8Test.java new file mode 100644 index 0000000000000..c10af29030133 --- /dev/null +++ b/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/WithKeysJava8Test.java @@ -0,0 +1,74 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms; + +import com.google.cloud.dataflow.sdk.Pipeline.PipelineExecutionException; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.RunnableOnService; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Java 8 Tests for {@link WithKeys}. + */ +@RunWith(JUnit4.class) +public class WithKeysJava8Test { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + @Category(RunnableOnService.class) + public void withLambdaAndTypeDescriptorShouldSucceed() { + TestPipeline p = TestPipeline.create(); + + PCollection values = p.apply(Create.of("1234", "3210", "0", "-12")); + PCollection> kvs = values.apply( + WithKeys.of((String s) -> Integer.valueOf(s)) + .withKeyType(TypeDescriptor.of(Integer.class))); + + DataflowAssert.that(kvs).containsInAnyOrder( + KV.of(1234, "1234"), KV.of(0, "0"), KV.of(-12, "-12"), KV.of(3210, "3210")); + + p.run(); + } + + @Test + public void withLambdaAndNoTypeDescriptorShouldThrow() { + TestPipeline p = TestPipeline.create(); + + PCollection values = p.apply(Create.of("1234", "3210", "0", "-12")); + + values.apply("ApplyKeysWithWithKeys", WithKeys.of((String s) -> Integer.valueOf(s))); + + thrown.expect(PipelineExecutionException.class); + thrown.expectMessage("Unable to return a default Coder for ApplyKeysWithWithKeys"); + thrown.expectMessage("Cannot provide a coder for type variable K"); + thrown.expectMessage("the actual type is unknown due to erasure."); + + p.run(); + } +} + From 3f9882eae89c197c17a79b907073f82f96737779 Mon Sep 17 00:00:00 2001 From: lcwik Date: Tue, 1 Dec 2015 14:32:44 -0800 Subject: [PATCH 1202/1541] Setup data structure to allow for overrides for batch pipelines ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109126289 --- .../sdk/runners/DataflowPipelineRunner.java | 52 ++++++++++--------- 1 file changed, 28 insertions(+), 24 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index e26b3cb4d5220..0f9f8ac6710c5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -137,8 +137,8 @@ public class DataflowPipelineRunner extends PipelineRunner /** Translator for this DataflowPipelineRunner, based on options. */ private final DataflowPipelineTranslator translator; - /** Custom transforms implementations for running in streaming mode. */ - private final Map, Class> streamingOverrides; + /** Custom transforms implementations. */ + private final Map, Class> overrides; /** A set of user defined functions to invoke at different points in execution. */ private DataflowPipelineRunnerHooks hooks; @@ -244,24 +244,29 @@ public static DataflowPipelineRunner fromOptions(PipelineOptions options) { this.dataflowClient = options.getDataflowClient(); this.translator = DataflowPipelineTranslator.fromOptions(options); - this.streamingOverrides = ImmutableMap., Class>builder() - .put(Combine.GloballyAsSingletonView.class, StreamingCombineGloballyAsSingletonView.class) - .put(Create.Values.class, StreamingCreate.class) - .put(View.AsMap.class, StreamingViewAsMap.class) - .put(View.AsMultimap.class, StreamingViewAsMultimap.class) - .put(View.AsSingleton.class, StreamingViewAsSingleton.class) - .put(View.AsList.class, StreamingViewAsList.class) - .put(View.AsIterable.class, StreamingViewAsIterable.class) - .put(Write.Bound.class, StreamingWrite.class) - .put(PubsubIO.Write.Bound.class, StreamingPubsubIOWrite.class) - .put(Read.Unbounded.class, StreamingUnboundedRead.class) - .put(Read.Bounded.class, StreamingUnsupportedIO.class) - .put(AvroIO.Read.Bound.class, StreamingUnsupportedIO.class) - .put(AvroIO.Write.Bound.class, StreamingUnsupportedIO.class) - .put(BigQueryIO.Read.Bound.class, StreamingUnsupportedIO.class) - .put(TextIO.Read.Bound.class, StreamingUnsupportedIO.class) - .put(TextIO.Write.Bound.class, StreamingUnsupportedIO.class) - .build(); + if (options.isStreaming()) { + overrides = ImmutableMap., Class>builder() + .put(Combine.GloballyAsSingletonView.class, StreamingCombineGloballyAsSingletonView.class) + .put(Create.Values.class, StreamingCreate.class) + .put(View.AsMap.class, StreamingViewAsMap.class) + .put(View.AsMultimap.class, StreamingViewAsMultimap.class) + .put(View.AsSingleton.class, StreamingViewAsSingleton.class) + .put(View.AsList.class, StreamingViewAsList.class) + .put(View.AsIterable.class, StreamingViewAsIterable.class) + .put(Write.Bound.class, StreamingWrite.class) + .put(PubsubIO.Write.Bound.class, StreamingPubsubIOWrite.class) + .put(Read.Unbounded.class, StreamingUnboundedRead.class) + .put(Read.Bounded.class, StreamingUnsupportedIO.class) + .put(AvroIO.Read.Bound.class, StreamingUnsupportedIO.class) + .put(AvroIO.Write.Bound.class, StreamingUnsupportedIO.class) + .put(BigQueryIO.Read.Bound.class, StreamingUnsupportedIO.class) + .put(TextIO.Read.Bound.class, StreamingUnsupportedIO.class) + .put(TextIO.Write.Bound.class, StreamingUnsupportedIO.class) + .build(); + } else { + overrides = ImmutableMap., Class>builder() + .build(); + } } /** @@ -289,16 +294,15 @@ public OutputT apply( pc.isBounded()); return outputT; - } else if (options.isStreaming() && streamingOverrides.containsKey(transform.getClass())) { - // It is the responsibility of whoever constructs streamingOverrides - // to ensure this is type safe. + } else if (overrides.containsKey(transform.getClass())) { + // It is the responsibility of whoever constructs overrides to ensure this is type safe. @SuppressWarnings("unchecked") Class> transformClass = (Class>) transform.getClass(); @SuppressWarnings("unchecked") Class> customTransformClass = - (Class>) streamingOverrides.get(transform.getClass()); + (Class>) overrides.get(transform.getClass()); PTransform customTransform = InstanceBuilder.ofType(customTransformClass) From b6c4f8f3f7701a4f0190508e65eeb4aa9d1a4b88 Mon Sep 17 00:00:00 2001 From: dhalperi Date: Tue, 1 Dec 2015 16:19:09 -0800 Subject: [PATCH 1203/1541] DataflowPipelineOptions: use gcloud project default if available The DefaultProjectFactory from GcpPipelineOptions should be used here, too. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109137241 --- .../cloud/dataflow/sdk/options/DataflowPipelineOptions.java | 1 + 1 file changed, 1 insertion(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java index 3e9a2424f0c57..a0f188af0785d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineOptions.java @@ -40,6 +40,7 @@ public interface DataflowPipelineOptions extends + "See https://cloud.google.com/storage/docs/projects for further details.") @Override @Validation.Required + @Default.InstanceFactory(DefaultProjectFactory.class) String getProject(); @Override void setProject(String value); From a8b19a609e206ce592fd2fefdf77fc9a64e108be Mon Sep 17 00:00:00 2001 From: dhalperi Date: Tue, 1 Dec 2015 19:01:44 -0800 Subject: [PATCH 1204/1541] Filter: fixup uses of #by and #byPredicate Uses, tests, javadoc ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109150424 --- .../dataflow/examples/complete/AutoComplete.java | 14 +++++++------- .../examples/complete/AutoCompleteTest.java | 2 +- .../cloud/dataflow/sdk/transforms/Filter.java | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/complete/AutoComplete.java b/examples/src/main/java/com/google/cloud/dataflow/examples/complete/AutoComplete.java index b1db84c88caa2..e8c6d405b7638 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/complete/AutoComplete.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/complete/AutoComplete.java @@ -232,12 +232,13 @@ public PCollectionList>> apply( .of(larger.get(1).apply(ParDo.of(new FlattenTops()))) // ...together with those (previously excluded) candidates of length // exactly minPrefix... - .and(input.apply(Filter.by(new SerializableFunction() { - @Override - public Boolean apply(CompletionCandidate c) { - return c.getValue().length() == minPrefix; - } - }))) + .and(input.apply(Filter.byPredicate( + new SerializableFunction() { + @Override + public Boolean apply(CompletionCandidate c) { + return c.getValue().length() == minPrefix; + } + }))) .apply("FlattenSmall", Flatten.pCollections()) // ...set the key to be the minPrefix-length prefix... .apply(ParDo.of(new AllPrefixes(minPrefix, minPrefix))) @@ -297,7 +298,6 @@ public String getValue() { } // Empty constructor required for Avro decoding. - @SuppressWarnings("unused") public CompletionCandidate() {} @Override diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/complete/AutoCompleteTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/complete/AutoCompleteTest.java index 20dbdc41cd01c..aec1557c28b0d 100644 --- a/examples/src/test/java/com/google/cloud/dataflow/examples/complete/AutoCompleteTest.java +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/complete/AutoCompleteTest.java @@ -84,7 +84,7 @@ public void testAutoComplete() { PCollection>> output = input.apply(new ComputeTopCompletions(2, recursive)) - .apply(Filter.by( + .apply(Filter.byPredicate( new SerializableFunction>, Boolean>() { @Override public Boolean apply(KV> element) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java index 89a549ed62cdb..9e123a19fcd1c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java @@ -53,7 +53,7 @@ public class Filter extends PTransform, PCollection> { } /** - * @deprecated use {@link byPredicate}, which returns a {@link Filter} transform instead of + * @deprecated use {@link #byPredicate}, which returns a {@link Filter} transform instead of * a {@link ParDo.Bound}. */ @Deprecated From 8db50044b6d38f06ee588e884f0d4a9ea2ae8f6e Mon Sep 17 00:00:00 2001 From: dhalperi Date: Tue, 1 Dec 2015 19:03:13 -0800 Subject: [PATCH 1205/1541] ByteArrayCoder: remove unused import, fix Javadoc link ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109150531 --- .../com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java index 13c0e0ab7074b..47efbedc66949 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteArrayCoder.java @@ -16,7 +16,6 @@ package com.google.cloud.dataflow.sdk.coders; -import com.google.cloud.dataflow.sdk.coders.Coder.Context; import com.google.cloud.dataflow.sdk.util.ExposedByteArrayOutputStream; import com.google.cloud.dataflow.sdk.util.StreamUtils; import com.google.cloud.dataflow.sdk.util.VarInt; @@ -117,8 +116,8 @@ public Object structuralValue(byte[] value) { /** * {@inheritDoc} * - * @return {@code true} since {@link #getEncodedElementByteSize()} runs in constant time using - * the {@code length} of the provided array. + * @return {@code true} since {@link #getEncodedElementByteSize(byte[], Context)} runs in + * constant time using the {@code length} of the provided array. */ @Override public boolean isRegisterByteSizeObserverCheap(byte[] value, Context context) { From f6ade89e2a34f44b27fb7a73e0088e384d1099ad Mon Sep 17 00:00:00 2001 From: dhalperi Date: Tue, 1 Dec 2015 22:12:43 -0800 Subject: [PATCH 1206/1541] Coders: fix Javadoc - throws are not linked - nothing is not a valid class - #f() invalid when f does not have a zero-args version ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109161228 --- .../google/cloud/dataflow/sdk/coders/CustomCoder.java | 2 +- .../cloud/dataflow/sdk/coders/DelegateCoder.java | 11 +++++------ .../sdk/coders/DeterministicStandardCoder.java | 6 +++--- .../cloud/dataflow/sdk/coders/IterableLikeCoder.java | 3 +-- .../google/cloud/dataflow/sdk/coders/MapCoder.java | 7 +++---- .../google/cloud/dataflow/sdk/coders/VarIntCoder.java | 2 +- .../cloud/dataflow/sdk/coders/VarLongCoder.java | 2 +- 7 files changed, 15 insertions(+), 18 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java index f88cdd87cf677..b34ef8cf6decf 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CustomCoder.java @@ -100,7 +100,7 @@ public CloudObject asCloudObject() { /** * {@inheritDoc} * - * @throws {@link NonDeterministicException}. A {@link CustomCoder} is presumed + * @throws NonDeterministicException a {@link CustomCoder} is presumed * nondeterministic. */ @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java index 365f1d43a139b..cdd882b07a198 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DelegateCoder.java @@ -27,7 +27,7 @@ /** * A {@code DelegateCoder} wraps a {@link Coder} for {@code IntermediateT} and - * encodes/decodes values of type {@code T}s by converting + * encodes/decodes values of type {@code T} by converting * to/from {@code IntermediateT} and then encoding/decoding using the underlying * {@code Coder}. * @@ -43,9 +43,8 @@ */ public class DelegateCoder extends CustomCoder { /** - * A {@link CodingFunction CodingFunction<InputT, OutputT>} is a serializable function - * from {@code InputT} to {@code OutputT} that - * may throw any {@code Exception}. + * A {@link DelegateCoder.CodingFunction CodingFunction<InputT, OutputT>} is a serializable + * function from {@code InputT} to {@code OutputT} that may throw any {@link Exception}. */ public static interface CodingFunction extends Serializable { public abstract OutputT apply(InputT input) throws Exception; @@ -80,8 +79,8 @@ public Coder getCoder() { * {@inheritDoc} * * @throws NonDeterministicException when the underlying coder's {@code verifyDeterministic()} - * throws a {@link NonDeterministicException}. For this to be safe, the intermediate - * {@code CodingFunction} must also be deterministic. + * throws a {@link Coder.NonDeterministicException}. For this to be safe, the + * intermediate {@code CodingFunction} must also be deterministic. */ @Override public void verifyDeterministic() throws NonDeterministicException { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DeterministicStandardCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DeterministicStandardCoder.java index 2382279957151..0e0018afd4d45 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DeterministicStandardCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/DeterministicStandardCoder.java @@ -19,7 +19,7 @@ /** * A {@link DeterministicStandardCoder} is a {@link StandardCoder} that is * deterministic, in the sense that for objects considered equal - * according to {@link Object#equals()}, the encoded bytes are + * according to {@link Object#equals(Object)}, the encoded bytes are * also equal. * * @param the type of the values being transcoded @@ -30,8 +30,8 @@ protected DeterministicStandardCoder() {} /** * {@inheritDoc} * - * @throws nothing unless overridden. - * A {@link DeterministicStandardCoder} is presumed deterministic. + * @throws NonDeterministicException never, unless overridden. A + * {@link DeterministicStandardCoder} is presumed deterministic. */ @Override public void verifyDeterministic() throws NonDeterministicException { } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java index 522e9b1ee74b2..7fb573a5c65c8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/IterableLikeCoder.java @@ -164,8 +164,7 @@ public List> getCoderArguments() { /** * {@inheritDoc} * - * @throws - * {@link NonDeterministicException} always. + * @throws NonDeterministicException always. * Encoding is not deterministic for the general {@link Iterable} case, as it depends * upon the type of iterable. This may allow two objects to compare as equal * while the encoding differs. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java index 5b8e750ea68cb..b6f31030e41f8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/MapCoder.java @@ -135,10 +135,9 @@ public List> getCoderArguments() { /** * {@inheritDoc} * - * @throws - * {@link NonDeterministicException} always. Not all maps have a deterministic encoding. - * For example, {@link HashMap} comparison does not depend on element order, so - * two {@link HashMap} instances may be equal but produce different encodings. + * @throws NonDeterministicException always. Not all maps have a deterministic encoding. + * For example, {@code HashMap} comparison does not depend on element order, so + * two {@code HashMap} instances may be equal but produce different encodings. */ @Override public void verifyDeterministic() throws NonDeterministicException { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java index 41be8b400466a..1010601f40553 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarIntCoder.java @@ -79,7 +79,7 @@ public boolean consistentWithEquals() { /** * {@inheritDoc} * - * @return {@code true}. {@link VarIntCoder#getEncodedElementByteSize()} runs in constant time. + * @return {@code true}. {@link #getEncodedElementByteSize(Integer, Context)} is cheap. */ @Override public boolean isRegisterByteSizeObserverCheap(Integer value, Context context) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java index 7530d64c2a4b2..177ea09d596cf 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/VarLongCoder.java @@ -78,7 +78,7 @@ public boolean consistentWithEquals() { /** * {@inheritDoc} * - * @return {@code true}. {@link VarLongCoder#getEncodedElementByteSize()} runs in constant time. + * @return {@code true}. {@link #getEncodedElementByteSize(Long, Context)} is cheap. */ @Override public boolean isRegisterByteSizeObserverCheap(Long value, Context context) { From 927c58665138f419b1771eb76fe0c15193669a88 Mon Sep 17 00:00:00 2001 From: dhalperi Date: Tue, 1 Dec 2015 22:13:55 -0800 Subject: [PATCH 1207/1541] Coder: fix javadoc ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109161310 --- .../main/java/com/google/cloud/dataflow/sdk/coders/Coder.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java index 75847219058af..89b35164d4990 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Coder.java @@ -35,7 +35,8 @@ import javax.annotation.Nullable; /** - * A {@link Coder} defines how to encode and decode values of type {@code T} into byte streams. + * A {@link Coder Coder<T>} defines how to encode and decode values of type {@code T} into + * byte streams. * *

{@link Coder} instances are serialized during job creation and deserialized * before use, via JSON serialization. See {@link SerializableCoder} for an example of a From 3bad26310570c9b62440ec2d09e50d7e8e9c78e2 Mon Sep 17 00:00:00 2001 From: dhalperi Date: Wed, 2 Dec 2015 11:11:26 -0800 Subject: [PATCH 1208/1541] BlockBasedSource: fix javadoc links ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109217232 --- .../com/google/cloud/dataflow/sdk/io/BlockBasedSource.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BlockBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BlockBasedSource.java index 62115dfbe5e69..9ef4cdcf30c15 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BlockBasedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BlockBasedSource.java @@ -185,8 +185,8 @@ protected boolean isAtSplitPoint() { } /** - * Reads the next record from the {@link getCurrentBlock() current block} if - * possible. Will call {@link readNextBlock()} to advance to the next block if not. + * Reads the next record from the {@link #getCurrentBlock() current block} if + * possible. Will call {@link #readNextBlock()} to advance to the next block if not. * *

The first record read from a block is treated as a split point. */ From d81e7506dab13d3443f0b4e83403cee9280d65a4 Mon Sep 17 00:00:00 2001 From: lcwik Date: Wed, 2 Dec 2015 14:18:08 -0800 Subject: [PATCH 1209/1541] Fix issue with SerializableCoder and JAXBCoder to not close stream Also add a guard in CoderProperties/CoderUtils to ensure that user coders when being tested do not attempt to close the stream. ----Release Notes---- Coders do not own the underlying stream and should not attempt to close it. [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109236509 --- .../cloud/dataflow/sdk/coders/JAXBCoder.java | 16 +++- .../sdk/coders/SerializableCoder.java | 7 +- .../dataflow/sdk/testing/CoderProperties.java | 13 +++- .../cloud/dataflow/sdk/util/CoderUtils.java | 4 +- .../dataflow/sdk/util/UnownedInputStream.java | 76 +++++++++++++++++++ .../sdk/util/UnownedOutputStream.java | 56 ++++++++++++++ .../sdk/testing/CoderPropertiesTest.java | 29 +++++++ .../dataflow/sdk/util/CoderUtilsTest.java | 44 +++++++++++ .../sdk/util/UnownedInputStreamTest.java | 76 +++++++++++++++++++ .../sdk/util/UnownedOutputStreamTest.java | 57 ++++++++++++++ 10 files changed, 368 insertions(+), 10 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UnownedInputStream.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UnownedOutputStream.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/UnownedInputStreamTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/UnownedOutputStreamTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/JAXBCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/JAXBCoder.java index 7275b81007bf3..2b0190b5f3d7c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/JAXBCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/JAXBCoder.java @@ -20,6 +20,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import java.io.FilterInputStream; +import java.io.FilterOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -67,7 +69,12 @@ public void encode(T value, OutputStream outStream, Context context) jaxbMarshaller = jaxbContext.createMarshaller(); } - jaxbMarshaller.marshal(value, outStream); + jaxbMarshaller.marshal(value, new FilterOutputStream(outStream) { + // JAXB closes the underyling stream so we must filter out those calls. + @Override + public void close() throws IOException { + } + }); } catch (JAXBException e) { throw new CoderException(e); } @@ -82,7 +89,12 @@ public T decode(InputStream inStream, Context context) throws CoderException, IO } @SuppressWarnings("unchecked") - T obj = (T) jaxbUnmarshaller.unmarshal(inStream); + T obj = (T) jaxbUnmarshaller.unmarshal(new FilterInputStream(inStream) { + // JAXB closes the underyling stream so we must filter out those calls. + @Override + public void close() throws IOException { + } + }); return obj; } catch (JAXBException e) { throw new CoderException(e); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java index e5f88c36fddb9..593c9f0f809b1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/SerializableCoder.java @@ -117,8 +117,10 @@ public Class getRecordType() { @Override public void encode(T value, OutputStream outStream, Context context) throws IOException, CoderException { - try (ObjectOutputStream oos = new ObjectOutputStream(outStream)) { + try { + ObjectOutputStream oos = new ObjectOutputStream(outStream); oos.writeObject(value); + oos.flush(); } catch (IOException exn) { throw new CoderException("unable to serialize record " + value, exn); } @@ -127,7 +129,8 @@ public void encode(T value, OutputStream outStream, Context context) @Override public T decode(InputStream inStream, Context context) throws IOException, CoderException { - try (ObjectInputStream ois = new ObjectInputStream(inStream)) { + try { + ObjectInputStream ois = new ObjectInputStream(inStream); return type.cast(ois.readObject()); } catch (ClassNotFoundException e) { throw new CoderException("unable to deserialize record", e); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/CoderProperties.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/CoderProperties.java index 4f0db99e6fff7..5705dc4c78b68 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/CoderProperties.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/testing/CoderProperties.java @@ -33,6 +33,9 @@ import com.google.cloud.dataflow.sdk.util.SerializableUtils; import com.google.cloud.dataflow.sdk.util.Serializer; import com.google.cloud.dataflow.sdk.util.Structs; +import com.google.cloud.dataflow.sdk.util.UnownedInputStream; +import com.google.cloud.dataflow.sdk.util.UnownedOutputStream; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterables; import java.io.ByteArrayInputStream; @@ -318,23 +321,25 @@ public static > void coderDecodesBase64Contents ////////////////////////////////////////////////////////////////////////// - private static byte[] encode( + @VisibleForTesting + static byte[] encode( Coder coder, Coder.Context context, T value) throws CoderException, IOException { @SuppressWarnings("unchecked") Coder deserializedCoder = Serializer.deserialize(coder.asCloudObject(), Coder.class); ByteArrayOutputStream os = new ByteArrayOutputStream(); - deserializedCoder.encode(value, os, context); + deserializedCoder.encode(value, new UnownedOutputStream(os), context); return os.toByteArray(); } - private static T decode( + @VisibleForTesting + static T decode( Coder coder, Coder.Context context, byte[] bytes) throws CoderException, IOException { @SuppressWarnings("unchecked") Coder deserializedCoder = Serializer.deserialize(coder.asCloudObject(), Coder.class); ByteArrayInputStream is = new ByteArrayInputStream(bytes); - return deserializedCoder.decode(is, context); + return deserializedCoder.decode(new UnownedInputStream(is), context); } private static T decodeEncode(Coder coder, Coder.Context context, T value) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CoderUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CoderUtils.java index 417305f455f93..84098a62d76fa 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CoderUtils.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CoderUtils.java @@ -116,7 +116,7 @@ public static byte[] encodeToByteArray(Coder coder, T value, Coder.Contex private static void encodeToSafeStream( Coder coder, T value, OutputStream stream, Coder.Context context) throws CoderException { try { - coder.encode(value, stream, context); + coder.encode(value, new UnownedOutputStream(stream), context); } catch (IOException exn) { Throwables.propagateIfPossible(exn, CoderException.class); throw new IllegalArgumentException( @@ -153,7 +153,7 @@ public static T decodeFromByteArray( private static T decodeFromSafeStream( Coder coder, InputStream stream, Coder.Context context) throws CoderException { try { - return coder.decode(stream, context); + return coder.decode(new UnownedInputStream(stream), context); } catch (IOException exn) { Throwables.propagateIfPossible(exn, CoderException.class); throw new IllegalArgumentException( diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UnownedInputStream.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UnownedInputStream.java new file mode 100644 index 0000000000000..3d80230a52cfc --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UnownedInputStream.java @@ -0,0 +1,76 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.common.base.MoreObjects; + +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** + * A {@link OutputStream} wrapper which protects against the user attempting to modify + * the underlying stream by closing it or using mark. + */ +public class UnownedInputStream extends FilterInputStream { + public UnownedInputStream(InputStream delegate) { + super(delegate); + } + + @Override + public void close() throws IOException { + throw new UnsupportedOperationException("Caller does not own the underlying input stream " + + " and should not call close()."); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof UnownedInputStream + && ((UnownedInputStream) obj).in.equals(in); + } + + @Override + public int hashCode() { + return in.hashCode(); + } + + @SuppressWarnings("UnsynchronizedOverridesSynchronized") + @Override + public void mark(int readlimit) { + throw new UnsupportedOperationException("Caller does not own the underlying input stream " + + " and should not call mark()."); + } + + @Override + public boolean markSupported() { + return false; + } + + @SuppressWarnings("UnsynchronizedOverridesSynchronized") + @Override + public void reset() throws IOException { + throw new UnsupportedOperationException("Caller does not own the underlying input stream " + + " and should not call reset()."); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(UnownedInputStream.class).add("in", in).toString(); + } +} + diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UnownedOutputStream.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UnownedOutputStream.java new file mode 100644 index 0000000000000..29187a1b9da6e --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/UnownedOutputStream.java @@ -0,0 +1,56 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import com.google.common.base.MoreObjects; + +import java.io.FilterOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +/** + * A {@link OutputStream} wrapper which protects against the user attempting to modify + * the underlying stream by closing it. + */ +public class UnownedOutputStream extends FilterOutputStream { + public UnownedOutputStream(OutputStream delegate) { + super(delegate); + } + + @Override + public void close() throws IOException { + throw new UnsupportedOperationException("Caller does not own the underlying output stream " + + " and should not call close()."); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof UnownedOutputStream + && ((UnownedOutputStream) obj).out.equals(out); + } + + @Override + public int hashCode() { + return out.hashCode(); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(UnownedOutputStream.class).add("out", out).toString(); + } +} + diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/CoderPropertiesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/CoderPropertiesTest.java index 30ec8137268b6..f0fe688459585 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/CoderPropertiesTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/testing/CoderPropertiesTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import com.google.cloud.dataflow.sdk.coders.Coder.Context; import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.coders.CustomCoder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; @@ -182,4 +183,32 @@ public void testBadCoderThatDependsOnStateThatIsLost() throws Exception { expectedException.expectMessage("I forgot something..."); CoderProperties.coderDecodeEncodeEqual(new ForgetfulSerializingCoder(1), "TestData"); } + + /** A coder which closes the underlying stream during encoding and decoding. */ + public static class ClosingCoder extends CustomCoder { + @Override + public void encode(String value, OutputStream outStream, Context context) throws IOException { + outStream.close(); + } + + @Override + public String decode(InputStream inStream, Context context) throws IOException { + inStream.close(); + return null; + } + } + + @Test + public void testClosingCoderFailsWhenDecoding() throws Exception { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Caller does not own the underlying"); + CoderProperties.decode(new ClosingCoder(), Context.NESTED, new byte[0]); + } + + @Test + public void testClosingCoderFailsWhenEncoding() throws Exception { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Caller does not own the underlying"); + CoderProperties.encode(new ClosingCoder(), Context.NESTED, "test-value"); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CoderUtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CoderUtilsTest.java index ef3f80c6aab96..e192f456bca68 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CoderUtilsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/CoderUtilsTest.java @@ -25,11 +25,13 @@ import com.google.cloud.dataflow.sdk.coders.AtomicCoder; import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.Coder.Context; import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.testing.CoderPropertiesTest.ClosingCoder; import org.hamcrest.CoreMatchers; import org.junit.Assert; @@ -182,4 +184,46 @@ public void testCreateUnknownCoder() throws Exception { "Unable to convert coder ID UnknownCoder to class")); } } + + @Test + public void testClosingCoderFailsWhenDecodingBase64() throws Exception { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Caller does not own the underlying"); + CoderUtils.decodeFromBase64(new ClosingCoder(), "test-value"); + } + + @Test + public void testClosingCoderFailsWhenDecodingByteArray() throws Exception { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Caller does not own the underlying"); + CoderUtils.decodeFromByteArray(new ClosingCoder(), new byte[0]); + } + + @Test + public void testClosingCoderFailsWhenDecodingByteArrayInContext() throws Exception { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Caller does not own the underlying"); + CoderUtils.decodeFromByteArray(new ClosingCoder(), new byte[0], Context.NESTED); + } + + @Test + public void testClosingCoderFailsWhenEncodingToBase64() throws Exception { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Caller does not own the underlying"); + CoderUtils.encodeToBase64(new ClosingCoder(), "test-value"); + } + + @Test + public void testClosingCoderFailsWhenEncodingToByteArray() throws Exception { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Caller does not own the underlying"); + CoderUtils.encodeToByteArray(new ClosingCoder(), "test-value"); + } + + @Test + public void testClosingCoderFailsWhenEncodingToByteArrayInContext() throws Exception { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Caller does not own the underlying"); + CoderUtils.encodeToByteArray(new ClosingCoder(), "test-value", Context.NESTED); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/UnownedInputStreamTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/UnownedInputStreamTest.java new file mode 100644 index 0000000000000..30da6ae7238b5 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/UnownedInputStreamTest.java @@ -0,0 +1,76 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.ByteArrayInputStream; + +/** Unit tests for {@link UnownedInputStream}. */ +@RunWith(JUnit4.class) +public class UnownedInputStreamTest { + @Rule public ExpectedException expectedException = ExpectedException.none(); + private ByteArrayInputStream bais; + private UnownedInputStream os; + + @Before + public void setup() { + bais = new ByteArrayInputStream(new byte[]{ 1, 2, 3 }); + os = new UnownedInputStream(bais); + } + + @Test + public void testHashCodeEqualsAndToString() throws Exception { + assertEquals(bais.hashCode(), os.hashCode()); + assertEquals("UnownedInputStream{in=" + bais + "}", os.toString()); + assertEquals(new UnownedInputStream(bais), os); + } + + @Test + public void testClosingThrows() throws Exception { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Caller does not own the underlying"); + expectedException.expectMessage("close()"); + os.close(); + } + + @Test + public void testMarkThrows() throws Exception { + assertFalse(os.markSupported()); + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Caller does not own the underlying"); + expectedException.expectMessage("mark()"); + os.mark(1); + } + + @Test + public void testResetThrows() throws Exception { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Caller does not own the underlying"); + expectedException.expectMessage("reset()"); + os.reset(); + } +} + diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/UnownedOutputStreamTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/UnownedOutputStreamTest.java new file mode 100644 index 0000000000000..eea70fe6cb9cf --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/UnownedOutputStreamTest.java @@ -0,0 +1,57 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.junit.Assert.assertEquals; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.ByteArrayOutputStream; + +/** Unit tests for {@link UnownedOutputStream}. */ +@RunWith(JUnit4.class) +public class UnownedOutputStreamTest { + @Rule public ExpectedException expectedException = ExpectedException.none(); + private ByteArrayOutputStream baos; + private UnownedOutputStream os; + + @Before + public void setup() { + baos = new ByteArrayOutputStream(); + os = new UnownedOutputStream(baos); + } + + @Test + public void testHashCodeEqualsAndToString() throws Exception { + assertEquals(baos.hashCode(), os.hashCode()); + assertEquals("UnownedOutputStream{out=" + baos + "}", os.toString()); + assertEquals(new UnownedOutputStream(baos), os); + } + + @Test + public void testClosingThrows() throws Exception { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Caller does not own the underlying"); + os.close(); + } +} + From 0c637d38a89eecf7d31bee913f00e1c89365e379 Mon Sep 17 00:00:00 2001 From: robertwb Date: Wed, 2 Dec 2015 14:41:02 -0800 Subject: [PATCH 1210/1541] Occasionally compact combined state on write. This is needed to avoid arbitrarily large blowup for many-element windows. Now, with a fixed probability (currently 0.2%) when writing the state, instead of doing a blind write we read in all accumulated values, combine, and replace everything with the final combined value. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109239045 --- .../worker/WindmillStateInternals.java | 36 +++++++++ .../worker/WindmillStateInternalsTest.java | 79 ++++++++++++++++--- 2 files changed, 105 insertions(+), 10 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java index 6f0b44c4c4559..9f88c78de915c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java @@ -33,6 +33,7 @@ import com.google.cloud.dataflow.sdk.util.state.StateTag.StateBinder; import com.google.cloud.dataflow.sdk.util.state.ValueState; import com.google.cloud.dataflow.sdk.util.state.WatermarkStateInternal; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Supplier; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.Futures; @@ -45,6 +46,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Random; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -102,6 +104,35 @@ public ValueState bindValue(StateTag> address, Coder cod private final boolean useStateFamilies; private final Supplier scopedReadStateSupplier; + @VisibleForTesting + static final ThreadLocal> COMPACT_NOW = + new ThreadLocal() { + public Supplier initialValue() { + return new Supplier() { + /* The rate at which, on average, this will return true. */ + static final double RATE = 0.002; + Random random = new Random(); + long counter = nextSample(); + + private long nextSample() { + // Use geometric distribution to find next true value. + // This lets us avoid invoking random.nextDouble() on every call. + return (long) Math.floor(Math.log(random.nextDouble()) / Math.log(1 - RATE)); + } + + public Boolean get() { + counter--; + if (counter < 0) { + counter = nextSample(); + return true; + } else { + return false; + } + } + }; + } + }; + public WindmillStateInternals(String prefix, boolean useStateFamilies, WindmillStateReader reader, Supplier scopedReadStateSupplier) { this.prefix = prefix; @@ -569,6 +600,11 @@ public void clear() { @Override public void persist(Windmill.WorkItemCommitRequest.Builder commitBuilder) throws IOException { if (hasLocalAdditions) { + // TODO: Take into account whether it's in the cache. + if (COMPACT_NOW.get().get()) { + // Implicitly clears the bag and combines local and persisted accumulators. + localAdditionsAccum = getAccum().read(); + } bag.add(combineFn.compact(localAdditionsAccum)); localAdditionsAccum = combineFn.createAccumulator(); hasLocalAdditions = false; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java index b976e05051af4..a63d9a0417cea 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java @@ -42,6 +42,7 @@ import com.google.cloud.dataflow.sdk.util.state.ValueState; import com.google.cloud.dataflow.sdk.util.state.WatermarkStateInternal; import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.SettableFuture; import com.google.protobuf.ByteString; @@ -70,6 +71,7 @@ public class WindmillStateInternalsTest { private static final StateTag> COMBINING_ADDR = StateTags.combiningValueFromInputInternal( "combining", VarIntCoder.of(), new Sum.SumIntegerFn()); + private static final ByteString COMBINING_KEY = key(NAMESPACE, "combining"); private final Coder accumCoder = new Sum.SumIntegerFn().getAccumulatorCoder(null, VarIntCoder.of()); @@ -81,11 +83,11 @@ public class WindmillStateInternalsTest { @Mock private Supplier readStateSupplier; - private ByteString key(StateNamespace namespace, String addrId) { + private static ByteString key(StateNamespace namespace, String addrId) { return key("", namespace, addrId); } - private ByteString key(String prefix, StateNamespace namespace, String addrId) { + private static ByteString key(String prefix, StateNamespace namespace, String addrId) { return ByteString.copyFromUtf8(prefix + namespace.stringKey() + "+u" + addrId); } @@ -286,7 +288,7 @@ public void testCombiningAddBeforeRead() throws Exception { CombiningValueState value = underTest.state(NAMESPACE, COMBINING_ADDR); SettableFuture> future = SettableFuture.create(); - when(mockReader.listFuture(key(NAMESPACE, "combining"), STATE_FAMILY, accumCoder)) + when(mockReader.listFuture(COMBINING_KEY, STATE_FAMILY, accumCoder)) .thenReturn(future); StateContents result = value.get(); @@ -326,10 +328,10 @@ public void testCombiningIsEmpty() throws Exception { CombiningValueState value = underTest.state(NAMESPACE, COMBINING_ADDR); SettableFuture> future = SettableFuture.create(); - when(mockReader.listFuture(key(NAMESPACE, "combining"), STATE_FAMILY, accumCoder)) + when(mockReader.listFuture(COMBINING_KEY, STATE_FAMILY, accumCoder)) .thenReturn(future); StateContents result = value.isEmpty(); - Mockito.verify(mockReader).listFuture(key(NAMESPACE, "combining"), STATE_FAMILY, accumCoder); + Mockito.verify(mockReader).listFuture(COMBINING_KEY, STATE_FAMILY, accumCoder); waitAndSet(future, Arrays.asList(new int[] {29}), 200); assertThat(result.read(), Matchers.is(false)); @@ -342,7 +344,7 @@ public void testCombiningIsEmptyAfterClear() throws Exception { value.clear(); StateContents result = value.isEmpty(); Mockito.verify(mockReader, never()) - .listFuture(key(NAMESPACE, "combining"), STATE_FAMILY, accumCoder); + .listFuture(COMBINING_KEY, STATE_FAMILY, accumCoder); assertThat(result.read(), Matchers.is(true)); value.add(87); @@ -351,6 +353,8 @@ public void testCombiningIsEmptyAfterClear() throws Exception { @Test public void testCombiningAddPersist() throws Exception { + disableCompactOnWrite(); + CombiningValueState value = underTest.state(NAMESPACE, COMBINING_ADDR); value.add(5); @@ -363,7 +367,7 @@ public void testCombiningAddPersist() throws Exception { assertEquals(1, commitBuilder.getListUpdatesCount()); TagList listUpdates = commitBuilder.getListUpdates(0); - assertEquals(key(NAMESPACE, "combining"), listUpdates.getTag()); + assertEquals(COMBINING_KEY, listUpdates.getTag()); assertEquals(1, listUpdates.getValuesCount()); assertEquals( 11, @@ -375,8 +379,45 @@ public void testCombiningAddPersist() throws Exception { Mockito.verifyNoMoreInteractions(mockReader); } + @Test + public void testCombiningAddPersistWithCompact() throws Exception { + forceCompactOnWrite(); + + Mockito.stub( + mockReader.listFuture( + org.mockito.Matchers.any(), + org.mockito.Matchers.any(), + org.mockito.Matchers.>any())) + .toReturn( + Futures.>immediateFuture( + ImmutableList.of(new int[] {40}, new int[] {60}))); + + CombiningValueState value = underTest.state(NAMESPACE, COMBINING_ADDR); + + value.add(5); + value.add(6); + + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + underTest.persist(commitBuilder); + + assertEquals(2, commitBuilder.getListUpdatesCount()); + assertEquals(0, commitBuilder.getListUpdates(0).getValuesCount()); + + TagList listUpdates = commitBuilder.getListUpdates(1); + assertEquals(COMBINING_KEY, listUpdates.getTag()); + assertEquals(1, listUpdates.getValuesCount()); + assertEquals( + 111, + CoderUtils.decodeFromByteArray( + accumCoder, listUpdates.getValues(0).getData().substring(1).toByteArray())[ + 0]); + } + @Test public void testCombiningClearPersist() throws Exception { + disableCompactOnWrite(); + CombiningValueState value = underTest.state(NAMESPACE, COMBINING_ADDR); value.clear(); @@ -390,12 +431,12 @@ public void testCombiningClearPersist() throws Exception { assertEquals(2, commitBuilder.getListUpdatesCount()); TagList listClear = commitBuilder.getListUpdates(0); - assertEquals(key(NAMESPACE, "combining"), listClear.getTag()); + assertEquals(COMBINING_KEY, listClear.getTag()); assertEquals(Long.MAX_VALUE, listClear.getEndTimestamp()); assertEquals(0, listClear.getValuesCount()); TagList listUpdates = commitBuilder.getListUpdates(1); - assertEquals(key(NAMESPACE, "combining"), listUpdates.getTag()); + assertEquals(COMBINING_KEY, listUpdates.getTag()); assertEquals(1, listUpdates.getValuesCount()); assertEquals( 11, @@ -403,7 +444,7 @@ public void testCombiningClearPersist() throws Exception { accumCoder, listUpdates.getValues(0).getData().substring(1).toByteArray())[0]); // Blind adds should not need to read the future. - Mockito.verify(mockReader).listFuture(key(NAMESPACE, "combining"), STATE_FAMILY, accumCoder); + Mockito.verify(mockReader).listFuture(COMBINING_KEY, STATE_FAMILY, accumCoder); Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @@ -866,4 +907,22 @@ public void testValueNoStateFamilies() throws Exception { assertEquals("World", value.get().read()); } + + private void disableCompactOnWrite() { + WindmillStateInternals.COMPACT_NOW.set( + new Supplier() { + public Boolean get() { + return false; + } + }); + } + + private void forceCompactOnWrite() { + WindmillStateInternals.COMPACT_NOW.set( + new Supplier() { + public Boolean get() { + return true; + } + }); + } } From cf1e3f341e52e94e51273436043c4841d9146d81 Mon Sep 17 00:00:00 2001 From: davor Date: Tue, 1 Dec 2015 13:12:14 -0800 Subject: [PATCH 1211/1541] Version management Prepare codebase for version 1.4.0. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109117479 --- examples/pom.xml | 2 +- maven-archetypes/examples/pom.xml | 2 +- maven-archetypes/starter/pom.xml | 2 +- pom.xml | 2 +- sdk/pom.xml | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index bd02ea8f9bc83..d33c610a1a32c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -22,7 +22,7 @@ com.google.cloud.dataflow google-cloud-dataflow-java-sdk-parent - 1.2.1-SNAPSHOT + 1.4.0-SNAPSHOT com.google.cloud.dataflow diff --git a/maven-archetypes/examples/pom.xml b/maven-archetypes/examples/pom.xml index 8b0d5e2c0fa7f..42a553aa02575 100644 --- a/maven-archetypes/examples/pom.xml +++ b/maven-archetypes/examples/pom.xml @@ -22,7 +22,7 @@ com.google.cloud.dataflow google-cloud-dataflow-java-sdk-parent - 1.2.1-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/maven-archetypes/starter/pom.xml b/maven-archetypes/starter/pom.xml index 0298889abf0a8..fee9e33947633 100644 --- a/maven-archetypes/starter/pom.xml +++ b/maven-archetypes/starter/pom.xml @@ -22,7 +22,7 @@ com.google.cloud.dataflow google-cloud-dataflow-java-sdk-parent - 1.2.1-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index f352b8435fa91..099352d7cf0b9 100644 --- a/pom.xml +++ b/pom.xml @@ -35,7 +35,7 @@ http://cloud.google.com/dataflow 2013 - 1.2.1-SNAPSHOT + 1.4.0-SNAPSHOT diff --git a/sdk/pom.xml b/sdk/pom.xml index 4514e2121ff4b..376f84a993034 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -22,7 +22,7 @@ com.google.cloud.dataflow google-cloud-dataflow-java-sdk-parent - 1.2.1-SNAPSHOT + 1.4.0-SNAPSHOT com.google.cloud.dataflow From 25236730f8bdc6ca38a70e19514a6b42cf16afce Mon Sep 17 00:00:00 2001 From: kirpichov Date: Thu, 3 Dec 2015 14:33:52 -0800 Subject: [PATCH 1212/1541] Improvements to DataflowWorkProgressUpdaterTest Improves workProgressUpdaterAdaptsProgressInterval by bringing when() close to verify() to improve readability, and using relative rather than absolute time and increased timeouts to decrease flakiness. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109347801 --- .../DataflowWorkProgressUpdaterTest.java | 79 +++++++++---------- 1 file changed, 39 insertions(+), 40 deletions(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java index a6d60d946ca38..c7ec363285d5b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java @@ -132,7 +132,6 @@ public void setWorkerProgress(ApproximateProgress progress) { private WorkItem workItem; private DataflowWorkerHarnessOptions options; private DataflowWorkProgressUpdater progressUpdater; - private long nowMillis; @Before public void initMocksAndWorkflowServiceAndWorkerAndWork() { @@ -151,13 +150,12 @@ public Collection> getOutputMetrics() { return metrics; } }; - nowMillis = System.currentTimeMillis(); workItem = new WorkItem(); workItem.setProjectId(PROJECT_ID); workItem.setJobId(JOB_ID); workItem.setId(WORK_ID); - workItem.setLeaseExpireTime(toCloudTime(new Instant(nowMillis + 1000))); + workItem.setLeaseExpireTime(toCloudTime(new Instant(System.currentTimeMillis() + 1000))); workItem.setReportStatusInterval(toCloudDuration(Duration.millis(300))); workItem.setInitialReportIndex(1L); @@ -181,7 +179,7 @@ protected long getLeaseRenewalLatencyMargin() { @Test(timeout = 1000) public void workProgressUpdaterUpdates() throws Exception { when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) - .thenReturn(generateServiceState(nowMillis + 2000, 1000, null, 2L)); + .thenReturn(generateServiceState(System.currentTimeMillis() + 2000, 1000, null, 2L)); setUpCounters(2); setUpMetrics(3); setUpProgress(approximateProgressAtIndex(1L)); @@ -196,59 +194,59 @@ public void workProgressUpdaterUpdates() throws Exception { // Verifies that ReportWorkItemStatusRequest contains correct progress report // and actual dynamic split result. - @Test(timeout = 5000) + @Test(timeout = 10000) public void workProgressUpdaterAdaptsProgressInterval() throws Exception { - // Mock that the next reportProgress call will return a response that asks - // us to truncate the task at index 3, and the next two will not ask us to - // truncate at all. - when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) - .thenReturn(generateServiceState(nowMillis + 2000, 1000, positionAtIndex(3L), 2L)) - .thenReturn(generateServiceState(nowMillis + 3000, 2000, null, 3L)) - .thenReturn(generateServiceState(nowMillis + 1000, 3000, null, 4L)) - .thenReturn(generateServiceState(nowMillis + 4000, 3000, null, 5L)); - setUpCounters(3); setUpMetrics(2); setUpProgress(approximateProgressAtIndex(1L)); progressUpdater.startReportingProgress(); - // The initial update should be sent after 300. - verify(workUnitClient, timeout(400)) - .reportWorkItemStatus(argThat( - new ExpectedDataflowWorkItemStatus().withCounters(3).withMetrics(2).withProgress( - approximateProgressAtIndex(1L)).withReportIndex(1L))); + + // In tests below, we allow 500ms leeway. + + when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) + // leaseExpirationTimestamp, progressReportInterval, suggestedStopPosition, nextReportIndex + .thenReturn(generateServiceState( + System.currentTimeMillis() + 2000, 1000, positionAtIndex(3L), 2L)); + // The initial update should be sent at nowMillis+300 (+500ms leeway). + verify(workUnitClient, timeout(800)).reportWorkItemStatus(argThat( + new ExpectedDataflowWorkItemStatus().withCounters(3).withMetrics(2).withProgress( + approximateProgressAtIndex(1L)).withReportIndex(1L))); setUpCounters(5); setUpMetrics(6); setUpProgress(approximateProgressAtIndex(2L)); - // The second update should be sent after one second as requested. - verify(workUnitClient, timeout(1100)) - .reportWorkItemStatus(argThat( - new ExpectedDataflowWorkItemStatus() - .withCounters(5) - .withMetrics(6) - .withProgress(approximateProgressAtIndex(2L)) - .withDynamicSplitAtPosition(positionAtIndex(3L)) - .withReportIndex(2L))); + when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) + .thenReturn(generateServiceState(System.currentTimeMillis() + 3000, 2000, null, 3L)); + // The second update should be sent after ~1000ms (previous requested report interval). + verify(workUnitClient, timeout(1500)).reportWorkItemStatus(argThat( + new ExpectedDataflowWorkItemStatus() + .withCounters(5) + .withMetrics(6) + .withProgress(approximateProgressAtIndex(2L)) + .withDynamicSplitAtPosition(positionAtIndex(3L)) + .withReportIndex(2L))); // After the request is sent, reset cached dynamic split result to null. assertNull(progressUpdater.getDynamicSplitResultToReport()); setUpProgress(approximateProgressAtIndex(3L)); - // The third update should be sent after 2 seconds. - verify(workUnitClient, timeout(2100)) - .reportWorkItemStatus(argThat( - new ExpectedDataflowWorkItemStatus().withProgress(approximateProgressAtIndex(3L)) - .withReportIndex(3L))); + when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) + .thenReturn(generateServiceState(System.currentTimeMillis() + 1000, 3000, null, 4L)); + // The third update should be sent after ~2000ms (previous requested report interval). + verify(workUnitClient, timeout(2500)).reportWorkItemStatus(argThat( + new ExpectedDataflowWorkItemStatus().withProgress(approximateProgressAtIndex(3L)) + .withReportIndex(3L))); setUpProgress(approximateProgressAtIndex(4L)); - // The fourth update should not respect the suggested report interval. - // It should be sent before the lease expires - verify(workUnitClient, timeout(900)) - .reportWorkItemStatus(argThat( - new ExpectedDataflowWorkItemStatus().withProgress(approximateProgressAtIndex(4L)) - .withReportIndex(4L))); + when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) + .thenReturn(generateServiceState(System.currentTimeMillis() + 4000, 3000, null, 5L)); + // The fourth update should not respect the suggested report interval (3000ms) + // because the lease expires in 1000ms. The update should be sent before the lease expires. + verify(workUnitClient, timeout(900)).reportWorkItemStatus(argThat( + new ExpectedDataflowWorkItemStatus().withProgress(approximateProgressAtIndex(4L)) + .withReportIndex(4L))); progressUpdater.stopReportingProgress(); @@ -261,7 +259,8 @@ public void workProgressUpdaterSendsLastPendingUpdateWhenStopped() throws Except // The setup process sends one update after 300ms. Enqueue another that should be scheduled // 1000ms after that. when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) - .thenReturn(generateServiceState(nowMillis + 2000, 1000, positionAtIndex(2L), 2L)); + .thenReturn(generateServiceState( + System.currentTimeMillis() + 2000, 1000, positionAtIndex(2L), 2L)); setUpProgress(approximateProgressAtIndex(1L)); progressUpdater.startReportingProgress(); From 8c8f0857c267ba172ab498998c1a13fa803c32ef Mon Sep 17 00:00:00 2001 From: tgroh Date: Fri, 4 Dec 2015 11:53:14 -0800 Subject: [PATCH 1213/1541] Reduce availability of StepContext#getExecutionContext getExecutionContext is used in only one place, from a concrete instance of StreamingModeExecutionContext.StepContext, and is immediately downcast to StreamingModeExecutionContext. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109435007 --- .../runners/worker/StreamingModeExecutionContext.java | 4 ++++ .../sdk/runners/worker/StreamingSideInputDoFnRunner.java | 5 +---- .../cloud/dataflow/sdk/util/BaseExecutionContext.java | 5 ----- .../google/cloud/dataflow/sdk/util/ExecutionContext.java | 5 ----- .../runners/worker/StreamingSideInputDoFnRunnerTest.java | 9 ++++----- 5 files changed, 9 insertions(+), 19 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java index 279c53b7f2b27..40eadbc72e1cc 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java @@ -396,6 +396,10 @@ public void flushState() { timerInternals.persistTo(outputBuilder); } + public Iterable getSideInputNotifications() { + return StreamingModeExecutionContext.this.getSideInputNotifications(); + } + @Override public void writePCollectionViewData( TupleTag tag, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunner.java index 85e79b6438073..d37aaa4a7ead6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunner.java @@ -65,7 +65,6 @@ public class StreamingSideInputDoFnRunner extends DoFnRunner { private StreamingModeExecutionContext.StepContext stepContext; - private StreamingModeExecutionContext execContext; private Map> sideInputViews; private final StateTag>> elementsAddr; @@ -100,8 +99,6 @@ public StreamingSideInputDoFnRunner( for (PCollectionView view : doFnInfo.getSideInputViews()) { sideInputViews.put(view.getTagInternal().getId(), view); } - this.execContext = - (StreamingModeExecutionContext) stepContext.getExecutionContext(); this.blockedMapAddr = blockedMapAddr(windowFn); this.elementsAddr = StateTags.makeSystemTagInternal(StateTags.bag("elem", @@ -129,7 +126,7 @@ StateTag>>> blockedMapAddr(WindowFn getReadyWindows() { Set readyWindows = new HashSet<>(); - for (Windmill.GlobalDataId id : execContext.getSideInputNotifications()) { + for (Windmill.GlobalDataId id : stepContext.getSideInputNotifications()) { if (sideInputViews.get(id.getTag()) == null) { // Side input is for a different DoFn; ignore it. continue; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BaseExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BaseExecutionContext.java index f4862decdddc8..76771c5a073dd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BaseExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BaseExecutionContext.java @@ -111,11 +111,6 @@ public String getTransformName() { return transformName; } - @Override - public ExecutionContext getExecutionContext() { - return executionContext; - } - @Override public void noteOutput(WindowedValue output) { executionContext.noteOutput(output); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ExecutionContext.java index 77c66b29a4e58..971c886c88b73 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ExecutionContext.java @@ -70,11 +70,6 @@ public interface StepContext { */ String getTransformName(); - /** - * The context in which this step is executing. - */ - ExecutionContext getExecutionContext(); - /** * Hook for subclasses to implement that will be called whenever * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#output} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunnerTest.java index 503474b12cb16..2f470140a8b76 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunnerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunnerTest.java @@ -86,7 +86,6 @@ public class StreamingSideInputDoFnRunnerTest { @Before public void setUp() { MockitoAnnotations.initMocks(this); - when(stepContext.getExecutionContext()).thenReturn(execContext); when(stepContext.stateInternals()).thenReturn(state); } @@ -94,7 +93,7 @@ public void setUp() { public void testSideInputReady() throws Exception { PCollectionView view = createView(); - when(execContext.getSideInputNotifications()) + when(stepContext.getSideInputNotifications()) .thenReturn(Arrays.asList()); when(stepContext.issueSideInputFetch( eq(view), any(BoundedWindow.class), eq(SideInputState.UNKNOWN))) @@ -119,7 +118,7 @@ public void testSideInputReady() throws Exception { public void testSideInputNotReady() throws Exception { PCollectionView view = createView(); - when(execContext.getSideInputNotifications()) + when(stepContext.getSideInputNotifications()) .thenReturn(Arrays.asList()); when(stepContext.issueSideInputFetch( eq(view), any(BoundedWindow.class), eq(SideInputState.UNKNOWN))) @@ -185,7 +184,7 @@ public void testSideInputNotification() throws Exception { runner.watermarkHold(createWindow(0)).add(new Instant(0)); runner.elementBag(createWindow(0)).add(createDatum("e", 0)); - when(execContext.getSideInputNotifications()).thenReturn(Arrays.asList(id)); + when(stepContext.getSideInputNotifications()).thenReturn(Arrays.asList(id)); when(stepContext.issueSideInputFetch( eq(view), any(BoundedWindow.class), eq(SideInputState.UNKNOWN))) .thenReturn(false); @@ -229,7 +228,7 @@ public void testMultipleSideInputs() throws Exception { StreamingSideInputDoFnRunner.blockedMapAddr(WINDOW_FN)); blockedMapState.set(blockedMap); - when(execContext.getSideInputNotifications()).thenReturn(Arrays.asList(id)); + when(stepContext.getSideInputNotifications()).thenReturn(Arrays.asList(id)); when(stepContext.issueSideInputFetch( any(PCollectionView.class), any(BoundedWindow.class), any(SideInputState.class))) .thenReturn(true); From 9e74526d4ca83d469b1ac8607a28590618c9dabc Mon Sep 17 00:00:00 2001 From: mariand Date: Fri, 4 Dec 2015 23:53:09 -0800 Subject: [PATCH 1214/1541] Fixed a race in StateSampler. stateTimestampNs must be initialized before starting the background thread which uses it, otherwise the initialization will race with run(). ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109479932 --- .../cloud/dataflow/sdk/util/common/worker/StateSampler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSampler.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSampler.java index 742c86b120327..df916a0a6abb8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSampler.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/common/worker/StateSampler.java @@ -119,6 +119,7 @@ public StateSampler(String prefix, // The current implementation uses a fixed-rate timer with a period samplingPeriodMs as a // trampoline to a one-shot random timer which fires with a random delay within // samplingPeriodMs. + stateTimestampNs = System.nanoTime(); invocationTriggerFuture = executorService.scheduleAtFixedRate( new Runnable() { @@ -145,7 +146,6 @@ public void run() { 0, samplingPeriodMs, TimeUnit.MILLISECONDS); - stateTimestampNs = System.nanoTime(); } /** From 3ae688368a3edaba6f08cb5a9c0d4d33aaff1387 Mon Sep 17 00:00:00 2001 From: tgroh Date: Mon, 7 Dec 2015 11:25:29 -0800 Subject: [PATCH 1215/1541] Genericize BaseExecutionContext This allows implementors that care about specific implementation details fo the execution context to return a more specific type of ExecutionContext.StepContext from getOrCreateStepContext. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109601837 --- .../worker/DataflowExecutionContext.java | 4 ++- .../worker/MapTaskExecutorFactory.java | 12 +++---- .../worker/StreamingModeExecutionContext.java | 5 +-- .../sdk/util/BaseExecutionContext.java | 33 ++++++++++++++----- .../sdk/util/BatchModeExecutionContext.java | 7 ++-- .../sdk/util/DirectModeExecutionContext.java | 7 ++-- .../dataflow/sdk/util/ExecutionContext.java | 2 +- .../worker/MapTaskExecutorFactoryTest.java | 5 +-- .../StreamingGroupAlsoByWindowsDoFnTest.java | 4 +-- ...ngGroupAlsoByWindowsReshuffleDoFnTest.java | 4 +-- 10 files changed, 53 insertions(+), 30 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowExecutionContext.java index 241641e2f1201..cd3ba394b5c67 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowExecutionContext.java @@ -18,13 +18,15 @@ import com.google.api.services.dataflow.model.SideInputInfo; import com.google.cloud.dataflow.sdk.util.BaseExecutionContext; +import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.SideInputReader; import com.google.cloud.dataflow.sdk.values.PCollectionView; /** * Extensions to {@link BaseExecutionContext} specific to the Dataflow worker. */ -public abstract class DataflowExecutionContext extends BaseExecutionContext { +public abstract class DataflowExecutionContext + extends BaseExecutionContext { /** * Returns a {@link SideInputReader} for all the side inputs described in the given * {@link SideInputInfo} descriptors. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java index 243f3f40ad5f1..d227e833eccaa 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactory.java @@ -80,7 +80,7 @@ public class MapTaskExecutorFactory { public static MapTaskExecutor create( PipelineOptions options, MapTask mapTask, - DataflowExecutionContext context, + DataflowExecutionContext context, CounterSet counters, StateSampler stateSampler) throws Exception { return create( @@ -100,7 +100,7 @@ public static MapTaskExecutor create( PipelineOptions options, MapTask mapTask, ReaderFactory.Registry registry, - DataflowExecutionContext context, + DataflowExecutionContext context, CounterSet counters, StateSampler stateSampler) throws Exception { @@ -133,7 +133,7 @@ public static MapTaskExecutor create( static Operation createOperation( PipelineOptions options, ParallelInstruction instruction, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, List priorOperations, String counterPrefix, String systemStageName, @@ -160,7 +160,7 @@ static Operation createOperation( PipelineOptions options, ParallelInstruction instruction, ReaderFactory.Registry registry, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, List priorOperations, String counterPrefix, String systemStageName, @@ -199,7 +199,7 @@ static ReadOperation createReadOperation( PipelineOptions options, ParallelInstruction instruction, ReaderFactory.Registry registry, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, @SuppressWarnings("unused") List priorOperations, String counterPrefix, String systemStageName, @@ -244,7 +244,7 @@ static WriteOperation createWriteOperation(PipelineOptions options, static ParDoOperation createParDoOperation( PipelineOptions options, ParallelInstruction instruction, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, List priorOperations, String counterPrefix, CounterSet.AddCounterMutator addCounterMutator, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java index 40eadbc72e1cc..d57ed1fe5bd78 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java @@ -56,7 +56,8 @@ /** * {@link ExecutionContext} for use in streaming mode. */ -public class StreamingModeExecutionContext extends DataflowExecutionContext { +public class StreamingModeExecutionContext + extends DataflowExecutionContext { private final String stageName; private final Map, Map> sideInputCache; // Per-key cache of active Reader objects in use by this process. @@ -99,7 +100,7 @@ public void start( } @Override - public ExecutionContext.StepContext createStepContext( + public StepContext createStepContext( String stepName, String transformName, StateSampler stateSampler) { StepContext context = new StepContext(stepName, transformName, stateSampler); context.start(stateReader, inputDataWatermark); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BaseExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BaseExecutionContext.java index 76771c5a073dd..9d048962038e7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BaseExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BaseExecutionContext.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -31,18 +32,34 @@ * Base class for implementations of {@link ExecutionContext}. * *

A concrete subclass should implement {@link #createStepContext} to create the appropriate - * {@link ExecutionContext.StepContext} implementation. Any {@code StepContext} created will + * {@link StepContext} implementation. Any {@code StepContext} created will * be cached for the lifetime of this {@link ExecutionContext}. + * + *

BaseExecutionContext is generic to allow implementing subclasses to return a concrete subclass + * of {@link StepContext} from {@link #getOrCreateStepContext(String, String, StateSampler)} and + * {@link #getAllStepContexts()} without forcing each subclass to override the method, e.g. + *

+ * @Override
+ * StreamingModeExecutionContext.StepContext getOrCreateStepContext(...) {
+ *   return (StreamingModeExecutionContext.StepContext) super.getOrCreateStepContext(...);
+ * }
+ * 
+ * + *

When a subclass of {@code BaseExecutionContext} has been downcast, the return types of + * {@link #createStepContext(String, String, StateSampler)}, + * {@link #getOrCreateStepContext(String, String, StateSampler}, and {@link #getAllStepContexts()} + * will be appropriately specialized. */ -public abstract class BaseExecutionContext implements ExecutionContext { +public abstract class BaseExecutionContext + implements ExecutionContext { - private Map cachedStepContexts = new HashMap<>(); + private Map cachedStepContexts = new HashMap<>(); /** * Implementations should override this to create the specific type * of {@link StepContext} they need. */ - protected abstract ExecutionContext.StepContext createStepContext( + protected abstract T createStepContext( String stepName, String transformName, StateSampler stateSampler); @@ -50,9 +67,9 @@ protected abstract ExecutionContext.StepContext createStepContext( * Returns the {@link StepContext} associated with the given step. */ @Override - public ExecutionContext.StepContext getOrCreateStepContext( + public T getOrCreateStepContext( String stepName, String transformName, StateSampler stateSampler) { - ExecutionContext.StepContext context = cachedStepContexts.get(stepName); + T context = cachedStepContexts.get(stepName); if (context == null) { context = createStepContext(stepName, transformName, stateSampler); cachedStepContexts.put(stepName, context); @@ -64,8 +81,8 @@ public ExecutionContext.StepContext getOrCreateStepContext( * Returns a collection view of all of the {@link StepContext}s. */ @Override - public Collection getAllStepContexts() { - return cachedStepContexts.values(); + public Collection getAllStepContexts() { + return Collections.unmodifiableCollection(cachedStepContexts.values()); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java index 4477bb0a54e6c..d4f239c826c59 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchModeExecutionContext.java @@ -31,7 +31,8 @@ /** * {@link ExecutionContext} for use in batch mode. */ -public class BatchModeExecutionContext extends DataflowExecutionContext { +public class BatchModeExecutionContext + extends DataflowExecutionContext { private Object key; private PipelineOptions options; @@ -59,7 +60,7 @@ public static BatchModeExecutionContext fromOptions(PipelineOptions options) { * Create a new {@link ExecutionContext.StepContext}. */ @Override - protected ExecutionContext.StepContext createStepContext( + protected StepContext createStepContext( String stepName, String transformName, StateSampler stateSampler) { return new StepContext(stepName, transformName); } @@ -117,7 +118,7 @@ public SideInputReader getSideInputReaderForViews( /** * {@link ExecutionContext.StepContext} used in batch mode. */ - class StepContext extends BaseExecutionContext.StepContext { + public class StepContext extends BaseExecutionContext.StepContext { private final InMemoryStateInternals stateInternals = new InMemoryStateInternals(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DirectModeExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DirectModeExecutionContext.java index d9474a61130a4..12f2d208d696c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DirectModeExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DirectModeExecutionContext.java @@ -31,7 +31,8 @@ /** * {@link ExecutionContext} for use in direct mode. */ -public class DirectModeExecutionContext extends BaseExecutionContext { +public class DirectModeExecutionContext + extends BaseExecutionContext { private Object key; private List> output = Lists.newArrayList(); @@ -44,7 +45,7 @@ public static DirectModeExecutionContext create() { } @Override - protected ExecutionContext.StepContext createStepContext( + protected StepContext createStepContext( String stepName, String transformName, StateSampler stateSampler) { return new StepContext(this, stepName, transformName); } @@ -96,7 +97,7 @@ public List> getSideOutput(TupleTag tag) { /** * {@link ExecutionContext.StepContext} used in direct mode. */ - static class StepContext extends BaseExecutionContext.StepContext { + public static class StepContext extends BaseExecutionContext.StepContext { private final Map stateInternals = Maps.newHashMap(); private InMemoryStateInternals currentStateInternals = null; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ExecutionContext.java index 971c886c88b73..83d74e6668abd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ExecutionContext.java @@ -39,7 +39,7 @@ StepContext getOrCreateStepContext( /** * Returns a collection view of all of the {@link StepContext}s. */ - Collection getAllStepContexts(); + Collection getAllStepContexts(); /** * Hook for subclasses to implement that will be called whenever diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java index 8778d7b7842a5..f8799bd9afd48 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/MapTaskExecutorFactoryTest.java @@ -202,7 +202,7 @@ public void testExecutionContextPlumbing() throws Exception { mapTask.setStageName("test"); mapTask.setInstructions(instructions); - DataflowExecutionContext context = BatchModeExecutionContext.fromOptions(options); + BatchModeExecutionContext context = BatchModeExecutionContext.fromOptions(options); CounterSet counters = new CounterSet(); try (MapTaskExecutor executor = @@ -413,7 +413,8 @@ public void testCreateParDoOperation() throws Exception { ParallelInstruction instruction = createParDoInstruction(producerIndex, producerOutputNum, "DoFn"); - DataflowExecutionContext context = BatchModeExecutionContext.fromOptions(options); + BatchModeExecutionContext context = + BatchModeExecutionContext.fromOptions(options); CounterSet counterSet = new CounterSet(); String counterPrefix = "test-"; String systemStageName = "stageName"; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsDoFnTest.java index a03dfed3c9560..6dadab49b4d53 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsDoFnTest.java @@ -102,9 +102,9 @@ public class StreamingGroupAlsoByWindowsDoFnTest { // StreamingGroupAlsoByWindows expects it to. So, hook that up. @Override - public ExecutionContext.StepContext createStepContext( + public StepContext createStepContext( String stepName, String transformName, StateSampler stateSampler) { - ExecutionContext.StepContext context = + StepContext context = Mockito.spy(super.createStepContext(stepName, transformName, stateSampler)); Mockito.doReturn(mockTimerInternals).when(context).timerInternals(); return context; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java index 01c0f3359b580..a17aca014ae5a 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java @@ -77,9 +77,9 @@ public class StreamingGroupAlsoByWindowsReshuffleDoFnTest { // StreamingGroupAlsoByWindows expects it to. So, hook that up. @Override - public ExecutionContext.StepContext createStepContext( + public StepContext createStepContext( String stepName, String transformName, StateSampler stateSampler) { - ExecutionContext.StepContext context = + StepContext context = Mockito.spy(super.createStepContext(stepName, transformName, stateSampler)); Mockito.doReturn(null).when(context).timerInternals(); return context; From cb5a5192d9a5562aa95c4c114c499ac0a4ec8c8f Mon Sep 17 00:00:00 2001 From: chamikara Date: Mon, 7 Dec 2015 15:03:12 -0800 Subject: [PATCH 1216/1541] Fixes a null pointer in SourceOperationExecutor. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109625553 --- .../worker/SourceOperationExecutor.java | 18 ++++++++++++++---- .../sdk/runners/worker/DataflowWorkerTest.java | 15 +++++++++++++++ 2 files changed, 29 insertions(+), 4 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java index f66c4d6e16574..f66476ab8e3f7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceOperationExecutor.java @@ -16,11 +16,13 @@ package com.google.cloud.dataflow.sdk.runners.worker; +import com.google.api.client.json.JsonFactory; import com.google.api.services.dataflow.model.SourceOperationRequest; import com.google.api.services.dataflow.model.SourceOperationResponse; import com.google.api.services.dataflow.model.SourceSplitResponse; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.dataflow.CustomSources; +import com.google.cloud.dataflow.sdk.util.Transport; import com.google.cloud.dataflow.sdk.util.common.CounterSet; import com.google.cloud.dataflow.sdk.util.common.worker.WorkExecutor; @@ -67,17 +69,25 @@ public SourceOperationResponse getResponse() { return response; } - static boolean isSplitResponseTooLarge(SourceOperationResponse operationResponse) { + static int determineSplitResponseSize(SourceOperationResponse operationResponse) { try { SourceSplitResponse splitResponse = operationResponse.getSplit(); - int size = splitResponse.getFactory().toByteArray(operationResponse).length; - return size >= SOURCE_OPERATION_RESPONSE_SIZE_LIMIT_MB * 1024 * 1024; + JsonFactory factory = splitResponse.getFactory(); + if (factory == null) { + factory = Transport.getJsonFactory(); + } + return factory.toByteArray(operationResponse).length; } catch (OutOfMemoryError e) { LOG.error("Got exception when trying to serialize split response: " + e.getMessage()); // We will go out of memory if split response is extremely large. - return true; + return Integer.MAX_VALUE; } catch (IOException e) { throw new RuntimeException(e); } } + + static boolean isSplitResponseTooLarge(SourceOperationResponse operationResponse) { + return determineSplitResponseSize(operationResponse) + >= SOURCE_OPERATION_RESPONSE_SIZE_LIMIT_MB * 1024 * 1024; + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java index 58d55ff2773bd..d575d6f429913 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkerTest.java @@ -18,18 +18,23 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.argThat; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.google.api.services.dataflow.model.SourceOperationResponse; +import com.google.api.services.dataflow.model.SourceSplitResponse; +import com.google.api.services.dataflow.model.SourceSplitShard; import com.google.api.services.dataflow.model.WorkItem; import com.google.api.services.dataflow.model.WorkItemStatus; import com.google.cloud.dataflow.sdk.options.DataflowWorkerHarnessOptions; import com.google.cloud.dataflow.sdk.testing.FastNanoClockAndSleeper; import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; import com.google.cloud.dataflow.sdk.util.common.worker.WorkExecutor; +import com.google.common.collect.ImmutableList; import org.eclipse.jetty.server.LocalConnector; import org.eclipse.jetty.server.Server; @@ -138,6 +143,16 @@ public void testUnknownHandler() throws Exception { assertThat(response, containsString("HTTP/1.1 404 Not Found")); } + @Test + public void testIsSplitResponseTooLarge() { + SourceSplitResponse splitResponse = new SourceSplitResponse(); + splitResponse.setShards( + ImmutableList.of(new SourceSplitShard(), new SourceSplitShard())); + assertTrue( + SourceOperationExecutor.determineSplitResponseSize( + new SourceOperationResponse().setSplit(splitResponse)) > 0); + } + @Test public void testWorkItemStatusWithStateSamplerInfo() throws Exception { WorkItem workItem = new WorkItem() From f52c53f05f4cdf93b6dd4f2115e292a2699bdae5 Mon Sep 17 00:00:00 2001 From: malo Date: Mon, 7 Dec 2015 17:13:19 -0800 Subject: [PATCH 1217/1541] Remove NullPointerException in DataflowWorkProgressUpdaterTest Only start progress updates when the mock has been set. Otherwise there is a race between setting the mock return value and sending the progress update. This change reduces the flakiness of this test. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109638627 --- .../sdk/runners/worker/DataflowWorkProgressUpdaterTest.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java index c7ec363285d5b..b46d34eab3531 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java @@ -199,7 +199,6 @@ public void workProgressUpdaterAdaptsProgressInterval() throws Exception { setUpCounters(3); setUpMetrics(2); setUpProgress(approximateProgressAtIndex(1L)); - progressUpdater.startReportingProgress(); // In tests below, we allow 500ms leeway. @@ -207,6 +206,10 @@ public void workProgressUpdaterAdaptsProgressInterval() throws Exception { // leaseExpirationTimestamp, progressReportInterval, suggestedStopPosition, nextReportIndex .thenReturn(generateServiceState( System.currentTimeMillis() + 2000, 1000, positionAtIndex(3L), 2L)); + + // Start progress updates. + progressUpdater.startReportingProgress(); + // The initial update should be sent at nowMillis+300 (+500ms leeway). verify(workUnitClient, timeout(800)).reportWorkItemStatus(argThat( new ExpectedDataflowWorkItemStatus().withCounters(3).withMetrics(2).withProgress( From f111e6762a3404c3241564808460789cffcff41c Mon Sep 17 00:00:00 2001 From: markshields Date: Tue, 8 Dec 2015 18:01:34 -0800 Subject: [PATCH 1218/1541] Refine LATE pane semantics and implementation * Rework definition of PaneInfo.{EARLY, ON_TIME, LATE} so that panes with only late data are always LATE, and an ON_TIME pane can never cause a later computation to yield a LATE pane. * Clarify watermark at T means any message with timestamp < T is 'late'. * Clarify timers fire when watermark progresses after timer's timestamp. * Distinguish 'input' from 'output' watermarks. * Protect against overflow in Windmill <-> Harness timestamp conversions. ----Release Notes---- Rework definition of PaneInfo.{EARLY, ON_TIME, LATE} so that panes with only late data are always LATE, and an ON_TIME pane can never cause a later computation to yield a LATE pane. [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109749349 --- .../dataflow/sdk/io/UnboundedSource.java | 12 +- .../sdk/runners/worker/PubsubSink.java | 4 +- .../sdk/runners/worker/StateFetcher.java | 7 +- .../worker/StreamingDataflowWorker.java | 53 ++- .../worker/StreamingModeExecutionContext.java | 73 +++- .../worker/StreamingSideInputDoFnRunner.java | 8 +- .../sdk/runners/worker/WindmillSink.java | 4 +- .../worker/WindmillStateInternals.java | 9 +- .../sdk/runners/worker/WindmillTimeUtils.java | 106 +++++ .../transforms/windowing/BoundedWindow.java | 2 +- .../sdk/transforms/windowing/PaneInfo.java | 88 +++- .../sdk/util/BatchTimerInternals.java | 36 +- ...GroupAlsoByWindowsViaOutputBufferDoFn.java | 15 +- .../dataflow/sdk/util/NonEmptyPanes.java | 2 +- .../dataflow/sdk/util/PaneInfoTracker.java | 95 ++++- .../dataflow/sdk/util/ReduceFnRunner.java | 402 +++++++++--------- .../dataflow/sdk/util/TimerInternals.java | 65 ++- .../dataflow/sdk/util/WatermarkHold.java | 394 ++++++++++++++--- .../dataflow/sdk/util/WindowTracing.java | 36 ++ .../state/MergedWatermarkStateInternal.java | 14 +- .../dataflow/sdk/util/state/StateTags.java | 16 +- .../util/state/WatermarkStateInternal.java | 13 +- .../runners/dataflow/CustomSourcesTest.java | 3 +- .../sdk/runners/worker/KeyedWorkItemTest.java | 5 +- .../worker/StreamingDataflowWorkerTest.java | 42 +- .../StreamingGroupAlsoByWindowsDoFnTest.java | 16 +- ...ngGroupAlsoByWindowsReshuffleDoFnTest.java | 3 +- .../StreamingModeExecutionContextTest.java | 8 +- .../worker/WindmillStateReaderTest.java | 4 +- .../transforms/windowing/AfterAllTest.java | 10 +- .../transforms/windowing/AfterEachTest.java | 4 +- .../transforms/windowing/AfterFirstTest.java | 2 +- .../windowing/AfterProcessingTimeTest.java | 2 +- .../windowing/AfterWatermarkTest.java | 48 +-- .../windowing/DefaultTriggerTest.java | 23 +- .../windowing/OrFinallyTriggerTest.java | 2 +- .../sdk/util/BatchTimerInternalsTest.java | 4 +- .../util/GroupAlsoByWindowsProperties.java | 68 ++- .../sdk/util/TriggerExecutorTest.java | 187 ++++---- .../dataflow/sdk/util/TriggerTester.java | 318 ++++++++++---- .../dataflow/sdk/util/state/StateTagTest.java | 5 +- 41 files changed, 1539 insertions(+), 669 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillTimeUtils.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowTracing.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/UnboundedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/UnboundedSource.java index c904dd25a74fe..e585151c892a6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/UnboundedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/UnboundedSource.java @@ -96,7 +96,8 @@ public boolean requiresDeduping() { } /** - * A marker representing the progress and state of an {@link UnboundedReader}. + * A marker representing the progress and state of an + * {@link com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader}. * *

For example, this could be offsets in a set of files being read. */ @@ -176,24 +177,25 @@ public byte[] getCurrentRecordId() throws NoSuchElementException { } /** - * Returns a lower bound on timestamps of future elements read by this reader. + * Returns a timestamp before or at the timestamps of all future elements read by this reader. * *

This can be approximate. If records are read that violate this guarantee, they will be * considered late, which will affect how they will be processed. See * {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} for more information on * late data and how to handle it. * - *

This bound should be as tight as possible. Downstream windows will not be able to close - * until this watermark passes the end of the window. + *

However, this value should be as late as possible. Downstream windows may not be able + * to close until this watermark passes their end. * *

For example, a source may know that the records it reads will be in timestamp order. In - * this case, the watermark can be the timestamp of the last record read minus one. For a + * this case, the watermark can be the timestamp of the last record read. For a * source that does not have natural timestamps, timestamps can be set to the time of * reading, in which case the watermark is the current clock time. * *

See {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window} and * {@link com.google.cloud.dataflow.sdk.transforms.windowing.Trigger} for more * information on timestamps and watermarks. + * *

May be called after {@link #advance} or {@link #start} has returned false, but not before * {@link #start} has been called. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PubsubSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PubsubSink.java index 47557b720102e..2c7c142fbe1d1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PubsubSink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/PubsubSink.java @@ -29,7 +29,6 @@ import com.google.protobuf.ByteString; import java.io.IOException; -import java.util.concurrent.TimeUnit; /** * A sink that writes to Pubsub, via a Windmill server. @@ -97,11 +96,10 @@ private ByteString encode(Coder coder, T object) throws IOException { public long add(WindowedValue data) throws IOException { ByteString byteString = encode(coder, data); - long timestampMicros = TimeUnit.MILLISECONDS.toMicros(data.getTimestamp().getMillis()); outputBuilder.addMessages( Windmill.Message.newBuilder() .setData(byteString) - .setTimestamp(timestampMicros) + .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(data.getTimestamp())) .build()); return byteString.size(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StateFetcher.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StateFetcher.java index b096193436201..3f2aa38adc776 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StateFetcher.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StateFetcher.java @@ -117,11 +117,10 @@ public SideInputCacheEntry call() throws Exception { .setVersion(windowStream.toByteString()) .build()) .setStateFamily(stateFamily) - .setExistenceWatermarkDeadline( - TimeUnit.MILLISECONDS.toMicros(sideWindowStrategy + .setExistenceWatermarkDeadline(WindmillTimeUtils.harnessToWindmillTimestamp( + sideWindowStrategy .getTrigger().getSpec() - .getWatermarkThatGuaranteesFiring(sideWindow) - .getMillis())) + .getWatermarkThatGuaranteesFiring(sideWindow))) .build(); Windmill.GetDataResponse response; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java index 141271c36ae71..81bab4e2402c9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -74,6 +74,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.Nullable; import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; @@ -418,16 +419,27 @@ private void dispatchLoop() { continue; } - long watermarkMicros = computationWork.getInputDataWatermark(); - final Instant inputDataWatermark = new Instant(watermarkMicros / 1000); + // May be null if input watermark not yet known. + // TODO: Can assert this is non-null once Windmill waits for known input watermark. + @Nullable + final Instant inputDataWatermark = + WindmillTimeUtils.windmillToHarnessInputWatermark( + computationWork.getInputDataWatermark()); ActiveWorkForComputation activeWork = activeWorkMap.get(computation); for (final Windmill.WorkItem workItem : computationWork.getWorkList()) { + // May be null if output watermark not yet known. + @Nullable + final Instant outputDataWatermark = + WindmillTimeUtils.windmillToHarnessOutputWatermark( + workItem.getOutputDataWatermark()); + Preconditions.checkState(inputDataWatermark == null || outputDataWatermark == null + || !outputDataWatermark.isAfter(inputDataWatermark)); Work work = new Work(workItem.getWorkToken()) { - @Override - public void run() { - process(computation, mapTask, inputDataWatermark, workItem); - } - }; + @Override + public void run() { + process(computation, mapTask, inputDataWatermark, outputDataWatermark, workItem); + } + }; if (activeWork.activateWork(workItem.getKey(), work)) { workUnitExecutor.execute(work); } @@ -447,10 +459,8 @@ public long getWorkToken() { } } - private void process( - final String computation, - final MapTask mapTask, - final Instant inputDataWatermark, + private void process(final String computation, final MapTask mapTask, + @Nullable final Instant inputDataWatermark, @Nullable final Instant outputDataWatermark, final Windmill.WorkItem work) { LOG.debug("Starting processing for {}:\n{}", computation, work); @@ -494,10 +504,10 @@ private void process( ParallelInstruction read = mapTask.getInstructions().get(0); if (CustomSources.class.getName().equals( read.getRead().getSource().getSpec().get("@type"))) { + Coder coder = Serializer.deserialize(read.getOutputs().get(0).getCodec(), Coder.class); readOperation.receivers[0].addOutputCounter( new OutputObjectAndByteCounter( - new MapTaskExecutorFactory.ElementByteSizeObservableCoder<>( - Serializer.deserialize(read.getOutputs().get(0).getCodec(), Coder.class)), + new MapTaskExecutorFactory.ElementByteSizeObservableCoder<>(coder), worker.getOutputCounters().getAddCounterMutator()) .setSamplingPeriod(100) .countBytes("dataflow_input_size-" + mapTask.getSystemName())); @@ -510,7 +520,8 @@ private void process( WindmillStateReader stateReader = new WindmillStateReader( metricTrackingWindmillServer, computation, work.getKey(), work.getWorkToken()); StateFetcher localStateFetcher = stateFetcher.byteTrackingView(); - context.start(work, inputDataWatermark, stateReader, localStateFetcher, outputBuilder); + context.start(work, inputDataWatermark, outputDataWatermark, stateReader, localStateFetcher, + outputBuilder); for (Long callbackId : context.getReadyCommitCallbackIds()) { final Runnable callback = commitCallbacks.remove(callbackId); @@ -521,6 +532,7 @@ public void run() { try { callback.run(); } catch (Throwable t) { + // TODO: Count interesting failures. LOG.error("Source checkpoint finalization failed:", t); } } @@ -607,13 +619,12 @@ public void run() { if (reportFailure(computation, work, t)) { // Try again, after some delay and at the end of the queue to avoid a tight loop. sleep(10000); - workUnitExecutor.forceExecute( - new Runnable() { - @Override - public void run() { - process(computation, mapTask, inputDataWatermark, work); - } - }); + workUnitExecutor.forceExecute(new Runnable() { + @Override + public void run() { + process(computation, mapTask, inputDataWatermark, outputDataWatermark, work); + } + }); } else { // If we failed to report the error, the item is invalid and should // not be retried internally. It will be retried at the higher level. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java index d57ed1fe5bd78..116d3c6a7dc5d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java @@ -51,7 +51,8 @@ import java.util.Set; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; + +import javax.annotation.Nullable; /** * {@link ExecutionContext} for use in streaming mode. @@ -60,12 +61,14 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext { private final String stageName; private final Map, Map> sideInputCache; + // Per-key cache of active Reader objects in use by this process. private final ConcurrentMap readerCache; private final ConcurrentMap stateNameMap; private Windmill.WorkItem work; - private Instant inputDataWatermark; + @Nullable private Instant inputDataWatermark; + @Nullable private Instant outputDataWatermark; private WindmillStateReader stateReader; private StateFetcher stateFetcher; private Windmill.WorkItemCommitRequest.Builder outputBuilder; @@ -83,19 +86,21 @@ public StreamingModeExecutionContext( public void start( Windmill.WorkItem work, - Instant inputDataWatermark, + @Nullable Instant inputDataWatermark, + @Nullable Instant outputDataWatermark, WindmillStateReader stateReader, StateFetcher stateFetcher, Windmill.WorkItemCommitRequest.Builder outputBuilder) { this.work = work; this.inputDataWatermark = inputDataWatermark; + this.outputDataWatermark = outputDataWatermark; this.stateReader = stateReader; this.stateFetcher = stateFetcher; this.outputBuilder = outputBuilder; this.sideInputCache.clear(); for (ExecutionContext.StepContext stepContext : getAllStepContexts()) { - ((StepContext) stepContext).start(stateReader, inputDataWatermark); + ((StepContext) stepContext).start(stateReader, inputDataWatermark, outputDataWatermark); } } @@ -103,7 +108,7 @@ public void start( public StepContext createStepContext( String stepName, String transformName, StateSampler stateSampler) { StepContext context = new StepContext(stepName, transformName, stateSampler); - context.start(stateReader, inputDataWatermark); + context.start(stateReader, inputDataWatermark, outputDataWatermark); return context; } @@ -232,6 +237,7 @@ public void run() { } }); + @SuppressWarnings("unchecked") Coder checkpointCoder = ((UnboundedSource) activeReader.getCurrentSource()) .getCheckpointMarkCoder(); @@ -244,7 +250,8 @@ public void run() { } sourceStateBuilder.setState(stream.toByteString()); } - outputBuilder.setSourceWatermark(TimeUnit.MILLISECONDS.toMicros(watermark.getMillis())); + outputBuilder.setSourceWatermark( + WindmillTimeUtils.harnessToWindmillTimestamp(watermark)); long backlogBytes = activeReader.getSplitBacklogBytes(); if (backlogBytes == UnboundedSource.UnboundedReader.BACKLOG_UNKNOWN @@ -285,13 +292,16 @@ public static ByteString timerTag(TimerData key) { } private static class WindmillTimerInternals implements TimerInternals { - private Map timers = new HashMap<>(); - private Instant inputDataWatermark; + @Nullable private Instant inputDataWatermark; + @Nullable private Instant outputDataWatermark; private String stateFamily; - public WindmillTimerInternals(String stateFamily, Instant inputDataWatermark) { + public WindmillTimerInternals( + String stateFamily, @Nullable Instant inputDataWatermark, + @Nullable Instant outputDataWatermark) { this.inputDataWatermark = inputDataWatermark; + this.outputDataWatermark = outputDataWatermark; this.stateFamily = stateFamily; } @@ -310,11 +320,36 @@ public Instant currentProcessingTime() { return Instant.now(); } + /** + * {@inheritDoc} + * + *

Note that this value may be arbitrarily behind the global input watermark. Windmill + * simply reports the last known input watermark value at the time the GetWork response was + * constructed. However, if an element in a GetWork request has a timestamp at or ahead + * of the local input watermark then Windmill will not allow the local input watermark + * to advance until that element has been committed. + */ @Override - public Instant currentWatermarkTime() { + @Nullable + public Instant currentInputWatermarkTime() { return inputDataWatermark; } + /** + * {@inheritDoc} + * + *

Note that Windmill will provisionally hold the output watermark to the timestamp of the + * earliest element in a computation's GetWork response. (Elements with timestamps already + * behind the output watermark at the point the GetWork response is constructed will have + * no influence on the output watermark). The provisional hold will last until this work item is + * committed. It is the responsibility of the harness to impose any persistent holds it needs. + */ + @Override + @Nullable + public Instant currentOutputWatermarkTime() { + return outputDataWatermark; + } + public void persistTo(Windmill.WorkItemCommitRequest.Builder outputBuilder) { for (Entry entry : timers.entrySet()) { Windmill.Timer.Builder timer = outputBuilder.addOutputTimersBuilder() @@ -326,9 +361,8 @@ public void persistTo(Windmill.WorkItemCommitRequest.Builder outputBuilder) { // If the timer was being set (not deleted) then set a timestamp for it. if (entry.getValue()) { - long timestampMicros = - TimeUnit.MILLISECONDS.toMicros(entry.getKey().getTimestamp().getMillis()); - timer.setTimestamp(timestampMicros); + timer.setTimestamp( + WindmillTimeUtils.harnessToWindmillTimestamp(entry.getKey().getTimestamp())); } } timers.clear(); @@ -383,13 +417,14 @@ public StateSampler.ScopedState get() { /** * Update the {@code stateReader} used by this {@code StepContext}. */ - public void start(WindmillStateReader stateReader, Instant inputDataWatermark) { + public void start( + WindmillStateReader stateReader, @Nullable Instant inputDataWatermark, + @Nullable Instant outputDataWatermark) { boolean useStateFamilies = !stateNameMap.isEmpty(); - this.stateInternals = - new WindmillStateInternals( - prefix, useStateFamilies, stateReader, scopedReadStateSupplier); - this.timerInternals = new WindmillTimerInternals( - stateFamily, Preconditions.checkNotNull(inputDataWatermark)); + this.stateInternals = new WindmillStateInternals( + prefix, useStateFamilies, stateReader, scopedReadStateSupplier); + this.timerInternals = + new WindmillTimerInternals(stateFamily, inputDataWatermark, outputDataWatermark); } public void flushState() { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunner.java index d37aaa4a7ead6..dcbdc2546a5e5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingSideInputDoFnRunner.java @@ -53,7 +53,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.TimeUnit; /** * Runs a DoFn by constructing the appropriate contexts and passing them in. @@ -286,12 +285,11 @@ private Windmill.GlobalDataRequest buildGlob .setTag(view.getTagInternal().getId()) .setVersion(windowStream.toByteString()) .build()) - .setExistenceWatermarkDeadline( - TimeUnit.MILLISECONDS.toMicros(sideWindowStrategy + .setExistenceWatermarkDeadline(WindmillTimeUtils.harnessToWindmillTimestamp( + sideWindowStrategy .getTrigger() .getSpec() - .getWatermarkThatGuaranteesFiring(sideInputWindow) - .getMillis())) + .getWatermarkThatGuaranteesFiring(sideInputWindow))) .build(); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillSink.java index 1d6b15cebb9cb..9fc6cf75effbd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillSink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillSink.java @@ -41,7 +41,6 @@ import java.util.Collection; import java.util.HashMap; import java.util.Map; -import java.util.concurrent.TimeUnit; class WindmillSink extends Sink> { private WindmillStreamWriter writer; @@ -143,9 +142,8 @@ public long add(WindowedValue data) throws IOException { productionMap.put(key, keyedOutput); } - long timestampMicros = TimeUnit.MILLISECONDS.toMicros(data.getTimestamp().getMillis()); Windmill.Message.Builder builder = Windmill.Message.newBuilder() - .setTimestamp(timestampMicros) + .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(data.getTimestamp())) .setData(value) .setMetadata(metadata); keyedOutput.addMessages(builder.build()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java index 9f88c78de915c..9f79ad6463839 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java @@ -49,7 +49,6 @@ import java.util.Random; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; /** * Implementation of {@link StateInternals} using Windmill to manage the underlying data. @@ -496,7 +495,7 @@ public void persist(Windmill.WorkItemCommitRequest.Builder commitBuilder) { .setTag(stateKey) .setStateFamily(stateFamily) .setReset(true) - .addTimestamps(TimeUnit.MILLISECONDS.toMicros(localAdditions.getMillis())); + .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); } else if (!cleared && localAdditions != null){ // Otherwise, we need to combine the local additions with the already persisted data combineWithPersisted(commitBuilder); @@ -529,7 +528,8 @@ private void combineWithPersisted(Windmill.WorkItemCommitRequest.Builder commitB commitBuilder.addWatermarkHoldsBuilder() .setTag(stateKey) .setStateFamily(stateFamily) - .addTimestamps(TimeUnit.MILLISECONDS.toMicros(localAdditions.getMillis())); + .addTimestamps( + WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); } else { // The non-fast path does a read-modify-write Instant priorHold; @@ -546,7 +546,8 @@ private void combineWithPersisted(Windmill.WorkItemCommitRequest.Builder commitB .setTag(stateKey) .setStateFamily(stateFamily) .setReset(true) - .addTimestamps(TimeUnit.MILLISECONDS.toMicros(combinedHold.getMillis())); + .addTimestamps( + WindmillTimeUtils.harnessToWindmillTimestamp(combinedHold)); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillTimeUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillTimeUtils.java new file mode 100644 index 0000000000000..e107aea3806cb --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillTimeUtils.java @@ -0,0 +1,106 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.common.base.Preconditions; + +import org.joda.time.Instant; + +import javax.annotation.Nullable; + +/** + * Some timestamp conversion helpers for working with Windmill. + */ +class WindmillTimeUtils { + /** + * Convert a Windmill output watermark to a harness watermark. + * + *

Windmill tracks time in microseconds while the harness uses milliseconds. + * Windmill will 'speculatively' hold the output watermark for a computation to the + * earliest input message timestamp, provided that message timestamp is at or after + * the current output watermark. Thus for soundness we must ensure + * 'Windmill considers message late' implies 'harness considers message late'. Thus we + * round up when converting from microseconds to milliseconds. + * + *

In other words, harness output watermark >= windmill output watermark. + */ + @Nullable + static Instant windmillToHarnessOutputWatermark(long watermarkUs) { + if (watermarkUs == Long.MIN_VALUE) { + // Unknown. + return null; + } else if (watermarkUs == Long.MAX_VALUE) { + // End of time. + return BoundedWindow.TIMESTAMP_MAX_VALUE; + } else { + // Round up to nearest millisecond. + return new Instant((watermarkUs + 999) / 1000); + } + } + + /** + * Convert a Windmill input watermark to a harness input watermark. + * + *

We round down, thus harness input watermark <= windmill output watermark. + */ + @Nullable + static Instant windmillToHarnessInputWatermark(long watermarkUs) { + if (watermarkUs == Long.MIN_VALUE) { + // Unknown. + return null; + } else if (watermarkUs == Long.MAX_VALUE) { + // End of time. + return BoundedWindow.TIMESTAMP_MAX_VALUE; + } else { + // Round down to nearest millisecond. + return new Instant(watermarkUs / 1000); + } + } + + /** + * Convert a Windmill message timestamp to a harness timestamp. + * + *

For soundness we require the test + * {@code harness message timestamp >= harness output watermark} to imply + * {@code windmill message timestamp >= windmill output watermark}. Thus + * we round timestamps down and output watermarks up. + */ + static Instant windmillToHarnessTimestamp(long timestampUs) { + // Windmill should never send us an unknown timestamp. + Preconditions.checkArgument(timestampUs != Long.MIN_VALUE); + if (timestampUs == Long.MAX_VALUE) { + // End of time. + return BoundedWindow.TIMESTAMP_MAX_VALUE; + } else { + // Round down to nearest millisecond. + return new Instant(timestampUs / 1000); + } + } + + /** + * Convert a harness timestamp to a Windmill timestamp. + */ + static long harnessToWindmillTimestamp(Instant timestamp) { + if (timestamp.equals(BoundedWindow.TIMESTAMP_MAX_VALUE)) { + // End of time. + return Long.MAX_VALUE; + } else { + return timestamp.getMillis() * 1000; + } + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/BoundedWindow.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/BoundedWindow.java index 6b3ba58802671..0afd8e33c2d7c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/BoundedWindow.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/BoundedWindow.java @@ -40,7 +40,7 @@ public abstract class BoundedWindow { new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE)); /** - * Returns the upper bound of timestamps for values in this window. + * Returns the inclusive upper bound of timestamps for values in this window. */ public abstract Instant maxTimestamp(); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PaneInfo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PaneInfo.java index 67d2d1b06b07a..18f7a973cc9d2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PaneInfo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/PaneInfo.java @@ -19,6 +19,7 @@ import com.google.cloud.dataflow.sdk.coders.AtomicCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.GroupByKey; import com.google.cloud.dataflow.sdk.util.VarInt; import com.google.common.base.MoreObjects; @@ -31,26 +32,95 @@ import java.util.Objects; /** - * Provides information about the pane this value belongs to. Every pane is implicitly associated - * with a window. + * Provides information about the pane an element belongs to. Every pane is implicitly associated + * with a window. Panes are observable only via the + * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.ProcessContext#pane} method of the context + * passed to a {@link DoFn#processElement} overridden method. * *

Note: This does not uniquely identify a pane, and should not be used for comparisons. */ public final class PaneInfo { - /** - * Enumerates the possibilities for how the timing of this pane firing related to the watermark. + * Enumerates the possibilities for the timing of this pane firing related to the + * input and output watermarks for its computation. + * + *

A window may fire multiple panes, and the timing of those panes generally follows the + * regular expression {@code EARLY* ON_TIME? LATE*}. Generally a pane is considered: + *

    + *
  1. {@code EARLY} if the system cannot be sure it has seen all data which may contribute + * to the pane's window. + *
  2. {@code ON_TIME} if the system predicts it has seen all the data which may contribute + * to the pane's window. + *
  3. {@code LATE} if the system has encountered new data after predicting no more could arrive. + * It is possible an {@code ON_TIME} pane has already been emitted, in which case any + * following panes are considered {@code LATE}. + *
+ * + *

Only an + * {@link AfterWatermark#pastEndOfWindow} trigger may produce an {@code ON_TIME} pane. + * With merging {@link WindowFn}'s, windows may be merged to produce new windows that satisfy + * their own instance of the above regular expression. The only guarantee is that once a window + * produces a final pane, it will not be merged into any new windows. + * + *

The predictions above are made using the mechanism of watermarks. + * See {@link com.google.cloud.dataflow.sdk.util.TimerInternals} for more information + * about watermarks. + * + *

We can state some properties of {@code LATE} and {@code ON_TIME} panes, but first need some + * definitions: + *

    + *
  1. We'll call a pipeline 'simple' if it does not use + * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#outputWithTimestamp} in + * any {@code DoFn}, and it uses the same + * {@link com.google.cloud.dataflow.sdk.transforms.windowing.Window.Bound#withAllowedLateness} + * argument value on all windows (or uses the default of {@link org.joda.time.Duration#ZERO}). + *
  2. We'll call an element 'locally late', from the point of view of a computation on a + * worker, if the element's timestamp is before the input watermark for that computation + * on that worker. The element is otherwise 'locally on-time'. + *
  3. We'll say 'the pane's timestamp' to mean the timestamp of the element produced to + * represent the pane's contents. + *
+ * + *

Then in simple pipelines: + *

    + *
  1. (Soundness) An {@code ON_TIME} pane can never cause a later computation to generate a + * {@code LATE} pane. (If it did, it would imply a later computation's input watermark progressed + * ahead of an earlier stage's output watermark, which by design is not possible.) + *
  2. (Liveness) An {@code ON_TIME} pane is emitted as soon as possible after the input + * watermark passes the end of the pane's window. + *
  3. (Consistency) A pane with only locally on-time elements will always be {@code ON_TIME}. + * And a {@code LATE} pane cannot contain locally on-time elements. + *
+ * + * However, note that: + *
    + *
  1. An {@code ON_TIME} pane may contain locally late elements. It may even contain only + * locally late elements. Provided a locally late element finds its way into an {@code ON_TIME} + * pane its lateness becomes unobservable. + *
  2. A {@code LATE} pane does not necessarily cause any following computation panes to be + * marked as {@code LATE}. + *
*/ public enum Timing { - /** Pane was fired before the watermark passed the end of the window. */ + /** + * Pane was fired before the input watermark had progressed after the end of the window. + */ EARLY, - /** First pane fired after the watermark passed the end of the window. */ + /** + * Pane was fired by a {@link AfterWatermark#pastEndOfWindow} trigger because the input + * watermark progressed after the end of the window. However the output watermark has not + * yet progressed after the end of the window. Thus it is still possible to assign a timestamp + * to the element representing this pane which cannot be considered locally late by any + * following computation. + */ ON_TIME, - /** Panes fired after the {@code ON_TIME} firing. */ + /** + * Pane was fired after the output watermark had progressed past the end of the window. + */ LATE, /** - * This element was not produced in a triggered pane and its relation to the watermark is - * unknown. + * This element was not produced in a triggered pane and its relation to input and + * output watermarks is unknown. */ UNKNOWN; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchTimerInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchTimerInternals.java index d70900d3877ce..29492c3892335 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchTimerInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/BatchTimerInternals.java @@ -18,6 +18,7 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.common.base.MoreObjects; +import com.google.common.base.Preconditions; import org.joda.time.Instant; @@ -25,11 +26,12 @@ import java.util.PriorityQueue; import java.util.Set; +import javax.annotation.Nullable; + /** * TimerInternals that uses priority queues to manage the timers that are ready to fire. */ public class BatchTimerInternals implements TimerInternals { - /** Set of timers that are scheduled used for deduplicating timers. */ private Set existingTimers = new HashSet<>(); @@ -37,7 +39,7 @@ public class BatchTimerInternals implements TimerInternals { private PriorityQueue watermarkTimers = new PriorityQueue<>(11); private PriorityQueue processingTimers = new PriorityQueue<>(11); - private Instant watermarkTime; + private Instant inputWatermarkTime; private Instant processingTime; private PriorityQueue queue(TimeDomain domain) { @@ -46,7 +48,7 @@ private PriorityQueue queue(TimeDomain domain) { public BatchTimerInternals(Instant processingTime) { this.processingTime = processingTime; - this.watermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE; + this.inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE; } @Override @@ -68,8 +70,15 @@ public Instant currentProcessingTime() { } @Override - public Instant currentWatermarkTime() { - return watermarkTime; + public Instant currentInputWatermarkTime() { + return inputWatermarkTime; + } + + @Override + @Nullable + public Instant currentOutputWatermarkTime() { + // The output watermark is always undefined in batch mode. + return null; } @Override @@ -80,13 +89,18 @@ public String toString() { .toString(); } - public void advanceWatermark(ReduceFnRunner runner, Instant newWatermark) { - this.watermarkTime = newWatermark; - advance(runner, newWatermark, TimeDomain.EVENT_TIME); + public void advanceInputWatermark(ReduceFnRunner runner, Instant newInputWatermark) { + Preconditions.checkState(!newInputWatermark.isBefore(inputWatermarkTime), + "Cannot move input watermark time backwards from %s to %s", inputWatermarkTime, + newInputWatermark); + inputWatermarkTime = newInputWatermark; + advance(runner, newInputWatermark, TimeDomain.EVENT_TIME); } public void advanceProcessingTime(ReduceFnRunner runner, Instant newProcessingTime) { - this.processingTime = newProcessingTime; + Preconditions.checkState(!newProcessingTime.isBefore(processingTime), + "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime); + processingTime = newProcessingTime; advance(runner, newProcessingTime, TimeDomain.PROCESSING_TIME); } @@ -96,8 +110,8 @@ private void advance(ReduceFnRunner runner, Instant newTime, TimeDom do { TimerData timer = timers.peek(); - // Timers fire if the new time is >= the timer - shouldFire = timer != null && !newTime.isBefore(timer.getTimestamp()); + // Timers fire if the new time is ahead of the timer + shouldFire = timer != null && newTime.isAfter(timer.getTimestamp()); if (shouldFire) { // Remove before firing, so that if the trigger adds another identical // timer we don't remove it. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java index 4afd7cb4483db..9da4b4deb3998 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java @@ -71,19 +71,22 @@ public void processElement( // Process the chunk of elements. runner.processElements(chunk); - // Then, since elements are sorted by their timestamp, advance the watermark to the first - // element, and fire any timers that may have been scheduled. - timerInternals.advanceWatermark(runner, chunk.iterator().next().getTimestamp()); + // Then, since elements are sorted by their timestamp, advance the input watermark + // to the first element, and fire any timers that may have been scheduled. + timerInternals.advanceInputWatermark(runner, chunk.iterator().next().getTimestamp()); // Fire any processing timers that need to fire timerInternals.advanceProcessingTime(runner, Instant.now()); + + // Leave the output watermark undefined. Since there's no late data in batch mode + // there's really no need to track it as we do for streaming. } - // Finish any pending windows by advancing the watermark to infinity. - timerInternals.advanceWatermark(runner, new Instant(Long.MAX_VALUE)); + // Finish any pending windows by advancing the input watermark to infinity. + timerInternals.advanceInputWatermark(runner, BoundedWindow.TIMESTAMP_MAX_VALUE); // Finally, advance the processing time to infinity to fire any timers. - timerInternals.advanceProcessingTime(runner, new Instant(Long.MAX_VALUE)); + timerInternals.advanceProcessingTime(runner, BoundedWindow.TIMESTAMP_MAX_VALUE); runner.persist(); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NonEmptyPanes.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NonEmptyPanes.java index b1f13c13b8777..c4f39a9b23813 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NonEmptyPanes.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NonEmptyPanes.java @@ -53,7 +53,7 @@ public static NonEmptyPanes create( public abstract void clearPane(ReduceFn.Context context); /** - * Return true if the current pane for the window in {@code context} is non-empty. + * Return true if the current pane for the window in {@code context} is empty. */ public abstract StateContents isEmpty(ReduceFn.Context context); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PaneInfoTracker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PaneInfoTracker.java index 3b7858c1e71bc..05360f79d49e3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PaneInfoTracker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PaneInfoTracker.java @@ -15,6 +15,7 @@ */ package com.google.cloud.dataflow.sdk.util; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo.Timing; @@ -24,40 +25,52 @@ import com.google.cloud.dataflow.sdk.util.state.StateTags; import com.google.cloud.dataflow.sdk.util.state.ValueState; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import org.joda.time.Instant; /** - * Encapsulates the logic for tracking the current {@link PaneInfo} and producing new PaneInfo for - * a trigger firing. + * Determine the timing and other properties of a new pane for a given computation, key and window. + * Incorporates any previous pane, whether the pane has been produced because an + * on-time {@link AfterWatermark} trigger firing, and the relation between the element's timestamp + * and the current output watermark. */ public class PaneInfoTracker { - private TimerInternals timerInternals; public PaneInfoTracker(TimerInternals timerInternals) { this.timerInternals = timerInternals; } - @VisibleForTesting static final StateTag> PANE_INFO_TAG = + @VisibleForTesting + static final StateTag> PANE_INFO_TAG = StateTags.makeSystemTagInternal(StateTags.value("pane", PaneInfoCoder.INSTANCE)); public void clear(StateContext state) { state.access(PANE_INFO_TAG).clear(); } - + /** + * Return a (future for) the pane info appropriate for {@code context}. The pane info + * includes the timing for the pane, who's calculation is quite subtle. + * + * @param isWatermarkTrigger should be {@code true} only if the pane is being emitted + * because a {@link AfterWatermark#pastEndOfWindow} trigger has fired. + * @param isFinal should be {@code true} only if the triggering machinery can guarantee + * no further firings for the + */ public StateContents getNextPaneInfo(ReduceFn.Context context, - final boolean isForWatermarkTrigger, final boolean isFinal) { + final boolean isWatermarkTrigger, final boolean isFinal) { + final Object key = context.key(); final StateContents previousPaneFuture = context.state().access(PaneInfoTracker.PANE_INFO_TAG).get(); - final Instant endOfWindow = context.window().maxTimestamp(); + final Instant windowMaxTimestamp = context.window().maxTimestamp(); return new StateContents() { @Override public PaneInfo read() { PaneInfo previousPane = previousPaneFuture.read(); - return describePane(endOfWindow, previousPane, isForWatermarkTrigger, isFinal); + return describePane(key, windowMaxTimestamp, previousPane, isWatermarkTrigger, isFinal); } }; } @@ -66,21 +79,65 @@ public void storeCurrentPaneInfo(ReduceFn.Context context, PaneInfo context.state().access(PANE_INFO_TAG).set(currentPane); } - private PaneInfo describePane(Instant endOfWindow, PaneInfo prevPane, - boolean isForWatermarkTrigger, boolean isFinal) { - boolean isSpeculative = endOfWindow.isAfter(timerInternals.currentWatermarkTime()); - boolean isFirst = (prevPane == null); + private PaneInfo describePane(Object key, Instant windowMaxTimestamp, PaneInfo previousPane, + boolean isWatermarkTrigger, boolean isFinal) { + boolean isFirst = previousPane == null; + Timing previousTiming = isFirst ? null : previousPane.getTiming(); + long index = isFirst ? 0 : previousPane.getIndex() + 1; + long nonSpeculativeIndex = isFirst ? 0 : previousPane.getNonSpeculativeIndex() + 1; + Instant outputWM = timerInternals.currentOutputWatermarkTime(); + Instant inputWM = timerInternals.currentInputWatermarkTime(); + + // True if it is not possible to assign the element representing this pane a timestamp + // which will make an ON_TIME pane for any following computation. + // Ie true if the element's latest possible timestamp is before the current output watermark. + boolean isLateForOutput = outputWM != null && windowMaxTimestamp.isBefore(outputWM); + + // True if all emitted panes (if any) were EARLY panes. + // Once the ON_TIME pane has fired, all following panes must be considered LATE even + // if the output watermark is behind the end of the window. + boolean onlyEarlyPanesSoFar = previousTiming == null || previousTiming == Timing.EARLY; - long index = isFirst ? 0 : prevPane.getIndex() + 1; - long nonSpeculativeIndex; Timing timing; - if (isSpeculative) { + if (isLateForOutput || !onlyEarlyPanesSoFar) { + // The output watermark has already passed the end of this window, or we have already + // emitted a non-EARLY pane. Irrespective of how this pane was triggered we must + // consider this pane LATE. + timing = Timing.LATE; + } else if (isWatermarkTrigger) { + // This is the unique ON_TIME firing for the window. + timing = Timing.ON_TIME; + } else { + // All other cases are EARLY. timing = Timing.EARLY; nonSpeculativeIndex = -1; - } else { - boolean firstNonSpeculative = prevPane == null || prevPane.getTiming() == Timing.EARLY; - timing = (isForWatermarkTrigger && firstNonSpeculative) ? Timing.ON_TIME : Timing.LATE; - nonSpeculativeIndex = firstNonSpeculative ? 0 : prevPane.getNonSpeculativeIndex() + 1; + } + + WindowTracing.debug( + "describePane: {} pane (prev was {}) for key:{}; windowMaxTimestamp:{}; " + + "inputWatermark:{}; outputWatermark:{}; isWatermarkTrigger:{}; isLateForOutput:{}", + timing, previousTiming, key, windowMaxTimestamp, inputWM, outputWM, isWatermarkTrigger, + isLateForOutput); + + if (previousPane != null) { + // Timing transitions should follow EARLY* ON_TIME? LATE* + switch (previousTiming) { + case EARLY: + Preconditions.checkState( + timing == Timing.EARLY || timing == Timing.ON_TIME || timing == Timing.LATE, + "EARLY cannot transition to %s", timing); + break; + case ON_TIME: + Preconditions.checkState( + timing == Timing.LATE, "ON_TIME cannot transition to %s", timing); + break; + case LATE: + Preconditions.checkState(timing == Timing.LATE, "LATE cannot transtion to %s", timing); + break; + case UNKNOWN: + break; + } + Preconditions.checkState(!previousPane.isLast(), "Last pane was not last after all."); } return PaneInfo.createPane(isFirst, isFinal, timing, index, nonSpeculativeIndex); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java index d539c84ddc066..742806db48110 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java @@ -33,10 +33,12 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.FluentIterable; import com.google.common.collect.Maps; +import org.joda.time.Duration; import org.joda.time.Instant; import java.util.Collection; @@ -46,8 +48,6 @@ import java.util.Map; import java.util.Set; -import javax.annotation.Nullable; - /** * Manages the execution of a {@link ReduceFn} after a {@link GroupByKeyOnly} has partitioned the * {@link PCollection} by key. @@ -73,7 +73,6 @@ * @param The type of windows this operates on. */ public class ReduceFnRunner { - public static final String DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER = "DroppedDueToClosedWindow"; public static final String DROPPED_DUE_TO_LATENESS_COUNTER = "DroppedDueToLateness"; @@ -121,7 +120,7 @@ public ReduceFnRunner( key, reduceFn, this.windowingStrategy, this.windowingInternals.stateInternals(), this.activeWindows, timerInternals); - this.watermarkHold = new WatermarkHold<>(windowingStrategy); + this.watermarkHold = new WatermarkHold<>(timerInternals, windowingStrategy); this.triggerRunner = new TriggerRunner<>( windowingStrategy.getTrigger(), new TriggerContextFactory<>(windowingStrategy, this.windowingInternals.stateInternals(), @@ -156,7 +155,8 @@ public void processElements(Iterable> values) { // Trigger output from any window that was triggered by merging or processing elements. for (Map.Entry result : results.entrySet()) { - handleTriggerResult(contextFactory.base(result.getKey()), false, result.getValue()); + handleTriggerResult( + contextFactory.base(result.getKey()), false/*isEndOfWindow*/, result.getValue()); } } @@ -181,7 +181,7 @@ private Function premergeForValues( // For any new windows that survived merging, make sure we've scheduled cleanup for (W window : newWindows) { if (activeWindows.contains(window)) { - scheduleCleanup(contextFactory.base(window)); + scheduleEndOfWindowOrGarbageCollectionTimer(contextFactory.base(window)); } } @@ -226,7 +226,7 @@ public void onMerge(Collection mergedWindows, W resultWindow, boolean isResul } else { // If there were no windows, then merging didn't rearrange the cleanup timers. Make // sure that we have one properly scheduled - scheduleCleanup(contextFactory.base(resultWindow)); + scheduleEndOfWindowOrGarbageCollectionTimer(contextFactory.base(resultWindow)); } for (W mergedWindow : mergedWindows) { @@ -235,7 +235,7 @@ public void onMerge(Collection mergedWindows, W resultWindow, boolean isResul // If the window wasn't in the persisted original set, then we scheduled cleanup above // but didn't pass it to merge to have the cleanup canceled. Do so here if (!originalWindows.contains(mergedWindow)) { - cancelCleanup(contextFactory.base(mergedWindow)); + cancelEndOfWindowAndGarbageCollectionTimers(contextFactory.base(mergedWindow)); } } } @@ -247,6 +247,13 @@ public void onMerge(Collection mergedWindows, W resultWindow, boolean isResul return sourceWindowsToResultWindows; } + /** Is {@code value} late w.r.t. the garbage collection watermark? */ + private boolean canDropDueToLateness(WindowedValue value) { + Instant inputWM = timerInternals.currentInputWatermarkTime(); + return inputWM != null + && value.getTimestamp().isBefore(inputWM.minus(windowingStrategy.getAllowedLateness())); + } + /** * Add the initial windows from each of the values to the active window set. Returns the set of * new windows. @@ -254,7 +261,9 @@ public void onMerge(Collection mergedWindows, W resultWindow, boolean isResul private Set addToActiveWindows(Iterable> values) { Set newWindows = new HashSet<>(); for (WindowedValue value : values) { - if (getLateness(value.getTimestamp()).isPastAllowedLateness) { + if (canDropDueToLateness(value)) { + // This value will be dropped (and reported in a counter) by processElement. + // Hence it won't contribute to any new window. continue; } @@ -280,15 +289,21 @@ private Set addToActiveWindows(Iterable> values) { * for the given result. * @param value the value being processed */ - private void processElement(Function windowMapping, Map results, - WindowedValue value) { - Lateness lateness = getLateness(value.getTimestamp()); - if (lateness.isPastAllowedLateness) { + private void processElement( + Function windowMapping, Map results, WindowedValue value) { + if (canDropDueToLateness(value)) { // Drop the element in all assigned windows if it is past the allowed lateness limit. droppedDueToLateness.addValue((long) value.getWindows().size()); + WindowTracing.debug( + "processElement: Dropping element at {} for key:{} since too far " + + "behind inputWatermark:{}; outputWatermark:{}", + value.getTimestamp(), key, timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); return; } + // Only consider representative windows from among all windows in equivalence classes + // induced by window merging. @SuppressWarnings("unchecked") Iterable windows = FluentIterable.from((Collection) value.getWindows()).transform(windowMapping); @@ -315,16 +330,17 @@ private void processElement(Function windowMapping, Map nonEmptyPanes.recordContent(context); - // Make sure we've scheduled the cleanup timer for this window, if the premerge didn't already - // do that. + // Make sure we've scheduled the end-of-window or garbage collection timer for this window + // However if we have pre-merged then they will already have been scheduled. if (windowingStrategy.getWindowFn().isNonMerging()) { - // Since non-merging window functions don't track the active window set, we always schedule - // cleanup. - scheduleCleanup(context); + scheduleEndOfWindowOrGarbageCollectionTimer(context); } - // Update the watermark hold since the value will be part of the next pane. - watermarkHold.addHold(context, lateness.isLate); + // Hold back progress of the output watermark until we have processed the pane this + // element will be included within. Also add a hold at the end-of-window or garbage + // collection time to allow empty panes to contribute elements which won't be dropped + // due to lateness. + watermarkHold.addHolds(context); // Execute the reduceFn, which will buffer the value as appropriate try { @@ -348,14 +364,6 @@ private void processElement(Function windowMapping, Map } } - private void holdForEmptyPanes(ReduceFn.Context context) { - if (timerInternals.currentWatermarkTime().isAfter(context.window().maxTimestamp())) { - watermarkHold.holdForFinal(context); - } else { - watermarkHold.holdForOnTime(context); - } - } - /** * Make sure that all the state built up in this runner has been persisted. */ @@ -398,7 +406,7 @@ public TriggerResult onMerge( if (!mergedWindow.equals(resultWindow)) { try { ReduceFn.Context mergedContext = contextFactory.base(mergedWindow); - cancelCleanup(mergedContext); + cancelEndOfWindowAndGarbageCollectionTimers(mergedContext); triggerRunner.clearEverything(mergedContext); paneInfo.clear(mergedContext.state()); } catch (Exception e) { @@ -411,87 +419,101 @@ public TriggerResult onMerge( // Schedule cleanup if the window is new. Do this after cleaning up the old state in case one // of them had a timer at the same point. if (isResultWindowNew) { - scheduleCleanup(resultContext); + scheduleEndOfWindowOrGarbageCollectionTimer(resultContext); } return triggerResult; } /** - * Called when a timer fires. + * Called when an end-of-window, garbage collection, or trigger-specific timer fires. */ public void onTimer(TimerData timer) { - if (!(timer.getNamespace() instanceof WindowNamespace)) { - throw new IllegalArgumentException( - "Expected WindowNamespace, but was " + timer.getNamespace()); - } - + // Which window is the timer for? + Preconditions.checkArgument(timer.getNamespace() instanceof WindowNamespace, + "Expected timer to be in WindowNamespace, but was in %s", timer.getNamespace()); @SuppressWarnings("unchecked") WindowNamespace windowNamespace = (WindowNamespace) timer.getNamespace(); W window = windowNamespace.getWindow(); - if (!activeWindows.contains(window) && windowingStrategy.getWindowFn().isNonMerging()) { - throw new IllegalStateException( - "Internal Error: Received timer " + timer + " for inactive window: " + window); - } + // If the window is subject to merging then all timers should have been cleared upon merge. + Preconditions.checkState( + !windowingStrategy.getWindowFn().isNonMerging() || activeWindows.contains(window), + "Received timer %s for inactive window %s", timer, window); ReduceFn.Context context = contextFactory.base(window); - // If this timer firing is at the watermark, then it may cause a trigger firing of an - // AfterWatermark trigger. - boolean isAtWatermark = TimeDomain.EVENT_TIME == timer.getDomain() - && !timer.getTimestamp().isBefore(window.maxTimestamp()); - - if (shouldCleanup(timer, window)) { - // We're going to cleanup the window. We want to treat any potential output from this as - // the at-watermark firing if the current time is the at-watermark firing and there was a - // trigger waiting for it. - if (isAtWatermark) { - TriggerResult timerResult = runTriggersForTimer(context, timer); - isAtWatermark = (timerResult != null && timerResult.isFire()); + // If this is an end-of-window timer then we should test if an AfterWatermark trigger + // will fire. + // It's fine if the window trigger has such trigger, this flag is only used to decide + // if an emitted pane should be classified as ON_TIME. + boolean isEndOfWindowTimer = + TimeDomain.EVENT_TIME == timer.getDomain() + && timer.getTimestamp().equals(window.maxTimestamp()); + + // If this is a garbage collection timer then we should trigger and garbage collect the window. + Instant cleanupTime = window.maxTimestamp().plus(windowingStrategy.getAllowedLateness()); + boolean isGarbageCollection = + TimeDomain.EVENT_TIME == timer.getDomain() && timer.getTimestamp().equals(cleanupTime); + + if (isGarbageCollection) { + WindowTracing.debug( + "onTimer: Cleaning up for key:{}; window:{} at {} with " + + "inputWatermark:{}; outputWatermark:{}", + key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + + if (activeWindows.contains(window) && !triggerRunner.isClosed(context.state())) { + // We need to call onTrigger to emit the final pane if required. + // The final pane *may* be ON_TIME if: + // - AllowedLateness = 0 (ie the timer is at end-of-window), and; + // - The trigger fires on the end-of-window timer. + boolean isWatermarkTrigger = + isEndOfWindowTimer && runTriggersForTimer(context, timer).isFire(); + onTrigger(context, isWatermarkTrigger, true/*isFinish*/); } - // Do the actual cleanup + // Clear all the state for this window since we'll never see elements for it again. try { - doCleanup(context, isAtWatermark); + clearAllState(context); } catch (Exception e) { Throwables.propagateIfInstanceOf(e, UserCodeException.class); throw new RuntimeException( "Exception while garbage collecting window " + windowNamespace.getWindow(), e); } } else { + WindowTracing.debug( + "onTimer: Triggering for key:{}; window:{} at {} with " + + "inputWatermark:{}; outputWatermark:{}", + key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + boolean isFinish = false; if (activeWindows.contains(window) && !triggerRunner.isClosed(context.state())) { - handleTriggerResult(context, isAtWatermark, runTriggersForTimer(context, timer)); + TriggerResult result = runTriggersForTimer(context, timer); + handleTriggerResult(context, isEndOfWindowTimer, result); + isFinish = result.isFinish(); } - if (TimeDomain.EVENT_TIME == timer.getDomain() - // If we processed an on-time firing, we should schedule the GC timer. - && timer.getTimestamp().isEqual(window.maxTimestamp())) { - scheduleCleanup(context); + if (isEndOfWindowTimer && !isFinish) { + // Since we are processing an on-time firing we should schedule the garbage collection + // timer. (If getAllowedLateness is zero then the timer event will be considered a + // cleanup event and handled by the above). + Preconditions.checkState( + windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO), + "Unexpected zero getAllowedLateness"); + WindowTracing.debug( + "onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with " + + "inputWatermark:{}; outputWatermark:{}", + key, context.window(), cleanupTime, timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + context.timers().setTimer(cleanupTime, TimeDomain.EVENT_TIME); } } } - /** - * Return true if either the timer looks like a cleanup timer or the current watermark is so far - * gone that we should cleanup the window. - */ - private boolean shouldCleanup(TimerData timer, W window) { - return TimeDomain.EVENT_TIME == timer.getDomain() - && (isCleanupTime(window, timer.getTimestamp()) - || isCleanupTime(window, timerInternals.currentWatermarkTime())); - } - - @Nullable private TriggerResult runTriggersForTimer( ReduceFn.Context context, TimerData timer) { - triggerRunner.prefetchForTimer(context.state()); - // Skip timers for windows that were closed by triggers, but haven't expired yet. - if (triggerRunner.isClosed(context.state())) { - return null; - } - try { return triggerRunner.onTimer(context, timer); } catch (Exception e) { @@ -500,16 +522,17 @@ private TriggerResult runTriggersForTimer( } } - /** Called when the cleanup timer has fired for the given window. */ - private void doCleanup( - ReduceFn.Context context, boolean maybeAtWatermark) throws Exception { - // If the window isn't closed, or if we should always fire a final pane, then trigger output - if (!triggerRunner.isClosed(context.state()) - || windowingStrategy.getClosingBehavior() == ClosingBehavior.FIRE_ALWAYS) { - onTrigger(context, maybeAtWatermark, true /* isFinal */); - } - - // Cleanup the associated state. + /** + * Clear all the state associated with {@code context}'s window. + * Should only be invoked if we know all future elements for this window will be considered + * beyond allowed lateness. + * This is a superset of the clearing done by {@link #handleTriggerResult} below since: + *
    + *
  1. We can clear the trigger state tombstone since we'll never need to ask about it again. + *
  2. We can clear any remaining garbage collection hold. + *
+ */ + private void clearAllState(ReduceFn.Context context) throws Exception { nonEmptyPanes.clearPane(context); try { reduceFn.clearState(context); @@ -518,19 +541,39 @@ private void doCleanup( } triggerRunner.clearEverything(context); paneInfo.clear(context.state()); - watermarkHold.releaseOnTime(context); + watermarkHold.clear(context); + } + + /** Should the reduce function state be cleared? */ + private boolean shouldDiscardAfterFiring(TriggerResult result) { + if (result.isFinish()) { + // This is the last firing for trigger. + return true; + } + if (windowingStrategy.getMode() == AccumulationMode.DISCARDING_FIRED_PANES) { + // Nothing should be accumulated between panes. + return true; + } + return false; } - private void handleTriggerResult( - ReduceFn.Context context, - boolean maybeAtWatermark, TriggerResult result) { - // Unless the trigger is firing, there is nothing to do. + /** + * Possibly emit a pane if a trigger fired or timers require it, and cleanup state. + */ + private void handleTriggerResult(ReduceFn.Context context, + boolean isEndOfWindow, TriggerResult result) { if (!result.isFire()) { + // Ignore unless trigger fired. return; } + // If the trigger fired due to an end-of-window timer, treat it as an AfterWatermark trigger. + boolean isWatermarkTrigger = isEndOfWindow; + // Run onTrigger to produce the actual pane contents. - onTrigger(context, maybeAtWatermark, result.isFinish()); + // As a side effect it will clear all element holds, but not necessarily any + // end-of-window or garbage collection holds. + onTrigger(context, isWatermarkTrigger, result.isFinish()); // Now that we've triggered, the pane is empty. nonEmptyPanes.clearPane(context); @@ -550,10 +593,10 @@ private void handleTriggerResult( if (result.isFinish()) { // If we're finishing, clear up the trigger tree as well. + // However, we'll leave behind a tombstone so we know the trigger is finished. try { triggerRunner.clearState(context); paneInfo.clear(context.state()); - watermarkHold.releaseFinal(context); } catch (Exception e) { Throwables.propagateIfPossible(e); throw new RuntimeException("Exception while clearing trigger state", e); @@ -561,129 +604,108 @@ private void handleTriggerResult( } } - public static StateContents stateContentsOf(final T value) { - return new StateContents() { - @Override - public T read() { - return value; - } - }; + /** + * Do we need to emit a pane? + */ + private boolean needToEmit( + boolean isEmpty, boolean isWatermarkTrigger, boolean isFinish, PaneInfo.Timing timing) { + if (!isEmpty) { + // The pane has elements. + return true; + } + if (isWatermarkTrigger && timing == Timing.ON_TIME) { + // This is the unique ON_TIME pane, triggered by an AfterWatermark. + return true; + } + if (isFinish && windowingStrategy.getClosingBehavior() == ClosingBehavior.FIRE_ALWAYS) { + // This is known to be the final pane, and the user has requested it even when empty. + return true; + } + return false; } /** * Run the {@link ReduceFn#onTrigger} method and produce any necessary output. * * @param context the context for the pane to fire - * @param isAtWatermark true if this triggering is for an AfterWatermark trigger - * @param isFinal true if this will be the last triggering processed + * @param isWatermarkTrigger true if this triggering is for an AfterWatermark trigger + * @param isFinish true if this will be the last triggering processed */ private void onTrigger(final ReduceFn.Context context, - boolean isAtWatermark, boolean isFinal) { - StateContents outputTimestampFuture = watermarkHold.extractAndRelease(context); + boolean isWatermarkTrigger, boolean isFinish) { + // Collect state. + StateContents outputTimestampFuture = + watermarkHold.extractAndRelease(context, isFinish); StateContents paneFuture = - paneInfo.getNextPaneInfo(context, isAtWatermark, isFinal); + paneInfo.getNextPaneInfo(context, isWatermarkTrigger, isFinish); StateContents isEmptyFuture = nonEmptyPanes.isEmpty(context); reduceFn.prefetchOnTrigger(context.state()); + // Calculate the pane info. final PaneInfo pane = paneFuture.read(); + // Extract the window hold, and as a side effect clear it. final Instant outputTimestamp = outputTimestampFuture.read(); - boolean shouldOutput = - // If the pane is not empty - !isEmptyFuture.read() - // or this is the final pane, and the user has asked for it even if its empty - || (isFinal && windowingStrategy.getClosingBehavior() == ClosingBehavior.FIRE_ALWAYS) - // or this is the on-time firing, and the user explicitly requested it. - || (isAtWatermark && pane.getTiming() == Timing.ON_TIME); - - // We've consumed the empty pane hold by reading it, so reinstate that, if necessary. - if (!isFinal) { - holdForEmptyPanes(context); - } - - // If there is nothing to output, we're done. - if (!shouldOutput) { - return; - } - - // Run reduceFn.onTrigger method. - final List windows = Collections.singletonList(context.window()); - ReduceFn.OnTriggerContext triggerContext = contextFactory.forTrigger( - context.window(), paneFuture, new OnTriggerCallbacks() { - @Override - public void output(OutputT toOutput) { - // We're going to output panes, so commit the (now used) PaneInfo. - paneInfo.storeCurrentPaneInfo(context, pane); - - // Output the actual value. - windowingInternals.outputWindowedValue( - KV.of(key, toOutput), outputTimestamp, windows, pane); - } - }); + // Only emit a pane if it has data or empty panes are observable. + if (needToEmit(isEmptyFuture.read(), isWatermarkTrigger, isFinish, pane.getTiming())) { + // Run reduceFn.onTrigger method. + final List windows = Collections.singletonList(context.window()); + ReduceFn.OnTriggerContext triggerContext = contextFactory.forTrigger( + context.window(), paneFuture, new OnTriggerCallbacks() { + @Override + public void output(OutputT toOutput) { + // We're going to output panes, so commit the (now used) PaneInfo. + // TODO: Unnecessary if isFinal? + paneInfo.storeCurrentPaneInfo(context, pane); + + // Output the actual value. + windowingInternals.outputWindowedValue( + KV.of(key, toOutput), outputTimestamp, windows, pane); + } + }); - try { - reduceFn.onTrigger(triggerContext); - } catch (Exception e) { - throw wrapMaybeUserException(e); + try { + reduceFn.onTrigger(triggerContext); + } catch (Exception e) { + throw wrapMaybeUserException(e); + } } } - private Instant cleanupTime(W window) { - return window.maxTimestamp().plus(windowingStrategy.getAllowedLateness()); - } - - /** Return true if {@code timestamp} is past the cleanup time for {@code window}. */ - private boolean isCleanupTime(W window, Instant timestamp) { - return !timestamp.isBefore(cleanupTime(window)); - } - - private void scheduleCleanup(ReduceFn.Context context) { - if (timerInternals.currentWatermarkTime().isAfter(context.window().maxTimestamp())) { - context.timers().setTimer(cleanupTime(context.window()), TimeDomain.EVENT_TIME); - } else { - context.timers().setTimer(context.window().maxTimestamp(), TimeDomain.EVENT_TIME); + private void scheduleEndOfWindowOrGarbageCollectionTimer(ReduceFn.Context context) { + Instant fireTime = context.window().maxTimestamp(); + String which = "end-of-window"; + Instant inputWM = timerInternals.currentInputWatermarkTime(); + if (inputWM != null && fireTime.isBefore(inputWM)) { + fireTime = fireTime.plus(windowingStrategy.getAllowedLateness()); + which = "garbage collection"; + Preconditions.checkState(!fireTime.isBefore(inputWM), + "Asking to set a timer at %s behind input watermark %s", fireTime, inputWM); } + WindowTracing.trace( + "scheduleTimer: Scheduling {} timer at {} for " + + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}", + which, fireTime, key, context.window(), inputWM, + timerInternals.currentOutputWatermarkTime()); + context.timers().setTimer(fireTime, TimeDomain.EVENT_TIME); } - private void cancelCleanup(ReduceFn.Context context) { - context.timers().deleteTimer(cleanupTime(context.window()), TimeDomain.EVENT_TIME); - context.timers().deleteTimer(context.window().maxTimestamp(), TimeDomain.EVENT_TIME); - } - - private boolean shouldDiscardAfterFiring(TriggerResult result) { - return result.isFinish() - || (result.isFire() - && AccumulationMode.DISCARDING_FIRED_PANES == windowingStrategy.getMode()); - } - - ////////////////////////////////////////////////////////////////////////////////////////////////// - - private enum Lateness { - NOT_LATE(false, false), - LATE(true, false), - PAST_ALLOWED_LATENESS(true, true); - - private final boolean isLate; - private final boolean isPastAllowedLateness; - - private Lateness(boolean isLate, boolean isPastAllowedLateness) { - this.isLate = isLate; - this.isPastAllowedLateness = isPastAllowedLateness; + private void cancelEndOfWindowAndGarbageCollectionTimers(ReduceFn.Context context) { + WindowTracing.debug( + "cancelTimer: Deleting timers for " + + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}", + key, context.window(), timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + Instant timer = context.window().maxTimestamp(); + context.timers().deleteTimer(timer, TimeDomain.EVENT_TIME); + if (windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO)) { + timer = timer.plus(windowingStrategy.getAllowedLateness()); + context.timers().deleteTimer(timer, TimeDomain.EVENT_TIME); } } - private Lateness getLateness(Instant timestamp) { - Instant latestAllowed = - timerInternals.currentWatermarkTime().minus(windowingStrategy.getAllowedLateness()); - if (timestamp.isBefore(latestAllowed)) { - return Lateness.PAST_ALLOWED_LATENESS; - } else if (timestamp.isBefore(timerInternals.currentWatermarkTime())) { - return Lateness.LATE; - } else { - return Lateness.NOT_LATE; - } - } + ////////////////////////////////////////////////////////////////////////////////////////////////// private RuntimeException wrapMaybeUserException(Throwable t) { if (reduceFn instanceof SystemReduceFn) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerInternals.java index 522fbf64afd8a..ff7576cd88831 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TimerInternals.java @@ -23,6 +23,8 @@ import java.util.Objects; +import javax.annotation.Nullable; + /** * Encapsulate interaction with time within the execution environment. * @@ -51,9 +53,68 @@ public interface TimerInternals { Instant currentProcessingTime(); /** - * Returns an estimate of the current timestamp in the {@link TimeDomain#EVENT_TIME} time domain. + * Return the current, local input watermark timestamp for this computation + * in the {@link TimeDomain#EVENT_TIME} time domain. Return {@code null} if unknown. + * + *

This value: + *

    + *
  1. Is monotonically increasing. + *
  2. May differ between workers due to network and other delays. + *
  3. Will never be ahead of the global input watermark for this computation. But it + * may be arbitrarily behind the global input watermark. + *
  4. Any element with a timestamp before the local input watermark can be considered + * 'locally late' and be subject to special processing or be dropped entirely. + *
+ * + *

Note that because the local input watermark can be behind the global input watermark, + * it is possible for an element to be considered locally on-time even though it is + * globally late. + */ + @Nullable + Instant currentInputWatermarkTime(); + + /** + * Return the current, local output watermark timestamp for this computation + * in the {@link TimeDomain#EVENT_TIME} time domain. Return {@code null} if unknown. + * + *

This value: + *

    + *
  1. Is monotonically increasing. + *
  2. Will never be ahead of {@link #currentInputWatermarkTime} as returned above. + *
  3. May differ between workers due to network and other delays. + *
  4. However will never be behind the global input watermark for any following computation. + *
+ * + *

In pictures: + *

+   *  |              |       |       |       |
+   *  |              |   D   |   C   |   B   |   A
+   *  |              |       |       |       |
+   * GIWN     <=    GOWM <= LOWM <= LIWM <= GIWM
+   * (next stage)
+   * -------------------------------------------------> event time
+   * 
+ * where + *
    + *
  • LOWM = local output water mark. + *
  • GOWM = global output water mark. + *
  • GIWM = global input water mark. + *
  • LIWM = local input water mark. + *
  • A = A globally on-time element. + *
  • B = A globally late, but locally on-time element. + *
  • C = A locally late element which may still contribute to the timestamp of a pane. + *
  • D = A locally late element which cannot contribute to the timestamp of a pane. + *
+ * + *

Note that if a computation emits an element which is not before the current output watermark + * then that element will always appear locally on-time in all following computations. However, + * it is possible for an element emitted before the current output watermark to appear locally + * on-time in a following computation. Thus we must be careful to never assume locally late data + * viewed on the output of a computation remains locally late on the input of a following + * computation. */ - Instant currentWatermarkTime(); + @Nullable + Instant currentOutputWatermarkTime(); /** * Data about a timer as represented within {@link TimerInternals}. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java index d480369b52e65..cee3eeb0b95ed 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java @@ -17,13 +17,14 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns; import com.google.cloud.dataflow.sdk.transforms.windowing.Window.ClosingBehavior; -import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.state.StateContents; import com.google.cloud.dataflow.sdk.util.state.StateTag; import com.google.cloud.dataflow.sdk.util.state.StateTags; import com.google.cloud.dataflow.sdk.util.state.WatermarkStateInternal; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import org.joda.time.Duration; import org.joda.time.Instant; @@ -31,114 +32,383 @@ import java.io.Serializable; /** - * Implements the logic needed to hold the output watermark back to emit - * values with specific timestamps. + * Implements the logic needed to hold the output watermark for a computation back + * until it has seen all the elements it needs based on the input watermark for the + * computation. + * + *

The backend ensures the output watermark can never progress beyond the + * input watermark for a computation. GroupAlsoByWindows computations may add a 'hold' + * to the output watermark in order to prevent it progressing beyond a time within a window. + * The hold will be 'cleared' when the associated pane is emitted. * * @param The kind of {@link BoundedWindow} the hold is for. */ public class WatermarkHold implements Serializable { - - /** Watermark hold used for the actual data-based hold. */ - @VisibleForTesting static final String DATA_HOLD_ID = "hold"; - @VisibleForTesting static StateTag watermarkHoldTagForOutputTimeFn( + /** + * Return tag for state containing the output watermark hold + * used for elements. + */ + public static StateTag watermarkHoldTagForOutputTimeFn( OutputTimeFn outputTimeFn) { - return StateTags.makeSystemTagInternal( - StateTags.watermarkStateInternal(DATA_HOLD_ID, outputTimeFn)); + return StateTags.makeSystemTagInternal(StateTags.watermarkStateInternal("hold", outputTimeFn)); } + /** + * Tag for state containing end-of-window and garbage collection output watermark holds. + * (We can't piggy-back on the data hold state since the outputTimeFn may be + * {@link OutputTimeFns#outputAtLatestInputTimestamp()}, in which case every pane will + * would take the end-of-window time as its element time. + */ + @VisibleForTesting + public static final StateTag EXTRA_HOLD_TAG = + StateTags.makeSystemTagInternal(StateTags.watermarkStateInternal( + "extra", OutputTimeFns.outputAtEarliestInputTimestamp())); + + private final TimerInternals timerInternals; private final WindowingStrategy windowingStrategy; - private final StateTag watermarkHoldTag; + private final StateTag elementHoldTag; - public WatermarkHold(WindowingStrategy windowingStrategy) { + public WatermarkHold(TimerInternals timerInternals, WindowingStrategy windowingStrategy) { + this.timerInternals = timerInternals; this.windowingStrategy = windowingStrategy; - this.watermarkHoldTag = watermarkHoldTagForOutputTimeFn(windowingStrategy.getOutputTimeFn()); + this.elementHoldTag = watermarkHoldTagForOutputTimeFn(windowingStrategy.getOutputTimeFn()); } /** - * Update the watermark hold to include the appropriate output timestamp for the value in - * {@code c}. + * Add a hold to prevent the output watermark progressing beyond the (possibly adjusted) timestamp + * of the element in {@code context}. We allow the actual hold time to be shifted later by + * {@link OutputTimeFn#assignOutputTime}, but no further than the end of the window. The hold will + * remain until cleared by {@link #extractAndRelease}. + * + *

In the following we'll write {@code E} to represent an element, {@code IWM} for + * the local input watermark, {@code OWM} for the local output watermark, and {@code GCWM} for + * the garbage collection watermark (which is at {@code IWM - getAllowedLateness}). Time + * progresses from left to right, and we write {@code [ ... ]} to denote a bounded window with + * implied lower bound. + * + *

Note that the GCWM will be the same as the IWM if {@code getAllowedLateness} + * is {@code ZERO}. + * + *

Here are the cases we need to handle. They are conceptually considered in the + * sequence written since if getAllowedLateness is ZERO the GCWM is the same as the IWM. + *

    + *
  1. (Normal) + *
    +   *          |
    +   *      [   | E        ]
    +   *          |
    +   *         IWM
    +   * 
    + * This is, hopefully, the common and happy case. The element is locally on-time and can + * definitely make it to an {@code ON_TIME} pane which we can still set an end-of-window timer + * for. We place an element hold at E which will become the {@code ON_TIME} pane's timestamp + * if it is the earliest such hold. (Thus the OWM will not proceed past E until the next pane + * fires). We also place an end-of-window and (if required) garbage collection hold in case + * this is the first element seen for the window. + * + *
  2. (Discard - no target window) + *
    +   *                       |                            |
    +   *      [     E        ] |                            |
    +   *                       |                            |
    +   *                     GCWM  <-getAllowedLateness->  IWM
    +   * 
    + * The element is very locally late. The window has been garbage collected, thus there + * is no target pane E could be assigned to. We discard E. + * + *
  3. (Discard - beyond allowed lateness) + *
    +   *               |                            |
    +   *      [     E  |     ]                      |
    +   *               |                            |
    +   *             GCWM  <-getAllowedLateness->  IWM
    +   * 
    + * The element is very locally late, and the window is very close to being garbage collected, at + * which point a final {@code LATE} pane could be emitted. We *could* attempt to capture E within + * that pane, however that requires checking against all possible windows which may contain E. + * We instead discard E. + * + *
  4. (Unobservably late) + *
    +   *          |    |
    +   *      [   | E  |     ]
    +   *          |    |
    +   *         OWM  IWM
    +   * 
    + * The element is locally late, however we can still treat this case as for 'Normal' above + * since the IWM has not yet passed the end of the window and the element is ahead of the + * OWM. In effect, we get to 'launder' the locally late element and consider it as locally + * on-time because no downstream computation can observe the difference. + * + *
  5. (Input Late) + *
    +   *          |            |
    +   *      [   | E        ] |
    +   *          |            |
    +   *         OWM          IWM
    +   * 
    + * The end-of-window timer may have already fired for this window, and thus an {@code ON_TIME} + * pane may have already been emitted. We can still place an element hold, which will be + * cleared when the next pane fires (which could be {@code ON_TIME} or {@code LATE}). We + * should not place an end-of-window hold since we cannot guarantee it will be cleared until + * the garbage collection timer fires. We can still place a garbage collection hold if required. + * + *
  6. (Possibly unobservably late - 1) + *
    +   *               |   |
    +   *      [     E  |   | ]
    +   *               |   |
    +   *              OWM IWM
    +   * 
    + * The element is too late to contribute to the output watermark hold, and thus won't + * contribute the any pane's timestamp. We don't know if a hold has been placed at or later + * than the OWM for this window. Thus we can't be sure E will make an {@code ON_TIME} pane, + * even though we know the end-of-window timer is yet to fire. We can still place an + * end-of-window hold, and a garbage collection hold if required. + * + *
  7. (Possibly unobservably late - 2) + *
    +   *               |       |
    +   *      [     E  |     ] |
    +   *               |       |
    +   *              OWM     IWM
    +   * 
    + * As for the previous case, however we don't even know if the end-of-window timer has already + * fired, or it is about to fire. We can place only the garbage collection hold, if required. * - *

    If the value was not late, then the input watermark must be less than the timestamp, and we - * can use {@link WindowFn#getOutputTime} to determine the appropriate output time. + *

  8. (Definitely late) + *
    +   *                       |   |
    +   *      [     E        ] |   |
    +   *                       |   |
    +   *                      OWM IWM
    +   * 
    + * The element is definitely too late to make an {@code ON_TIME} pane. We are too late to + * place an end-of-window hold. We can still place a garbage collection hold if required. * - *

    If the value was late, we pessimistically assume the worst and attempt to hold the watermark - * to {@link BoundedWindow#maxTimestamp()} plus {@link WindowingStrategy#getAllowedLateness()}. - * That allows us to output the result at {@link BoundedWindow#maxTimestamp()} without being - * dropped. + *

*/ - public void addHold(ReduceFn.ProcessValueContext c, boolean isLate) { - Instant outputTime = isLate - ? c.window().maxTimestamp().plus(windowingStrategy.getAllowedLateness()) - : windowingStrategy.getOutputTimeFn().assignOutputTime(c.timestamp(), c.window()); - c.state().access(watermarkHoldTag).add(outputTime); + public void addHolds(ReduceFn.ProcessValueContext context) { + if (!addElementHold(context)) { + addEndOfWindowOrGarbageCollectionHolds(context); + } + } + + /** + * Return {@code timestamp}, possibly shifted forward in time according to the window + * strategy's output time function. + */ + private Instant shift(Instant timestamp, W window) { + Instant shifted = windowingStrategy.getOutputTimeFn().assignOutputTime(timestamp, window); + if (shifted.isBefore(timestamp)) { + throw new IllegalStateException( + String.format("OutputTimeFn moved element from %s to earlier time %s for window %s", + timestamp, shifted, window)); + } + if (!timestamp.isAfter(window.maxTimestamp()) && shifted.isAfter(window.maxTimestamp())) { + throw new IllegalStateException( + String.format("OutputTimeFn moved element from %s to %s which is beyond end of window %s", + timestamp, shifted, window)); + } + + return shifted; + } + + /** + * Add an element hold if possible. Return true if was added, false if too late to add. + */ + private boolean addElementHold(ReduceFn.ProcessValueContext context) { + // Give the window function a chance to move the hold timestamp forward to encourage progress. + // (A later hold implies less impediment to the output watermark making progress, which in + // turn encourages end-of-window triggers to fire earlier in following computations.) + Instant elementHold = shift(context.timestamp(), context.window()); + + Instant outputWM = timerInternals.currentOutputWatermarkTime(); + Instant inputWM = timerInternals.currentInputWatermarkTime(); + + Instant garbageWM = + inputWM == null ? null : inputWM.minus(windowingStrategy.getAllowedLateness()); + Preconditions.checkState(garbageWM == null || !elementHold.isBefore(garbageWM), + "Shifted timestamp %s cannot be beyond garbage collection watermark %s", elementHold, + garbageWM); + + // Only add the hold if we can be sure the backend will be able to respect it. + boolean tooLate; + if (outputWM != null && elementHold.isBefore(outputWM)) { + tooLate = true; + } else { + tooLate = false; + context.state().access(elementHoldTag).add(elementHold); + } + WindowTracing.trace( + "WatermarkHold.addHolds: element hold at {} is {} for " + + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}", + elementHold, tooLate ? "too late" : "on-time", context.key(), context.window(), inputWM, + outputWM); + + return !tooLate; + } + + /** + * Add an end-of-window hold or, if too late for that, a garbage collection hold (if required). + * + *

The end-of-window hold guarantees that an empty {@code ON_TIME} pane can be given + * a timestamp which will not be considered beyond allowed lateness by any downstream computation. + */ + private void addEndOfWindowOrGarbageCollectionHolds(ReduceFn.Context context) { + if (!addEndOfWindowHold(context)) { + addGarbageCollectionHold(context); + } + } + + /** + * Add an end-of-window hold. Return true if was added, false if too late to add. + * + *

The end-of-window hold guarantees that any empty {@code ON_TIME} pane can be given + * a timestamp which will not be considered beyond allowed lateness by any downstream computation. + */ + private boolean addEndOfWindowHold(ReduceFn.Context context) { + // Only add an end-of-window hold if we can be sure the end-of-window timer + // has not yet fired. Otherwise we risk holding up the output watermark until + // the garbage collection timer fires, which may be a very long time in the future. + Instant outputWM = timerInternals.currentOutputWatermarkTime(); + Instant inputWM = timerInternals.currentInputWatermarkTime(); + boolean tooLate; + Instant eowHold = context.window().maxTimestamp(); + if (inputWM != null && eowHold.isBefore(inputWM)) { + tooLate = true; + } else { + tooLate = false; + Preconditions.checkState(outputWM == null || !eowHold.isBefore(outputWM), + "End-of-window hold %s cannot be before output watermark %s", eowHold, outputWM); + context.state().access(EXTRA_HOLD_TAG).add(eowHold); + } + WindowTracing.trace( + "WatermarkHold.addEndOfWindowHold: end-of-window hold at {} is {} for " + + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}", + eowHold, tooLate ? "too late" : "on-time", context.key(), context.window(), inputWM, + outputWM); + return !tooLate; + } + + /** + * Add a garbage collection hold, if required. + * + *

The garbage collection hold gurantees that any empty final pane can be given + * a timestamp which will not be considered beyond allowed lateness by any downstream + * computation. If we are sure no empty final panes can be emitted then there's no need + * for an additional hold. + */ + private void addGarbageCollectionHold(ReduceFn.Context context) { + // Only add a garbage collection hold if we are sure we need an empty final pane and + // the window will be garbage collected after the end-of-window trigger. + if (context.windowingStrategy().getClosingBehavior() == ClosingBehavior.FIRE_ALWAYS + && windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO)) { + Instant gcHold = context.window().maxTimestamp().plus(windowingStrategy.getAllowedLateness()); + Instant outputWM = timerInternals.currentOutputWatermarkTime(); + Instant inputWM = timerInternals.currentInputWatermarkTime(); + WindowTracing.trace( + "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} for " + + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}", + gcHold, context.key(), context.window(), inputWM, outputWM); + Preconditions.checkState(inputWM == null || !gcHold.isBefore(inputWM), + "Garbage collection hold %s cannot be before input watermark %s", gcHold, inputWM); + context.state().access(EXTRA_HOLD_TAG).add(gcHold); + } } /** * Updates the watermark hold when windows merge. For example, if the new window implies * a later watermark hold, then earlier holds may be released. */ - public void mergeHolds(final ReduceFn.OnMergeContext c) { + public void mergeHolds(final ReduceFn.OnMergeContext context) { + WindowTracing.debug("mergeHolds: for key:{}; window:{}; inputWatermark:{}; outputWatermark:{}", + context.key(), context.window(), timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); // If the output hold depends only on the window, then there may not be a hold in place // for the new merged window, so add one. if (windowingStrategy.getOutputTimeFn().dependsOnlyOnWindow()) { Instant arbitraryTimestamp = new Instant(0); - c.state().access(watermarkHoldTag).add( - windowingStrategy.getOutputTimeFn().assignOutputTime( - arbitraryTimestamp, - c.window())); + context.state() + .access(elementHoldTag) + .add(windowingStrategy.getOutputTimeFn().assignOutputTime( + arbitraryTimestamp, context.window())); } - c.state().accessAcrossMergedWindows(watermarkHoldTag).releaseExtraneousHolds(); + context.state().accessAcrossMergedWindows(elementHoldTag).releaseExtraneousHolds(); + context.state().accessAcrossMergedWindows(EXTRA_HOLD_TAG).releaseExtraneousHolds(); } /** - * Returns the combined timestamp at which the output watermark was being held and releases - * the hold. + * Return (a future for) the earliest data hold for {@code context}. Clear the data hold after + * reading. If {@code isFinal}, also clear any end-of-window or garbage collection hold. * *

The returned timestamp is the output timestamp according to the {@link OutputTimeFn} * from the windowing strategy of this {@link WatermarkHold}, combined across all the non-late * elements in the current pane. */ - public StateContents extractAndRelease(final ReduceFn.Context c) { - final WatermarkStateInternal dataHold = c.state().accessAcrossMergedWindows(watermarkHoldTag); - final StateContents holdFuture = dataHold.get(); + public StateContents extractAndRelease( + final ReduceFn.Context context, final boolean isFinal) { + WindowTracing.debug( + "extractAndRelease: for key:{}; window:{}; inputWatermark:{}; outputWatermark:{}", + context.key(), context.window(), timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + final WatermarkStateInternal elementHoldState = + context.state().accessAcrossMergedWindows(elementHoldTag); + final StateContents elementHoldFuture = elementHoldState.get(); + final WatermarkStateInternal extraHoldState = + context.state().accessAcrossMergedWindows(EXTRA_HOLD_TAG); + final StateContents extraHoldFuture = extraHoldState.get(); return new StateContents() { @Override public Instant read() { - Instant hold = holdFuture.read(); - if (hold == null || hold.isAfter(c.window().maxTimestamp())) { - hold = c.window().maxTimestamp(); + // Read both the element and extra holds. + Instant elementHold = elementHoldFuture.read(); + Instant extraHold = extraHoldFuture.read(); + Instant hold; + // Find the minimum, accounting for null. + if (elementHold == null) { + hold = extraHold; + } else if (extraHold == null) { + hold = elementHold; + } else if (elementHold.isBefore(extraHold)) { + hold = elementHold; + } else { + hold = extraHold; + } + if (hold == null || hold.isAfter(context.window().maxTimestamp())) { + // If no hold (eg because all elements came in behind the output watermark), or + // the hold was for garbage collection, take the end of window as the result. + WindowTracing.debug( + "WatermarkHold.extractAndRelease.read: clipping from {} to end of window " + + "for key:{}; window:{}", + hold, context.key(), context.window()); + hold = context.window().maxTimestamp(); } + WindowTracing.debug("WatermarkHold.extractAndRelease.read: clearing for key:{}; window:{}", + context.key(), context.window()); // Clear the underlying state to allow the output watermark to progress. - dataHold.clear(); + elementHoldState.clear(); + extraHoldState.clear(); + + // Reinstate the end-of-window and garbage collection holds if still required. + if (!isFinal) { + addEndOfWindowOrGarbageCollectionHolds(context); + } return hold; } }; } - public void holdForOnTime(final ReduceFn.Context c) { - c.state().accessAcrossMergedWindows(watermarkHoldTag).add(c.window().maxTimestamp()); - } - - public void holdForFinal(final ReduceFn.Context c) { - if (c.windowingStrategy().getClosingBehavior() == ClosingBehavior.FIRE_ALWAYS) { - c.state().accessAcrossMergedWindows(watermarkHoldTag) - .add(c.window().maxTimestamp().plus(c.windowingStrategy().getAllowedLateness())); - } - } - - public void releaseOnTime(final ReduceFn.Context c) { - c.state().accessAcrossMergedWindows(watermarkHoldTag).clear(); - - if (c.windowingStrategy().getClosingBehavior() == ClosingBehavior.FIRE_ALWAYS - && c.windowingStrategy().getAllowedLateness().isLongerThan(Duration.ZERO)) { - holdForFinal(c); - } - } - - public void releaseFinal(final ReduceFn.Context c) { - c.state().accessAcrossMergedWindows(watermarkHoldTag).clear(); + /** Clear any remaining holds. */ + public void clear(ReduceFn.Context context) { + WindowTracing.debug( + "WatermarkHold.clear: for key:{}; window:{}; inputWatermark:{}; outputWatermark:{}", + context.key(), context.window(), timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + context.state().accessAcrossMergedWindows(elementHoldTag).clear(); + context.state().accessAcrossMergedWindows(EXTRA_HOLD_TAG).clear(); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowTracing.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowTracing.java new file mode 100644 index 0000000000000..6ae2f4206c484 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WindowTracing.java @@ -0,0 +1,36 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Logging for window operations. Generally only feasible to enable on hand-picked pipelines. + */ +public final class WindowTracing { + private static final Logger LOG = LoggerFactory.getLogger(WindowTracing.class); + + public static void debug(String format, Object... args) { + LOG.debug(format, args); + } + + @SuppressWarnings("unused") + public static void trace(String format, Object... args) { + LOG.trace(format, args); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergedWatermarkStateInternal.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergedWatermarkStateInternal.java index ec9c492417681..7a1e416aaacee 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergedWatermarkStateInternal.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergedWatermarkStateInternal.java @@ -124,11 +124,17 @@ public Boolean read() { @Override public void releaseExtraneousHolds() { if (outputTimeFn.dependsOnlyOnEarliestInputTimestamp()) { - // No need to do anything; the merged watermark state will hold to the earliest - // due to semantics of watermark holds. + // The backend is implicitly already holding the output watermark to + // the minimum of all holds in all merged windows. Therefore, we don't need to + // explicitly change it. + // When the final (post merged) session window fires, we will collect all holds + // over all intermediate (pre merged) windows, take their min, and clear them. + // Therefore we also don't need to garbage collect any state here. } else { - // In all other cases, get() implements the necessary combining logic, and actually - // performs compaction that releases the watermark. + // The output time function may be able to move the hold forward. get() implements + // the necessary combining logic, and as a side effect will compact the hold + // in Windmill state. This ensures Windmill's output watermark can progress, and + // there is no stale hold left behind. get().read(); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java index 8fe1681c3c970..4c6af1ae32d18 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java @@ -115,9 +115,9 @@ public static StateTag> bag(String id, Coder elemCoder) { /** * Create a state tag for holding the watermark. */ - public static StateTag watermarkStateInternal( - String id, OutputTimeFn outputTimeFn) { - return new WatermarkStateTagInternal(new StructuredId(id), outputTimeFn); + public static StateTag + watermarkStateInternal(String id, OutputTimeFn outputTimeFn) { + return new WatermarkStateTagInternal(new StructuredId(id), outputTimeFn); } /** @@ -357,7 +357,8 @@ private static class WatermarkStateTagInternal /** * When multiple output times are added to hold the watermark, this determines how they are - * combined, and also the behavior when merging windows. + * combined, and also the behavior when merging windows. Does not contribute to equality/hash + * since we have at most one watermark hold tag per computation. */ private final OutputTimeFn outputTimeFn; @@ -381,9 +382,8 @@ public boolean equals(Object obj) { return false; } - WatermarkStateTagInternal that = (WatermarkStateTagInternal) obj; - return Objects.equals(this.id, that.id) - && Objects.equals(this.outputTimeFn, that.outputTimeFn); + WatermarkStateTagInternal that = (WatermarkStateTagInternal) obj; + return Objects.equals(this.id, that.id); } @Override @@ -393,7 +393,7 @@ public int hashCode() { @Override protected StateTag asKind(StateKind kind) { - return new WatermarkStateTagInternal(id.asKind(kind), outputTimeFn); + return new WatermarkStateTagInternal(id.asKind(kind), outputTimeFn); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/WatermarkStateInternal.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/WatermarkStateInternal.java index 29ed90b0b8141..9521088185670 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/WatermarkStateInternal.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/WatermarkStateInternal.java @@ -17,7 +17,7 @@ import com.google.cloud.dataflow.sdk.annotations.Experimental; import com.google.cloud.dataflow.sdk.annotations.Experimental.Kind; -import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; import org.joda.time.Instant; @@ -31,15 +31,8 @@ public interface WatermarkStateInternal extends MergeableState { /** - * Release any holds that have become extraneous so they do not prevent progress of the - * output watermark. - * - *

For example, when using {@link OutputTimeFns#outputAtEndOfWindow()}, there will be holds - * in place at the end of every initial window that merges into the result window. These holds - * need to be released. It is implementation-dependent how (or whether) this happens. - * - *

This method is permitted to be "best effort" but should always try to release holds - * as far as possible to allow the output watermark to make progress. + * Release all holds for windows which have been merged away and incorporate their + * combined values (according to {@link OutputTimeFn#merge}) into the result window hold. */ void releaseExtraneousHolds(); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java index cb7a5c31401a9..e7541f2397622 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java @@ -636,7 +636,8 @@ public void testReadUnboundedReader() throws Exception { .setSourceState( Windmill.SourceState.newBuilder().setState(state).build()) // Source state. .build(), - new Instant(0), + new Instant(0), // input watermark + null, // output watermark null, // StateReader null, // StateFetcher Windmill.WorkItemCommitRequest.newBuilder()); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/KeyedWorkItemTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/KeyedWorkItemTest.java index d4498cff80d8c..60a151c1dd51c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/KeyedWorkItemTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/KeyedWorkItemTest.java @@ -44,7 +44,6 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; -import java.util.concurrent.TimeUnit; /** Tests for {@link KeyedWorkItem}. */ @RunWith(JUnit4.class) @@ -104,7 +103,7 @@ private void addElement( ByteString encodedMetadata = WindmillSink.encodeMetadata(WINDOWS_CODER, Collections.singletonList(window), pane); chunk.addMessagesBuilder() - .setTimestamp(TimeUnit.MILLISECONDS.toMicros(timestamp)) + .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(new Instant(timestamp))) .setData(ByteString.copyFromUtf8(value)) .setMetadata(encodedMetadata); } @@ -142,7 +141,7 @@ private static Windmill.Timer makeSerializedTimer( return Windmill.Timer.newBuilder() .setTag(ByteString.copyFromUtf8( ns.stringKey() + "+" + type + "-" + timestamp)) - .setTimestamp(TimeUnit.MILLISECONDS.toMicros(timestamp)) + .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(new Instant(timestamp))) .setType(type) .setStateFamily(STATE_FAMILY) .build(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java index 938975cd3e405..0c56886d07dbd 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorkerTest.java @@ -779,8 +779,10 @@ public void testMergeWindows() throws Exception { ByteString timerTag = ByteString.copyFromUtf8(window + "+0:999"); // GC timer just has window ByteString bufferTag = ByteString.copyFromUtf8(window + "+sbuf"); ByteString paneInfoTag = ByteString.copyFromUtf8(window + "+spane"); - ByteString watermarkHoldTag = + ByteString watermarkDataHoldTag = ByteString.copyFromUtf8(window + "+shold"); + ByteString watermarkExtraHoldTag = + ByteString.copyFromUtf8(window + "+sextra"); String stateFamily = "MergeWindows"; ByteString bufferData = ByteString.copyFromUtf8("\000data0"); // Encoded form for Iterable: -1, true, 'data0', false @@ -811,12 +813,14 @@ public void testMergeWindows() throws Exception { .build()) .build()))); - assertThat(actualOutput.getWatermarkHoldsList(), Matchers.contains( - Matchers.equalTo(Windmill.WatermarkHold.newBuilder() - .setTag(watermarkHoldTag) - .setStateFamily(stateFamily) - .addTimestamps(0) - .build()))); + assertThat( + actualOutput.getWatermarkHoldsList(), + Matchers.containsInAnyOrder( + Windmill.WatermarkHold.newBuilder() + .setTag(watermarkDataHoldTag) + .setStateFamily(stateFamily) + .addTimestamps(0) + .build())); List counters = actualOutput.getCounterUpdatesList(); // No state reads @@ -861,7 +865,11 @@ public void testMergeWindows() throws Exception { .setTimestamp(0) // is ignored .setData(bufferData); dataBuilder.addWatermarkHoldsBuilder() - .setTag(watermarkHoldTag) + .setTag(watermarkDataHoldTag) + .setStateFamily(stateFamily) + .addTimestamps(0); + dataBuilder.addWatermarkHoldsBuilder() + .setTag(watermarkExtraHoldTag) .setStateFamily(stateFamily) .addTimestamps(0); dataBuilder.addValuesBuilder() @@ -914,13 +922,17 @@ public void testMergeWindows() throws Exception { assertThat( actualOutput.getWatermarkHoldsList(), - Matchers.contains( - Matchers.equalTo( - Windmill.WatermarkHold.newBuilder() - .setTag(watermarkHoldTag) - .setStateFamily(stateFamily) - .setReset(true) - .build()))); + Matchers.containsInAnyOrder( + Windmill.WatermarkHold.newBuilder() + .setTag(watermarkDataHoldTag) + .setStateFamily(stateFamily) + .setReset(true) + .build(), + Windmill.WatermarkHold.newBuilder() + .setTag(watermarkExtraHoldTag) + .setStateFamily(stateFamily) + .setReset(true) + .build())); counters = actualOutput.getCounterUpdatesList(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsDoFnTest.java index 6dadab49b4d53..1c6edd4a74d50 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsDoFnTest.java @@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.when; import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder; import com.google.cloud.dataflow.sdk.coders.Coder; @@ -77,7 +76,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.concurrent.TimeUnit; /** Unit tests for {@link StreamingGroupAlsoByWindowsDoFn}. */ @RunWith(JUnit4.class) @@ -145,7 +143,7 @@ private void addTimer(WorkItem.Builder workItem, namespace, timestamp, type == Windmill.Timer.Type.WATERMARK ? TimeDomain.EVENT_TIME : TimeDomain.PROCESSING_TIME))) - .setTimestamp(TimeUnit.MILLISECONDS.toMicros(timestamp.getMillis())) + .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(timestamp)) .setType(type) .setStateFamily(STATE_FAMILY); } @@ -162,7 +160,7 @@ private void addElement( messageBundle.addMessagesBuilder() .setMetadata(WindmillSink.encodeMetadata(windowsCoder, windows, PaneInfo.NO_FIRING)) .setData(dataOutput.toByteString()) - .setTimestamp(TimeUnit.MILLISECONDS.toMicros(timestamp.getMillis())); + .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(timestamp)); } private WindowedValue> createValue( @@ -181,7 +179,6 @@ private WindowedValue> createValue( outputTag, outputManager, WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))); runner.startBundle(); - when(mockTimerInternals.currentWatermarkTime()).thenReturn(new Instant(0)); WorkItem.Builder workItem1 = WorkItem.newBuilder(); workItem1.setKey(ByteString.copyFromUtf8(KEY)); @@ -238,7 +235,6 @@ private WindowedValue> createValue( SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10)))); runner.startBundle(); - when(mockTimerInternals.currentWatermarkTime()).thenReturn(new Instant(0)); WorkItem.Builder workItem1 = WorkItem.newBuilder(); workItem1.setKey(ByteString.copyFromUtf8(KEY)); @@ -304,7 +300,6 @@ private WindowedValue> createValue( WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))); runner.startBundle(); - when(mockTimerInternals.currentWatermarkTime()).thenReturn(new Instant(0)); WorkItem.Builder workItem1 = WorkItem.newBuilder(); workItem1.setKey(ByteString.copyFromUtf8(KEY)); @@ -326,7 +321,8 @@ private WindowedValue> createValue( WorkItem.Builder workItem2 = WorkItem.newBuilder(); workItem2.setKey(ByteString.copyFromUtf8(KEY)); workItem2.setWorkToken(WORK_TOKEN); - addTimer(workItem2, window(0, 10), new Instant(9), Timer.Type.WATERMARK); + // Note that the WATERMARK timer for Instant(9) will have been deleted by + // ReduceFnRunner when window(0, 10) was merged away. addTimer(workItem2, window(0, 15), new Instant(14), Timer.Type.WATERMARK); addTimer(workItem2, window(15, 25), new Instant(24), Timer.Type.WATERMARK); @@ -398,7 +394,6 @@ public Long extractOutput(Long accumulator) { appliedCombineFn); runner.startBundle(); - when(mockTimerInternals.currentWatermarkTime()).thenReturn(new Instant(0)); WorkItem.Builder workItem1 = WorkItem.newBuilder(); workItem1.setKey(ByteString.copyFromUtf8(KEY)); @@ -420,7 +415,8 @@ public Long extractOutput(Long accumulator) { WorkItem.Builder workItem2 = WorkItem.newBuilder(); workItem2.setKey(ByteString.copyFromUtf8(KEY)); workItem2.setWorkToken(WORK_TOKEN); - addTimer(workItem2, window(0, 10), new Instant(9), Timer.Type.WATERMARK); + // Note that the WATERMARK timer for Instant(9) will have been deleted by + // ReduceFnRunner when window(0, 10) was merged away. addTimer(workItem2, window(0, 15), new Instant(14), Timer.Type.WATERMARK); addTimer(workItem2, window(15, 25), new Instant(24), Timer.Type.WATERMARK); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java index a17aca014ae5a..6bb9e5b83d49b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java @@ -57,7 +57,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.concurrent.TimeUnit; /** Unit tests for {@link StreamingGroupAlsoByWindowsReshuffleDoFn}. */ @RunWith(JUnit4.class) @@ -116,7 +115,7 @@ private void addElement( messageBundle.addMessagesBuilder() .setMetadata(WindmillSink.encodeMetadata(windowsCoder, windows, PaneInfo.NO_FIRING)) .setData(dataOutput.toByteString()) - .setTimestamp(TimeUnit.MILLISECONDS.toMicros(timestamp.getMillis())); + .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(timestamp)); } private WindowedValue> createValue( diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContextTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContextTest.java index 2c4e11fb84bee..c0062ba9284bd 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContextTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContextTest.java @@ -81,7 +81,10 @@ public void testTimerInternalsSetTimer() { Windmill.WorkItemCommitRequest.Builder outputBuilder = Windmill.WorkItemCommitRequest.newBuilder(); - executionContext.start(null, new Instant(1000), stateReader, stateFetcher, outputBuilder); + executionContext.start(null, + new Instant(1000), // input watermark + null, // output watermark + stateReader, stateFetcher, outputBuilder); ExecutionContext.StepContext step = executionContext.getOrCreateStepContext("step", "transform", null); @@ -141,7 +144,8 @@ private void startContext( .setState(ByteString.EMPTY) .build()) // Source state. .build(), - new Instant(0), + new Instant(0), // input watermark + null, // output watermark null, // StateReader null, // StateFetcher Windmill.WorkItemCommitRequest.newBuilder()); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateReaderTest.java index f6ad4190fd1ff..961c33025c3d9 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateReaderTest.java @@ -42,7 +42,6 @@ import java.io.IOException; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; /** * Tests for {@link WindmillStateReader}. @@ -83,7 +82,8 @@ private Windmill.Value intValue(int value, boolean padded) throws IOException { return Windmill.Value.newBuilder() .setData(output.toByteString()) - .setTimestamp(TimeUnit.MILLISECONDS.toMicros(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis())) + .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp( + BoundedWindow.TIMESTAMP_MAX_VALUE)) .build(); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java index 87a00db9b55e9..e4855fb1f7084 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java @@ -117,7 +117,7 @@ public void testOnTimerFire() throws Exception { when(mockTrigger1.onTimer(Mockito..OnTimerContext>any())) .thenReturn(TriggerResult.FIRE_AND_FINISH); tester.fireTimer(firstWindow, new Instant(11), TimeDomain.EVENT_TIME); - tester.advanceWatermark(new Instant(12)); + tester.advanceInputWatermark(new Instant(12)); assertThat(tester.extractOutput(), Matchers.contains( isSingleWindowedValue(Matchers.containsInAnyOrder(1), 1, 0, 10))); @@ -139,7 +139,7 @@ public void testOnTimerFireAndFinish() throws Exception { .thenReturn(TriggerResult.FIRE_AND_FINISH); tester.fireTimer(firstWindow, new Instant(11), TimeDomain.EVENT_TIME); - tester.advanceWatermark(new Instant(12)); + tester.advanceInputWatermark(new Instant(12)); assertThat(tester.extractOutput(), Matchers.emptyIterable()); assertFalse(tester.isMarkedFinished(firstWindow)); @@ -239,7 +239,7 @@ public void testAfterAllRealTriggersFixedWindow() throws Exception { TimestampedValue.of(5, new Instant(2))); assertThat(tester.extractOutput(), Matchers.emptyIterable()); - tester.advanceProcessingTime(new Instant(5)); + tester.advanceProcessingTime(new Instant(6)); assertThat(tester.extractOutput(), Matchers.contains( isSingleWindowedValue(Matchers.containsInAnyOrder(0, 1, 2, 3, 4, 5), 0, 0, 50))); @@ -251,7 +251,7 @@ public void testAfterAllRealTriggersFixedWindow() throws Exception { TimestampedValue.of(7, new Instant(3)), TimestampedValue.of(8, new Instant(4)), TimestampedValue.of(9, new Instant(5))); - tester.advanceProcessingTime(new Instant(20)); + tester.advanceProcessingTime(new Instant(21)); assertThat(tester.extractOutput(), Matchers.emptyIterable()); tester.injectElements( TimestampedValue.of(10, new Instant(6))); @@ -279,7 +279,7 @@ public void testAfterAllMergingWindowSomeFinished() throws Exception { tester.advanceProcessingTime(new Instant(10)); tester.injectElements( TimestampedValue.of(1, new Instant(1))); // in [1, 11), timer for 15 - tester.advanceProcessingTime(new Instant(15)); + tester.advanceProcessingTime(new Instant(16)); tester.injectElements( TimestampedValue.of(2, new Instant(1)), // in [1, 11) count = 1 TimestampedValue.of(3, new Instant(2))); // in [2, 12), timer for 16 diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEachTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEachTest.java index a119425f08d28..93084bfe0fde0 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEachTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEachTest.java @@ -140,7 +140,7 @@ public void testOnTimerFinish() throws Exception { when(mockTrigger1.onTimer(Mockito.isA(OnTimerContext.class))) .thenReturn(TriggerResult.FIRE_AND_FINISH); - tester.advanceWatermark(new Instant(12)); + tester.advanceInputWatermark(new Instant(12)); assertThat(tester.extractOutput(), Matchers.contains( isSingleWindowedValue(Matchers.containsInAnyOrder(1), 1, 0, 10))); @@ -261,7 +261,7 @@ public void testAfterEachMergingWindowSomeFinished() throws Exception { tester.advanceProcessingTime(new Instant(10)); tester.injectElements( TimestampedValue.of(1, new Instant(1))); // in [1, 11), timer for 15 - tester.advanceProcessingTime(new Instant(15)); + tester.advanceProcessingTime(new Instant(16)); assertThat(tester.extractOutput(), Matchers.contains( WindowMatchers.isSingleWindowedValue(Matchers.contains(1), 1, 1, 11))); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirstTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirstTest.java index 23910f1a92cdb..63a240a466a27 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirstTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirstTest.java @@ -236,7 +236,7 @@ public void testAfterFirstRealTriggersFixedWindow() throws Exception { TimestampedValue.of(8, new Instant(5))); tester.advanceProcessingTime(new Instant(5)); assertThat(tester.extractOutput(), Matchers.emptyIterable()); - tester.advanceProcessingTime(new Instant(6)); + tester.advanceProcessingTime(new Instant(7)); assertThat(tester.extractOutput(), Matchers.contains( isSingleWindowedValue(Matchers.containsInAnyOrder(5, 6, 7, 8), 2, 0, 50))); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java index 0096216d71f58..cada240cd0e70 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java @@ -122,7 +122,7 @@ public void testAfterProcessingTimeWithMergingWindow() throws Exception { tester.injectElements( TimestampedValue.of(2, new Instant(2))); // in [2, 12), timer for 16 - tester.advanceProcessingTime(new Instant(15)); + tester.advanceProcessingTime(new Instant(16)); // This fires, because the earliest element in [1, 12) arrived at time 10 assertThat(tester.extractOutput(), Matchers.contains( WindowMatchers.isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 1, 12))); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermarkTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermarkTest.java index 7450752ec2346..9f34a9c920e14 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermarkTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermarkTest.java @@ -73,12 +73,12 @@ public void testFirstInPaneWithFixedWindow() throws Exception { tester.injectElements( TimestampedValue.of(1, new Instant(1))); // first in window [0, 10), timer set for 6 - tester.advanceWatermark(new Instant(5)); + tester.advanceInputWatermark(new Instant(5)); tester.injectElements( TimestampedValue.of(2, new Instant(9)), TimestampedValue.of(3, new Instant(8))); - tester.advanceWatermark(new Instant(6)); + tester.advanceInputWatermark(new Instant(7)); assertThat(tester.extractOutput(), Matchers.contains( WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2, 3), 1, 0, 10))); @@ -114,12 +114,12 @@ public void testAlignAndDelay() throws Exception { TimestampedValue.of(2, zero.plus(Duration.standardSeconds(5))), TimestampedValue.of(3, zero.plus(Duration.standardSeconds(55)))); - // Advance almost to 6m, but not quite. No output should be produced. - tester.advanceWatermark(zero.plus(Duration.standardMinutes(6)).minus(1)); + // Advance to 6m. No output should be produced. + tester.advanceInputWatermark(zero.plus(Duration.standardMinutes(6))); assertThat(tester.extractOutput(), Matchers.emptyIterable()); - // Advance to 6m and see our output - tester.advanceWatermark(zero.plus(Duration.standardMinutes(6))); + // Advance to 6m+1ms and see our output + tester.advanceInputWatermark(zero.plus(Duration.standardMinutes(6).plus(1))); assertThat(tester.extractOutput(), Matchers.contains( WindowMatchers.isSingleWindowedValue( containsInAnyOrder(1, 2, 3), @@ -136,12 +136,12 @@ public void testFirstInPaneWithMerging() throws Exception { AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); - tester.advanceWatermark(new Instant(1)); + tester.advanceInputWatermark(new Instant(1)); tester.injectElements( TimestampedValue.of(1, new Instant(1)), // in [1, 11), timer for 6 TimestampedValue.of(2, new Instant(2))); // in [2, 12), timer for 7 - tester.advanceWatermark(new Instant(6)); + tester.advanceInputWatermark(new Instant(7)); // We merged, and updated the watermark timer to the earliest timer, which was still 6. assertThat(tester.extractOutput(), Matchers.contains( @@ -164,13 +164,13 @@ public void testEndOfWindowFixedWindow() throws Exception { tester.injectElements( TimestampedValue.of(1, new Instant(1))); // first in window [0, 10), timer set for 9 - tester.advanceWatermark(new Instant(8)); + tester.advanceInputWatermark(new Instant(8)); assertThat(tester.extractOutput(), Matchers.emptyIterable()); tester.injectElements( TimestampedValue.of(2, new Instant(9)), TimestampedValue.of(3, new Instant(8))); - tester.advanceWatermark(new Instant(9)); + tester.advanceInputWatermark(new Instant(10)); assertThat(tester.extractOutput(), Matchers.contains( WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2, 3), 1, 0, 10))); @@ -196,19 +196,19 @@ public void testEndOfWindowWithMerging() throws Exception { AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); - tester.advanceWatermark(new Instant(1)); + tester.advanceInputWatermark(new Instant(1)); tester.injectElements( - TimestampedValue.of(1, new Instant(1)), // in [1, 11), timer for 10 - TimestampedValue.of(2, new Instant(2))); // in [2, 12), timer for 11 - tester.advanceWatermark(new Instant(10)); + TimestampedValue.of(1, new Instant(1)), // in [1, 11], timer for 11 + TimestampedValue.of(2, new Instant(2))); // in [2, 12], timer for 12 + tester.advanceInputWatermark(new Instant(10)); // We merged, and updated the watermark timer to the end of the new window. assertThat(tester.extractOutput(), Matchers.emptyIterable()); tester.injectElements( - TimestampedValue.of(3, new Instant(1))); // in [1, 11), timer for 10 - tester.advanceWatermark(new Instant(11)); + TimestampedValue.of(3, new Instant(1))); // in [1, 11], timer for 11 + tester.advanceInputWatermark(new Instant(12)); assertThat(tester.extractOutput(), Matchers.contains( WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2, 3), 1, 1, 12))); @@ -286,7 +286,7 @@ public void testEarlyAndAtWatermarkProcessElement() throws Exception { TimestampedValue.of(2, new Instant(10L))); // Fires due to early trigger tester.injectElements(TimestampedValue.of(3, new Instant(15L))); - tester.advanceWatermark(new Instant(100L)); // Fires due to AtWatermark + tester.advanceInputWatermark(new Instant(100L)); // Fires due to AtWatermark List>> output = tester.extractOutput(); assertThat(output, Matchers.contains( isSingleWindowedValue(containsInAnyOrder(1, 2), 5, 0, 100), @@ -308,7 +308,7 @@ public void testLateAndAtWatermarkProcessElement() throws Exception { TimestampedValue.of(2, new Instant(10L)), TimestampedValue.of(3, new Instant(15L))); - tester.advanceWatermark(new Instant(100L)); // Fires due to AtWatermark + tester.advanceInputWatermark(new Instant(100L)); // Fires due to AtWatermark when(mockLate.onElement(Mockito..OnElementContext>any())) .thenReturn(TriggerResult.CONTINUE) @@ -354,7 +354,7 @@ public void testEarlyLateAndAtWatermarkProcessElement() throws Exception { tester.injectElements( TimestampedValue.of(3, new Instant(15L))); - tester.advanceWatermark(new Instant(100L)); // Fires due to AtWatermark + tester.advanceInputWatermark(new Instant(100L)); // Fires due to AtWatermark when(mockLate.onElement(Mockito..OnElementContext>any())) .thenReturn(TriggerResult.CONTINUE) @@ -396,7 +396,7 @@ public void testEarlyAndAtWatermarkSessions() throws Exception { assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(5), new Instant(40)))); tester.injectElements(TimestampedValue.of(3, new Instant(6L))); - tester.advanceWatermark(new Instant(100L)); // Fires due to AtWatermark + tester.advanceInputWatermark(new Instant(100L)); // Fires due to AtWatermark List>> output = tester.extractOutput(); assertThat(output, Matchers.contains( isSingleWindowedValue(containsInAnyOrder(1, 2), 5, 5, 40), @@ -419,7 +419,7 @@ public void testLateAndAtWatermarkSessionsProcessingTime() throws Exception { TimestampedValue.of(1, new Instant(5L)), TimestampedValue.of(2, new Instant(20L))); - tester.advanceWatermark(new Instant(70L)); // Fires due to AtWatermark + tester.advanceInputWatermark(new Instant(70L)); // Fires due to AtWatermark tester.injectElements(TimestampedValue.of(3, new Instant(6L))); @@ -449,7 +449,7 @@ public void testLateAndAtWatermarkSessions() throws Exception { TimestampedValue.of(1, new Instant(5L)), TimestampedValue.of(2, new Instant(20L))); - tester.advanceWatermark(new Instant(70L)); // Fires due to AtWatermark + tester.advanceInputWatermark(new Instant(70L)); // Fires due to AtWatermark IntervalWindow window = new IntervalWindow(new Instant(5), new Instant(40)); assertFalse(tester.isMarkedFinished(window)); @@ -482,9 +482,9 @@ public void testEarlyLateAndAtWatermarkSessions() throws Exception { TimestampedValue.of(1, new Instant(5L)), TimestampedValue.of(2, new Instant(20L))); - tester.advanceProcessingTime(new Instant(55)); // Fires due to early trigger + tester.advanceProcessingTime(new Instant(56)); // Fires due to early trigger - tester.advanceWatermark(new Instant(70L)); // Fires due to AtWatermark + tester.advanceInputWatermark(new Instant(70L)); // Fires due to AtWatermark assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(5), new Instant(40)))); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/DefaultTriggerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/DefaultTriggerTest.java index 01788e00d9428..ab9c868a5e5a1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/DefaultTriggerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/DefaultTriggerTest.java @@ -58,18 +58,18 @@ public void testDefaultTriggerWithFixedWindow() throws Exception { // Advance the watermark almost to the end of the first window. tester.advanceProcessingTime(new Instant(500)); - tester.advanceWatermark(new Instant(8)); + tester.advanceInputWatermark(new Instant(8)); assertThat(tester.extractOutput(), Matchers.emptyIterable()); - // Advance watermark to 9 (the exact end of the window), which causes the first fixed window to + // Advance watermark to 10 (past end of the window), which causes the first fixed window to // be emitted - tester.advanceWatermark(new Instant(9)); + tester.advanceInputWatermark(new Instant(10)); assertThat(tester.extractOutput(), Matchers.contains( isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 0, 10))); // Advance watermark to 100, which causes the remaining two windows to be emitted. // Since their timers were at different timestamps, they should fire in order. - tester.advanceWatermark(new Instant(100)); + tester.advanceInputWatermark(new Instant(100)); assertThat(tester.extractOutput(), Matchers.contains( isSingleWindowedValue(Matchers.containsInAnyOrder(3, 4), 15, 10, 20), isSingleWindowedValue(Matchers.contains(5), 30, 30, 40))); @@ -93,14 +93,14 @@ public void testDefaultTriggerWithSessionWindow() throws Exception { TimestampedValue.of(2, new Instant(9))); // no output, because we merged into the [9-19) session - tester.advanceWatermark(new Instant(10)); + tester.advanceInputWatermark(new Instant(10)); assertThat(tester.extractOutput(), Matchers.emptyIterable()); tester.injectElements( TimestampedValue.of(3, new Instant(15)), TimestampedValue.of(4, new Instant(30))); - tester.advanceWatermark(new Instant(100)); + tester.advanceInputWatermark(new Instant(100)); assertThat(tester.extractOutput(), Matchers.contains( isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2, 3), 1, 1, 25), isSingleWindowedValue(Matchers.contains(4), 30, 30, 40))); @@ -124,19 +124,18 @@ public void testDefaultTriggerWithSlidingWindow() throws Exception { TimestampedValue.of(2, new Instant(4)), TimestampedValue.of(3, new Instant(9))); - tester.advanceWatermark(new Instant(100)); + tester.advanceInputWatermark(new Instant(100)); assertThat(tester.extractOutput(), Matchers.contains( isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, -5, 5), isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2, 3), 5, 0, 10), isSingleWindowedValue(Matchers.containsInAnyOrder(3), 10, 5, 15))); - // This data is late, so it will hold the watermark to 109 + // This data is too late to hold the output watermark, either to the element + // or the end of window. tester.injectElements( TimestampedValue.of(4, new Instant(8))); - tester.advanceWatermark(new Instant(101)); - assertThat(tester.getWatermarkHold(), Matchers.equalTo(new Instant(109))); - tester.advanceWatermark(new Instant(120)); + tester.advanceInputWatermark(new Instant(120)); List>> output = tester.extractOutput(); assertThat(output, Matchers.contains( isSingleWindowedValue(Matchers.contains(4), 9, 0, 10), @@ -160,7 +159,7 @@ public void testDefaultTriggerWithContainedSessionWindow() throws Exception { TimestampedValue.of(2, new Instant(9)), TimestampedValue.of(3, new Instant(7))); - tester.advanceWatermark(new Instant(20)); + tester.advanceInputWatermark(new Instant(20)); Iterable>> extractOutput = tester.extractOutput(); assertThat(extractOutput, Matchers.contains( isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2, 3), 1, 1, 19))); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTriggerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTriggerTest.java index b6b768947ff35..e5be23527fa63 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTriggerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTriggerTest.java @@ -252,7 +252,7 @@ public void testOrFinallyRealTriggersFixedWindow() throws Exception { TimestampedValue.of(1, new Instant(0)), TimestampedValue.of(2, new Instant(1)), TimestampedValue.of(3, new Instant(1))); - tester.advanceProcessingTime(new Instant(5)); + tester.advanceProcessingTime(new Instant(6)); assertThat(tester.extractOutput(), Matchers.emptyIterable()); tester.injectElements( diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BatchTimerInternalsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BatchTimerInternalsTest.java index b92f5a227f4f3..1accd76248516 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BatchTimerInternalsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/BatchTimerInternalsTest.java @@ -86,7 +86,7 @@ public void testTimerOrdering() { underTest.setTimer(processingTime2); underTest.setTimer(watermarkTime2); - underTest.advanceWatermark(mockRunner, new Instant(30)); + underTest.advanceInputWatermark(mockRunner, new Instant(30)); Mockito.verify(mockRunner).onTimer(watermarkTime1); Mockito.verify(mockRunner).onTimer(watermarkTime2); Mockito.verifyNoMoreInteractions(mockRunner); @@ -107,7 +107,7 @@ public void testDeduplicate() { underTest.setTimer(processingTime); underTest.setTimer(processingTime); underTest.advanceProcessingTime(mockRunner, new Instant(20)); - underTest.advanceWatermark(mockRunner, new Instant(20)); + underTest.advanceInputWatermark(mockRunner, new Instant(20)); Mockito.verify(mockRunner).onTimer(processingTime); Mockito.verify(mockRunner).onTimer(watermarkTime); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsProperties.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsProperties.java index 10af67d68b9b7..34edb2489e388 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsProperties.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/GroupAlsoByWindowsProperties.java @@ -423,62 +423,46 @@ public static void groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp( */ public static void groupsElementsIntoFixedWindowsWithCustomTimestamp( GroupAlsoByWindowsDoFnFactory> gabwFactory) - throws Exception { - + throws Exception { WindowingStrategy windowingStrategy = WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) - .withOutputTimeFn(new OutputTimeFn.Defaults() { - - @Override - public Instant assignOutputTime(Instant inputTimestamp, IntervalWindow window) { - return window.start(); - } - - @Override - public Instant combine(Instant outputTime, Instant otherOutputTime) { - return outputTime; - } - - @Override - public boolean dependsOnlyOnEarliestInputTimestamp() { - return true; - } - }); - - List>>> result = - runGABW(gabwFactory, windowingStrategy, "key", - WindowedValue.of( - "v1", - new Instant(1), - Arrays.asList(window(0, 10)), - PaneInfo.NO_FIRING), - WindowedValue.of( - "v2", - new Instant(2), - Arrays.asList(window(0, 10)), - PaneInfo.NO_FIRING), - WindowedValue.of( - "v3", - new Instant(13), - Arrays.asList(window(10, 20)), - PaneInfo.NO_FIRING)); + .withOutputTimeFn(new OutputTimeFn.Defaults() { + @Override + public Instant assignOutputTime(Instant inputTimestamp, IntervalWindow window) { + return inputTimestamp.isBefore(window.maxTimestamp()) + ? inputTimestamp.plus(1) : window.maxTimestamp(); + } + + @Override + public Instant combine(Instant outputTime, Instant otherOutputTime) { + return outputTime.isBefore(otherOutputTime) ? outputTime : otherOutputTime; + } + + @Override + public boolean dependsOnlyOnEarliestInputTimestamp() { + return true; + } + }); + + List>>> result = runGABW(gabwFactory, + windowingStrategy, "key", + WindowedValue.of("v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + WindowedValue.of("v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + WindowedValue.of("v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result.size(), equalTo(2)); WindowedValue>> item0 = result.get(0); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); assertThat(item0.getWindows(), contains(window(0, 10))); - assertThat(item0.getTimestamp(), - equalTo(((IntervalWindow) Iterables.getOnlyElement(item0.getWindows())).start())); + assertThat(item0.getTimestamp(), equalTo(new Instant(2))); WindowedValue>> item1 = result.get(1); assertThat(item1.getValue().getValue(), contains("v3")); assertThat(item1.getWindows(), contains(window(10, 20))); - assertThat(item1.getTimestamp(), - equalTo(((IntervalWindow) Iterables.getOnlyElement(item1.getWindows())).start())); + assertThat(item1.getTimestamp(), equalTo(new Instant(14))); } - /** * Tests that for a simple sequence of elements on the same key, the given GABW implementation * correctly groups them according to fixed windows and also sets the output timestamp diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerExecutorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerExecutorTest.java index 104304fae5d47..c6d56bc15d887 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerExecutorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerExecutorTest.java @@ -194,81 +194,109 @@ public void testOnElementCombiningAccumulating() throws Exception { @Test public void testWatermarkHoldAndLateData() throws Exception { // Test handling of late data. Specifically, ensure the watermark hold is correct. - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( - FixedWindows.of(Duration.millis(10)), - mockTrigger, - AccumulationMode.ACCUMULATING_FIRED_PANES, - Duration.millis(10)); + TriggerTester, IntervalWindow> tester = + TriggerTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger, + AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(10)); + + // Input watermark -> null + assertEquals(null, tester.getWatermarkHold()); + assertEquals(null, tester.getOutputWatermark()); // All on time data, verify watermark hold. injectElement(tester, 1, TriggerResult.CONTINUE); injectElement(tester, 3, TriggerResult.CONTINUE); assertEquals(new Instant(1), tester.getWatermarkHold()); injectElement(tester, 2, TriggerResult.FIRE); + assertEquals(1, tester.getOutputSize()); // Holding for the end-of-window transition. assertEquals(new Instant(9), tester.getWatermarkHold()); - + // Nothing dropped. assertEquals(0, tester.getElementsDroppedDueToLateness()); assertEquals(0, tester.getElementsDroppedDueToClosedWindow()); + // Input watermark -> 4 + tester.advanceInputWatermark(new Instant(4)); + assertEquals(new Instant(4), tester.getOutputWatermark()); + // Some late, some on time. Verify that we only hold to the minimum of on-time. - tester.advanceWatermark(new Instant(4)); injectElement(tester, 2, TriggerResult.CONTINUE); injectElement(tester, 3, TriggerResult.CONTINUE); assertEquals(new Instant(9), tester.getWatermarkHold()); injectElement(tester, 5, TriggerResult.CONTINUE); assertEquals(new Instant(5), tester.getWatermarkHold()); injectElement(tester, 4, TriggerResult.FIRE); + assertEquals(2, tester.getOutputSize()); // All late -- output at end of window timestamp. when(mockTrigger.onTimer(Mockito..OnTimerContext>any())) - .thenReturn(TriggerResult.CONTINUE); - tester.advanceWatermark(new Instant(8)); + .thenReturn(TriggerResult.CONTINUE); + // Input watermark -> 8 + tester.advanceInputWatermark(new Instant(8)); + assertEquals(new Instant(8), tester.getOutputWatermark()); injectElement(tester, 6, TriggerResult.CONTINUE); injectElement(tester, 5, TriggerResult.CONTINUE); assertEquals(new Instant(9), tester.getWatermarkHold()); - injectElement(tester, 4, TriggerResult.FIRE); + injectElement(tester, 4, TriggerResult.CONTINUE); - // This is "pending" at the time the watermark makes it way-late. - // Because we're about to expire the window, we output it. + // This is behind both the input and output watermarks, but will still make it + // into an ON_TIME pane. + when(mockTrigger.onTimer(Mockito..OnTimerContext>any())) + .thenReturn(TriggerResult.FIRE); + // Input watermark -> 10 + tester.advanceInputWatermark(new Instant(10)); + assertEquals(3, tester.getOutputSize()); + assertEquals(new Instant(10), tester.getOutputWatermark()); injectElement(tester, 8, TriggerResult.CONTINUE); assertEquals(0, tester.getElementsDroppedDueToLateness()); assertEquals(0, tester.getElementsDroppedDueToClosedWindow()); // All very late -- gets dropped. - tester.advanceWatermark(new Instant(50)); + // Input watermark -> 50 + tester.advanceInputWatermark(new Instant(50)); + assertEquals(new Instant(50), tester.getOutputWatermark()); assertEquals(null, tester.getWatermarkHold()); - injectElement(tester, 2, TriggerResult.FIRE); + injectElement(tester, 22, TriggerResult.FIRE); + assertEquals(4, tester.getOutputSize()); assertEquals(null, tester.getWatermarkHold()); - // Late timers are ignored - tester.fireTimer(new IntervalWindow(new Instant(0), new Instant(10)), - new Instant(12), TimeDomain.EVENT_TIME); - assertEquals(1, tester.getElementsDroppedDueToLateness()); assertEquals(0, tester.getElementsDroppedDueToClosedWindow()); + // Late timers are ignored + tester.fireTimer(new IntervalWindow(new Instant(0), new Instant(10)), new Instant(12), + TimeDomain.EVENT_TIME); + List>> output = tester.extractOutput(); - assertThat(output, Matchers.contains( - isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2, 3), 1, 0, 10), - isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2, 3, 2, 3, 4, 5), 4, 0, 10), - // Output time is end of the window, because all the new data was late - isSingleWindowedValue(Matchers.containsInAnyOrder( - 1, 2, 3, 2, 3, 4, 5, 4, 5, 6), 9, 0, 10), - // Output time is not end of the window, because the new data (8) wasn't late - isSingleWindowedValue(Matchers.containsInAnyOrder( - 1, 2, 3, 2, 3, 4, 5, 4, 5, 6, 8), 8, 0, 10))); + assertThat( + output, Matchers.contains( + isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2, 3), 1, 0, 10), + isSingleWindowedValue( + Matchers.containsInAnyOrder(1, 2, 3, 2, 3, 4, 5), 4, 0, 10), + // Output time is end of the window, because all the new data was late + isSingleWindowedValue( + Matchers.containsInAnyOrder(1, 2, 3, 2, 3, 4, 5, 4, 5, 6), 9, 0, 10), + // Output time is still end of the window, because the new data (8) was behind + // the output watermark. + isSingleWindowedValue( + Matchers.containsInAnyOrder(1, 2, 3, 2, 3, 4, 5, 4, 5, 6, 8), 9, 0, 10))); + + assertThat( + output.get(0).getPane(), + Matchers.equalTo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))); assertThat( - output.get(0).getPane(), - Matchers.equalTo(PaneInfo.createPane(true, false, Timing.EARLY))); + output.get(1).getPane(), + Matchers.equalTo(PaneInfo.createPane(false, false, Timing.EARLY, 1, -1))); + + assertThat( + output.get(2).getPane(), + Matchers.equalTo(PaneInfo.createPane(false, false, Timing.ON_TIME, 2, 0))); - // By the time this firing is produced, the input WM already passed the end of the window. assertThat( output.get(3).getPane(), - Matchers.equalTo(PaneInfo.createPane(false, true, Timing.LATE, 3, 0))); + Matchers.equalTo(PaneInfo.createPane(false, true, Timing.LATE, 3, 1))); // And because we're past the end of window + allowed lateness, everything should be cleaned up. assertFalse(tester.isMarkedFinished(firstWindow)); @@ -286,7 +314,7 @@ public void testPaneInfoAllStates() throws Exception { when(mockTrigger.onTimer(Mockito..OnTimerContext>any())) .thenReturn(TriggerResult.CONTINUE); - tester.advanceWatermark(new Instant(0)); + tester.advanceInputWatermark(new Instant(0)); injectElement(tester, 1, TriggerResult.FIRE); assertThat(tester.extractOutput(), Matchers.contains( WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY)))); @@ -295,19 +323,24 @@ public void testPaneInfoAllStates() throws Exception { assertThat(tester.extractOutput(), Matchers.contains( WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.EARLY, 1, -1)))); - tester.advanceWatermark(new Instant(15)); + tester.advanceInputWatermark(new Instant(15)); injectElement(tester, 3, TriggerResult.FIRE); - assertThat(tester.extractOutput(), Matchers.contains( - // This is late, because the trigger wasn't waiting for AfterWatermark - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.LATE, 2, 0)))); + assertThat( + tester.extractOutput(), + Matchers.contains(WindowMatchers.valueWithPaneInfo( + PaneInfo.createPane(false, false, Timing.EARLY, 2, -1)))); injectElement(tester, 4, TriggerResult.FIRE); - assertThat(tester.extractOutput(), Matchers.contains( - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.LATE, 3, 1)))); + assertThat( + tester.extractOutput(), + Matchers.contains(WindowMatchers.valueWithPaneInfo( + PaneInfo.createPane(false, false, Timing.EARLY, 3, -1)))); injectElement(tester, 5, TriggerResult.FIRE_AND_FINISH); - assertThat(tester.extractOutput(), Matchers.contains( - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 4, 2)))); + assertThat( + tester.extractOutput(), + Matchers.contains(WindowMatchers.valueWithPaneInfo( + PaneInfo.createPane(false, true, Timing.EARLY, 4, -1)))); } @Test @@ -322,7 +355,7 @@ public void testPaneInfoAllStatesAfterWatermark() throws Exception { .withAllowedLateness(Duration.millis(100)) .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS)); - tester.advanceWatermark(new Instant(0)); + tester.advanceInputWatermark(new Instant(0)); tester.injectElements( TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); @@ -333,7 +366,7 @@ public void testPaneInfoAllStatesAfterWatermark() throws Exception { assertThat(output, Matchers.contains( WindowMatchers.isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 0, 10))); - tester.advanceWatermark(new Instant(50)); + tester.advanceInputWatermark(new Instant(50)); // We should get the ON_TIME pane even though it is empty, // because we have an AfterWatermark.pastEndOfWindow() trigger. @@ -344,7 +377,7 @@ public void testPaneInfoAllStatesAfterWatermark() throws Exception { WindowMatchers.isSingleWindowedValue(Matchers.emptyIterable(), 9, 0, 10))); // We should get the final pane even though it is empty. - tester.advanceWatermark(new Instant(150)); + tester.advanceInputWatermark(new Instant(150)); output = tester.extractOutput(); assertThat(output, Matchers.contains( WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1)))); @@ -364,7 +397,7 @@ public void testPaneInfoAllStatesAfterWatermarkAccumulating() throws Exception { .withAllowedLateness(Duration.millis(100)) .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS)); - tester.advanceWatermark(new Instant(0)); + tester.advanceInputWatermark(new Instant(0)); tester.injectElements( TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); @@ -375,7 +408,7 @@ public void testPaneInfoAllStatesAfterWatermarkAccumulating() throws Exception { assertThat(output, Matchers.contains( WindowMatchers.isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 0, 10))); - tester.advanceWatermark(new Instant(50)); + tester.advanceInputWatermark(new Instant(50)); // We should get the ON_TIME pane even though it is empty, // because we have an AfterWatermark.pastEndOfWindow() trigger. @@ -386,7 +419,7 @@ public void testPaneInfoAllStatesAfterWatermarkAccumulating() throws Exception { WindowMatchers.isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 9, 0, 10))); // We should get the final pane even though it is empty. - tester.advanceWatermark(new Instant(150)); + tester.advanceInputWatermark(new Instant(150)); output = tester.extractOutput(); assertThat(output, Matchers.contains( WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1)))); @@ -405,7 +438,7 @@ public void testPaneInfoFinalAndOnTime() throws Exception { .withAllowedLateness(Duration.millis(100)) .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS)); - tester.advanceWatermark(new Instant(0)); + tester.advanceInputWatermark(new Instant(0)); tester.injectElements( TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); @@ -413,7 +446,7 @@ public void testPaneInfoFinalAndOnTime() throws Exception { assertThat(tester.extractOutput(), Matchers.contains( WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)))); - tester.advanceWatermark(new Instant(150)); + tester.advanceInputWatermark(new Instant(150)); assertThat(tester.extractOutput(), Matchers.contains( WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.ON_TIME, 1, 0)))); } @@ -426,7 +459,7 @@ public void testPaneInfoSkipToFinish() throws Exception { AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); - tester.advanceWatermark(new Instant(0)); + tester.advanceInputWatermark(new Instant(0)); injectElement(tester, 1, TriggerResult.FIRE_AND_FINISH); assertThat(tester.extractOutput(), Matchers.contains( WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.EARLY)))); @@ -440,7 +473,7 @@ public void testPaneInfoSkipToNonSpeculativeAndFinish() throws Exception { AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); - tester.advanceWatermark(new Instant(15)); + tester.advanceInputWatermark(new Instant(15)); injectElement(tester, 1, TriggerResult.FIRE_AND_FINISH); assertThat(tester.extractOutput(), Matchers.contains( WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.LATE)))); @@ -468,13 +501,14 @@ public void testMergeBeforeFinalizing() throws Exception { when(mockTrigger.onTimer(Mockito..OnTimerContext>any())) .thenReturn(TriggerResult.CONTINUE); - tester.advanceWatermark(new Instant(100)); + tester.advanceInputWatermark(new Instant(100)); List>> output = tester.extractOutput(); assertThat(output.size(), Matchers.equalTo(1)); assertThat(output.get(0), isSingleWindowedValue(Matchers.containsInAnyOrder(1, 10), 1, 1, 20)); - assertThat(output.get(0).getPane(), - Matchers.equalTo(PaneInfo.createPane(true, true, Timing.LATE, 0, 0))); + assertThat( + output.get(0).getPane(), + Matchers.equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0))); } @Test @@ -494,7 +528,7 @@ public void testDropDataMultipleWindows() throws Exception { assertEquals(0, tester.getElementsDroppedDueToLateness()); assertEquals(0, tester.getElementsDroppedDueToClosedWindow()); - tester.advanceWatermark(new Instant(70)); + tester.advanceInputWatermark(new Instant(70)); tester.injectElements( TimestampedValue.of(14, new Instant(60))); // [-30, 70) = closed, [0, 100), [30, 130) @@ -523,16 +557,16 @@ public void testIdempotentEmptyPanes() throws Exception { injectElement(tester, 2, TriggerResult.CONTINUE); when(mockTrigger.onTimer(Mockito..OnTimerContext>any())) .thenReturn(TriggerResult.CONTINUE); - tester.advanceWatermark(new Instant(12)); + tester.advanceInputWatermark(new Instant(12)); when(mockTrigger.onTimer(Mockito..OnTimerContext>any())) .thenReturn(TriggerResult.FIRE); - tester.fireTimer(firstWindow, new Instant(10), TimeDomain.EVENT_TIME); + tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME); // Fire another timer (with no data, so it's an uninteresting pane). when(mockTrigger.onTimer(Mockito..OnTimerContext>any())) - .thenReturn(TriggerResult.FIRE); - tester.fireTimer(firstWindow, new Instant(10), TimeDomain.EVENT_TIME); + .thenReturn(TriggerResult.FIRE); + tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME); // Finish it off with another datum. injectElement(tester, 3, TriggerResult.FIRE_AND_FINISH); @@ -571,16 +605,16 @@ public void testIdempotentEmptyPanesAccumulating() throws Exception { injectElement(tester, 2, TriggerResult.CONTINUE); when(mockTrigger.onTimer(Mockito..OnTimerContext>any())) .thenReturn(TriggerResult.CONTINUE); - tester.advanceWatermark(new Instant(12)); + tester.advanceInputWatermark(new Instant(12)); when(mockTrigger.onTimer(Mockito..OnTimerContext>any())) .thenReturn(TriggerResult.FIRE); - tester.fireTimer(firstWindow, new Instant(10), TimeDomain.EVENT_TIME); + tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME); // Fire another timer (with no data, so it's an uninteresting pane). when(mockTrigger.onTimer(Mockito..OnTimerContext>any())) - .thenReturn(TriggerResult.FIRE); - tester.fireTimer(firstWindow, new Instant(10), TimeDomain.EVENT_TIME); + .thenReturn(TriggerResult.FIRE); + tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME); // Finish it off with another datum. injectElement(tester, 3, TriggerResult.FIRE_AND_FINISH); @@ -591,6 +625,8 @@ public void testIdempotentEmptyPanesAccumulating() throws Exception { // The on-time pane is as expected. assertThat(output.get(0), isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)); + assertThat( + output.get(0).getPane(), equalTo(PaneInfo.createPane(true, false, Timing.ON_TIME, 0, 0))); // The late pane has the correct indices. assertThat(output.get(1).getValue(), containsInAnyOrder(1, 2, 3)); @@ -639,7 +675,7 @@ public void testEmptyOnTimeFromOrFinally() throws Exception { new Sum.SumIntegerFn().asKeyedFn(), VarIntCoder.of(), Duration.millis(100)); - tester.advanceWatermark(new Instant(0)); + tester.advanceInputWatermark(new Instant(0)); tester.advanceProcessingTime(new Instant(0)); tester.injectElements( @@ -648,9 +684,9 @@ public void testEmptyOnTimeFromOrFinally() throws Exception { TimestampedValue.of(1, new Instant(7)), TimestampedValue.of(1, new Instant(5))); - tester.advanceProcessingTime(new Instant(5)); + tester.advanceProcessingTime(new Instant(6)); - tester.advanceWatermark(new Instant(11)); + tester.advanceInputWatermark(new Instant(11)); List> output = tester.extractOutput(); assertEquals(2, output.size()); @@ -684,7 +720,7 @@ public void testProcessingTime() throws Exception { new Sum.SumIntegerFn().asKeyedFn(), VarIntCoder.of(), Duration.millis(100)); - tester.advanceWatermark(new Instant(0)); + tester.advanceInputWatermark(new Instant(0)); tester.advanceProcessingTime(new Instant(0)); tester.injectElements( @@ -692,30 +728,37 @@ public void testProcessingTime() throws Exception { TimestampedValue.of(1, new Instant(3)), TimestampedValue.of(1, new Instant(7)), TimestampedValue.of(1, new Instant(5))); + // 4 elements all at processing time 0 - tester.advanceProcessingTime(new Instant(5)); + tester.advanceProcessingTime(new Instant(6)); // fire [1,3,7,5] since 6 > 0 + 5 tester.injectElements( TimestampedValue.of(1, new Instant(8)), TimestampedValue.of(1, new Instant(4))); + // 6 elements - tester.advanceWatermark(new Instant(11)); + tester.advanceInputWatermark(new Instant(11)); // fire [1,3,7,5,8,4] since 11 > 9 tester.injectElements( TimestampedValue.of(1, new Instant(8)), TimestampedValue.of(1, new Instant(4)), TimestampedValue.of(1, new Instant(5))); + // 9 elements - tester.advanceWatermark(new Instant(12)); + tester.advanceInputWatermark(new Instant(12)); tester.injectElements( TimestampedValue.of(1, new Instant(3))); + // 10 elements tester.advanceProcessingTime(new Instant(15)); tester.injectElements( TimestampedValue.of(1, new Instant(5))); - tester.advanceProcessingTime(new Instant(30)); + // 11 elements + tester.advanceProcessingTime(new Instant(32)); // fire since 32 > 6 + 25 tester.injectElements( TimestampedValue.of(1, new Instant(3))); - tester.advanceWatermark(new Instant(125)); + // 12 elements + // fire [1,3,7,5,8,4,8,4,5,3,5,3] since 125 > 6 + 25 + tester.advanceInputWatermark(new Instant(125)); List> output = tester.extractOutput(); assertEquals(4, output.size()); @@ -755,7 +798,7 @@ public void testMultipleTimerTypes() throws Exception { doAnswer(result) .when(trigger) .onTimer(Mockito..OnTimerContext>any()); - tester.advanceWatermark(new Instant(1000)); + tester.advanceInputWatermark(new Instant(1000)); assertEquals(TriggerResult.FIRE, result.get()); tester.advanceProcessingTime(Instant.now().plus(Duration.millis(10))); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerTester.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerTester.java index f5bef0f3da40f..5d119196086b1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerTester.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerTester.java @@ -49,6 +49,7 @@ import com.google.cloud.dataflow.sdk.values.TimestampedValue; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.base.Function; +import com.google.common.base.MoreObjects; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.FluentIterable; @@ -59,6 +60,8 @@ import org.joda.time.Duration; import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.ArrayList; @@ -68,29 +71,28 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.PriorityQueue; import java.util.Set; +import javax.annotation.Nullable; + /** * Test utility that runs a {@link WindowFn}, {@link Trigger} using in-memory stub implementations * to provide the {@link TimerInternals} and {@link WindowingInternals} needed to run * {@code Trigger}s and {@code ReduceFn}s. * - *

To have all interactions between the trigger and underlying components logged, call - * {@link #logInteractions(boolean)}. - * * @param The element types. * @param The final type for elements in the window (for instance, * {@code Iterable}) * @param The type of windows being used. */ public class TriggerTester { + private static final Logger LOG = LoggerFactory.getLogger(TriggerTester.class); - private Instant watermark = BoundedWindow.TIMESTAMP_MIN_VALUE; - private Instant processingTime = BoundedWindow.TIMESTAMP_MIN_VALUE; - - private final BatchTimerInternals timerInternals = new BatchTimerInternals(processingTime); + private final TestInMemoryStateInternals stateInternals = new TestInMemoryStateInternals(); + private final TestTimerInternals timerInternals = new TestTimerInternals(); private final WindowFn windowFn; - private final StubContexts stubContexts; + private final TestWindowingInternals windowingInternals; private final Coder outputCoder; private final WindowingStrategy objectStrategy; private final ReduceFn reduceFn; @@ -114,25 +116,21 @@ public static TriggerTester public static TriggerTester, W> nonCombining( WindowFn windowFn, TriggerBuilder trigger, AccumulationMode mode, Duration allowedDataLateness) throws Exception { - - WindowingStrategy strategy = WindowingStrategy.of(windowFn) - .withTrigger(trigger.buildTrigger()) - .withMode(mode) - .withAllowedLateness(allowedDataLateness); + WindowingStrategy strategy = + WindowingStrategy.of(windowFn) + .withTrigger(trigger.buildTrigger()) + .withMode(mode) + .withAllowedLateness(allowedDataLateness); return nonCombining(strategy); } - public static - TriggerTester combining( - WindowFn windowFn, Trigger trigger, AccumulationMode mode, - KeyedCombineFn combineFn, - Coder outputCoder, + public static TriggerTester + combining(WindowFn windowFn, Trigger trigger, AccumulationMode mode, + KeyedCombineFn combineFn, Coder outputCoder, Duration allowedDataLateness) throws Exception { - - WindowingStrategy strategy = WindowingStrategy.of(windowFn) - .withTrigger(trigger) - .withMode(mode) - .withAllowedLateness(allowedDataLateness); + WindowingStrategy strategy = + WindowingStrategy.of(windowFn).withTrigger(trigger).withMode(mode).withAllowedLateness( + allowedDataLateness); CoderRegistry registry = new CoderRegistry(); registry.registerStandardCoders(); @@ -142,29 +140,26 @@ TriggerTester combining( return new TriggerTester( strategy, - SystemReduceFn.combining( - StringUtf8Coder.of(), fn).create(KEY), + SystemReduceFn.combining(StringUtf8Coder.of(), fn) + .create(KEY), outputCoder); } - private TriggerTester( - WindowingStrategy wildcardStrategy, - ReduceFn reduceFn, - Coder outputCoder) throws Exception { + private TriggerTester(WindowingStrategy wildcardStrategy, + ReduceFn reduceFn, Coder outputCoder) throws Exception { @SuppressWarnings("unchecked") WindowingStrategy objectStrategy = (WindowingStrategy) wildcardStrategy; this.objectStrategy = objectStrategy; this.reduceFn = reduceFn; this.windowFn = objectStrategy.getWindowFn(); - this.stubContexts = new StubContexts(); + this.windowingInternals = new TestWindowingInternals(); this.outputCoder = outputCoder; executableTrigger = wildcardStrategy.getTrigger(); } ReduceFnRunner createRunner() { - return new ReduceFnRunner<>( - KEY, objectStrategy, timerInternals, stubContexts, + return new ReduceFnRunner<>(KEY, objectStrategy, timerInternals, windowingInternals, droppedDueToClosedWindow, droppedDueToLateness, reduceFn); } @@ -187,11 +182,9 @@ public final void assertHasOnlyGlobalAndFinishedSetsFor(W... expectedWindows) { public final void assertHasOnlyGlobalAndFinishedSetsAndPaneInfoFor(W... expectedWindows) { assertHasOnlyGlobalAndAllowedTags( ImmutableSet.copyOf(expectedWindows), - ImmutableSet.>of( - TriggerRunner.FINISHED_BITS_TAG, - PaneInfoTracker.PANE_INFO_TAG, - WatermarkHold.watermarkHoldTagForOutputTimeFn( - objectStrategy.getOutputTimeFn()))); + ImmutableSet.>of(TriggerRunner.FINISHED_BITS_TAG, PaneInfoTracker.PANE_INFO_TAG, + WatermarkHold.watermarkHoldTagForOutputTimeFn(objectStrategy.getOutputTimeFn()), + WatermarkHold.EXTRA_HOLD_TAG)); } public final void assertHasOnlyGlobalState() { @@ -205,8 +198,8 @@ public final void assertHasOnlyGlobalAndPaneInfoFor(W... expectedWindows) { ImmutableSet.copyOf(expectedWindows), ImmutableSet.>of( PaneInfoTracker.PANE_INFO_TAG, - WatermarkHold.watermarkHoldTagForOutputTimeFn( - objectStrategy.getOutputTimeFn()))); + WatermarkHold.watermarkHoldTagForOutputTimeFn(objectStrategy.getOutputTimeFn()), + WatermarkHold.EXTRA_HOLD_TAG)); } /** @@ -221,11 +214,11 @@ private void assertHasOnlyGlobalAndAllowedTags( } Set actualWindows = new HashSet<>(); - for (StateNamespace namespace : stubContexts.state.getNamespacesInUse()) { + for (StateNamespace namespace : stateInternals.getNamespacesInUse()) { if (namespace instanceof StateNamespaces.GlobalNamespace) { continue; } else if (namespace instanceof StateNamespaces.WindowNamespace) { - Set> tagsInUse = stubContexts.state.getTagsInUse(namespace); + Set> tagsInUse = stateInternals.getTagsInUse(namespace); if (tagsInUse.isEmpty()) { continue; } @@ -237,7 +230,7 @@ private void assertHasOnlyGlobalAndAllowedTags( fail(namespace + " has unexpected states: " + tagsInUse); } } else if (namespace instanceof StateNamespaces.WindowAndTriggerNamespace) { - Set> tagsInUse = stubContexts.state.getTagsInUse(namespace); + Set> tagsInUse = stateInternals.getTagsInUse(namespace); assertTrue(namespace + " contains " + tagsInUse, tagsInUse.isEmpty()); } else { fail("Unrecognized namespace " + namespace); @@ -252,7 +245,11 @@ private StateNamespace windowNamespace(W window) { } public Instant getWatermarkHold() { - return stubContexts.state.minimumWatermarkHold(); + return stateInternals.earliestWatermarkHold(); + } + + public Instant getOutputWatermark() { + return timerInternals.currentOutputWatermarkTime(); } public long getElementsDroppedDueToClosedWindow() { @@ -263,40 +260,42 @@ public long getElementsDroppedDueToLateness() { return droppedDueToLateness.getSum(); } + /** + * How many panes do we have in the output? + */ + public int getOutputSize() { + return windowingInternals.outputs.size(); + } + /** * Retrieve the values that have been output to this time, and clear out the output accumulator. */ public List> extractOutput() { - ImmutableList> result = FluentIterable.from(stubContexts.outputs) - .transform(new Function>, WindowedValue>() { - @Override - public WindowedValue apply(WindowedValue> input) { - return input.withValue(input.getValue().getValue()); - } - }) - .toList(); - stubContexts.outputs.clear(); + ImmutableList> result = + FluentIterable.from(windowingInternals.outputs) + .transform(new Function>, WindowedValue>() { + @Override + public WindowedValue apply(WindowedValue> input) { + return input.withValue(input.getValue().getValue()); + } + }) + .toList(); + windowingInternals.outputs.clear(); return result; } - /** Advance the watermark to the specified time, firing any timers that should fire. */ - public void advanceWatermark(Instant newWatermark) throws Exception { - Preconditions.checkState(!newWatermark.isBefore(watermark), - "Cannot move watermark time backwards from %s to %s", - watermark.getMillis(), newWatermark.getMillis()); - watermark = newWatermark; + /** + * Advance the input watermark to the specified time, firing any timers that should + * fire. Then advance the output watermark as far as possible. + */ + public void advanceInputWatermark(Instant newInputWatermark) throws Exception { ReduceFnRunner runner = createRunner(); - timerInternals.advanceWatermark(runner, newWatermark); + timerInternals.advanceInputWatermark(runner, newInputWatermark); runner.persist(); } /** Advance the processing time to the specified time, firing any timers that should fire. */ - public void advanceProcessingTime( - Instant newProcessingTime) throws Exception { - Preconditions.checkState(!newProcessingTime.isBefore(processingTime), - "Cannot move processing time backwards from %s to %s", - processingTime.getMillis(), newProcessingTime.getMillis()); - processingTime = newProcessingTime; + public void advanceProcessingTime(Instant newProcessingTime) throws Exception { ReduceFnRunner runner = createRunner(); timerInternals.advanceProcessingTime(runner, newProcessingTime); runner.persist(); @@ -308,6 +307,9 @@ public void advanceProcessingTime( */ @SafeVarargs public final void injectElements(TimestampedValue... values) throws Exception { + for (TimestampedValue value : values) { + WindowTracing.trace("TriggerTester.injectElements: {}", value); + } ReduceFnRunner runner = createRunner(); runner.processElements(Iterables.transform( Arrays.asList(values), new Function, WindowedValue>() { @@ -316,7 +318,7 @@ public WindowedValue apply(TimestampedValue input) { try { InputT value = input.getValue(); Instant timestamp = input.getTimestamp(); - Collection windows = windowFn.assignWindows(new TriggerTester.StubAssignContext( + Collection windows = windowFn.assignWindows(new TriggerTester.TestAssignContext( windowFn, value, timestamp, Arrays.asList(GlobalWindow.INSTANCE))); return WindowedValue.of(value, timestamp, windows, PaneInfo.NO_FIRING); } catch (Exception e) { @@ -331,14 +333,16 @@ public WindowedValue apply(TimestampedValue input) { public void fireTimer(W window, Instant timestamp, TimeDomain domain) { ReduceFnRunner runner = createRunner(); - runner.onTimer(TimerData.of( - StateNamespaces.window(windowFn.windowCoder(), window), timestamp, domain)); + runner.onTimer( + TimerData.of(StateNamespaces.window(windowFn.windowCoder(), window), timestamp, domain)); runner.persist(); } - private static class TestingInMemoryStateInternals extends InMemoryStateInternals { - - protected Set> getTagsInUse(StateNamespace namespace) { + /** + * Simulate state. + */ + private static class TestInMemoryStateInternals extends InMemoryStateInternals { + public Set> getTagsInUse(StateNamespace namespace) { Set> inUse = new HashSet<>(); for (Map.Entry, State> entry : inMemoryState.getTagsInUse(namespace).entrySet()) { if (!isEmptyForTesting(entry.getValue())) { @@ -352,7 +356,8 @@ public Set getNamespacesInUse() { return inMemoryState.getNamespacesInUse(); } - public Instant minimumWatermarkHold() { + /** Return the earliest output watermark hold in state, or null if none. */ + public Instant earliestWatermarkHold() { Instant minimum = null; for (State storage : inMemoryState.values()) { if (storage instanceof WatermarkStateInternal) { @@ -366,10 +371,11 @@ public Instant minimumWatermarkHold() { } } - private class StubContexts implements WindowingInternals> { - - private TestingInMemoryStateInternals state = new TestingInMemoryStateInternals(); - + /** + * Convey the simulated state and implement {@link #outputWindowedValue} to capture all output + * elements. + */ + private class TestWindowingInternals implements WindowingInternals> { private List>> outputs = new ArrayList<>(); @Override @@ -385,7 +391,7 @@ public void outputWindowedValue(KV output, Instant timestamp, @Override public TimerInternals timerInternals() { throw new UnsupportedOperationException( - "getTimerInternals() should not be called on StubContexts."); + "Testing triggers should not use timers from WindowingInternals."); } @Override @@ -399,27 +405,28 @@ public PaneInfo pane() { throw new UnsupportedOperationException( "Testing triggers should not use pane from WindowingInternals."); } + @Override - public void writePCollectionViewData(TupleTag tag, Iterable> data, - Coder elemCoder) throws IOException { + public void writePCollectionViewData( + TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { throw new UnsupportedOperationException( "Testing triggers should not use writePCollectionViewData from WindowingInternals."); } @Override public StateInternals stateInternals() { - return state; + return stateInternals; } } - private static class StubAssignContext + private static class TestAssignContext extends WindowFn.AssignContext { private Object element; private Instant timestamp; private Collection windows; - public StubAssignContext(WindowFn windowFn, - Object element, Instant timestamp, Collection windows) { + public TestAssignContext(WindowFn windowFn, Object element, Instant timestamp, + Collection windows) { windowFn.super(); this.element = element; this.timestamp = timestamp; @@ -443,7 +450,6 @@ public Collection windows() { } private static class InMemoryLongSumAggregator implements Aggregator { - private final String name; private long sum = 0; @@ -470,4 +476,148 @@ public long getSum() { return sum; } } + + /** + * Simulate the firing of timers and progression of input and output watermarks for a + * single computation and key in a Windmill-like streaming environment. Similar to + * {@link BatchTimerInternals}, but also tracks the output watermark. + */ + private class TestTimerInternals implements TimerInternals { + /** At most one timer per timestamp is kept. */ + private Set existingTimers = new HashSet<>(); + + /** Pending input watermark timers, in timestamp order. */ + private PriorityQueue watermarkTimers = new PriorityQueue<>(11); + + /** Pending processing time timers, in timestamp order. */ + private PriorityQueue processingTimers = new PriorityQueue<>(11); + + /** Current input watermark. */ + @Nullable + private Instant inputWatermarkTime = null; + + /** Current output watermark. */ + @Nullable + private Instant outputWatermarkTime = null; + + /** Current processing time. */ + private Instant processingTime = BoundedWindow.TIMESTAMP_MIN_VALUE; + + private PriorityQueue queue(TimeDomain domain) { + return TimeDomain.EVENT_TIME.equals(domain) ? watermarkTimers : processingTimers; + } + + @Override + public void setTimer(TimerData timer) { + WindowTracing.trace("TestTimerInternals.setTimer: {}", timer); + if (existingTimers.add(timer)) { + queue(timer.getDomain()).add(timer); + } + } + + @Override + public void deleteTimer(TimerData timer) { + WindowTracing.trace("TestTimerInternals.deleteTimer: {}", timer); + existingTimers.remove(timer); + queue(timer.getDomain()).remove(timer); + } + + @Override + public Instant currentProcessingTime() { + return processingTime; + } + + @Override + @Nullable + public Instant currentInputWatermarkTime() { + return inputWatermarkTime; + } + + @Override + @Nullable + public Instant currentOutputWatermarkTime() { + return outputWatermarkTime; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(getClass()) + .add("watermarkTimers", watermarkTimers) + .add("processingTimers", processingTimers) + .add("inputWatermarkTime", inputWatermarkTime) + .add("outputWatermarkTime", outputWatermarkTime) + .add("processingTime", processingTime) + .toString(); + } + + public void advanceInputWatermark( + ReduceFnRunner runner, Instant newInputWatermark) { + Preconditions.checkNotNull(newInputWatermark); + Preconditions.checkState( + inputWatermarkTime == null || !newInputWatermark.isBefore(inputWatermarkTime), + "Cannot move input watermark time backwards from %s to %s", inputWatermarkTime, + newInputWatermark); + WindowTracing.trace("TestTimerInternals.advanceInputWatermark: from {} to {}", + inputWatermarkTime, newInputWatermark); + inputWatermarkTime = newInputWatermark; + advanceAndFire(runner, newInputWatermark, TimeDomain.EVENT_TIME); + + Instant hold = stateInternals.earliestWatermarkHold(); + if (hold == null) { + WindowTracing.trace("TestTimerInternals.advanceInputWatermark: no holds, " + + "so output watermark = input watermark"); + hold = inputWatermarkTime; + } + advanceOutputWatermark(hold); + } + + private void advanceOutputWatermark(Instant newOutputWatermark) { + Preconditions.checkNotNull(newOutputWatermark); + Preconditions.checkNotNull(inputWatermarkTime); + if (newOutputWatermark.isAfter(inputWatermarkTime)) { + WindowTracing.trace( + "TestTimerInternals.advanceOutputWatermark: clipping output watermark from {} to {}", + newOutputWatermark, inputWatermarkTime); + newOutputWatermark = inputWatermarkTime; + } + Preconditions.checkState( + outputWatermarkTime == null || !newOutputWatermark.isBefore(outputWatermarkTime), + "Cannot move output watermark time backwards from %s to %s", outputWatermarkTime, + newOutputWatermark); + WindowTracing.trace("TestTimerInternals.advanceOutputWatermark: from {} to {}", + outputWatermarkTime, newOutputWatermark); + outputWatermarkTime = newOutputWatermark; + } + + public void advanceProcessingTime( + ReduceFnRunner runner, Instant newProcessingTime) { + Preconditions.checkState(!newProcessingTime.isBefore(processingTime), + "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime); + WindowTracing.trace("TestTimerInternals.advanceProcessingTime: from {} to {}", processingTime, + newProcessingTime); + processingTime = newProcessingTime; + advanceAndFire(runner, newProcessingTime, TimeDomain.PROCESSING_TIME); + } + + private void advanceAndFire( + ReduceFnRunner runner, Instant currentTime, TimeDomain domain) { + PriorityQueue queue = queue(domain); + boolean shouldFire = false; + + do { + TimerData timer = queue.peek(); + // Timers fire when the current time progresses past the timer time. + shouldFire = timer != null && currentTime.isAfter(timer.getTimestamp()); + if (shouldFire) { + WindowTracing.trace( + "TestTimerInternals.advanceAndFire: firing {} at {}", timer, currentTime); + // Remove before firing, so that if the trigger adds another identical + // timer we don't remove it. + queue.remove(); + + runner.onTimer(timer); + } + } while (shouldFire); + } + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/StateTagTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/StateTagTest.java index 9b0fed7d11b51..fa3c82634a943 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/StateTagTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/StateTagTest.java @@ -72,9 +72,12 @@ public void testWatermarkBagEquality() { StateTag bar2 = StateTags.watermarkStateInternal( "bar", OutputTimeFns.outputAtLatestInputTimestamp()); + // Same id, same fn. assertEquals(foo1, foo2); + // Different id, same fn. assertNotEquals(foo1, bar); - assertNotEquals(bar, bar2); + // Same id, different fn. + assertEquals(bar, bar2); } @Test From 79b427495958a6ea435f818a1c568bce26e2443e Mon Sep 17 00:00:00 2001 From: markshields Date: Wed, 9 Dec 2015 10:50:41 -0800 Subject: [PATCH 1219/1541] Use 8MB page sizes to match minimums used elsewhere Eg: GCS recommends 8MB minimum buffer, so if you're streaming data from a GBK Iterable into GCS seems best to have our buffer at least in the same ballpack. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109809534 --- .../dataflow/sdk/runners/worker/WindmillStateReader.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateReader.java index 88390e2b37664..fb0421a703757 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateReader.java @@ -30,8 +30,6 @@ import com.google.protobuf.ByteString; import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.io.InputStream; @@ -60,7 +58,7 @@ class WindmillStateReader { * Ideal maximum bytes in a TagList response. However, Windmill will always return * at least one value if possible irrespective of this limit. */ - public static final long MAX_LIST_BYTES = 1L << 20; // 1MB + public static final long MAX_LIST_BYTES = 8L << 20; // 8MB /** * When combined with a key and computationId, represents the unique address for @@ -144,8 +142,6 @@ public ValuesAndContToken(List values, @Nullable ByteString continuationToken } } - private static final Logger LOG = LoggerFactory.getLogger(WindmillStateReader.class); - private final String computation; private final ByteString key; private final long workToken; @@ -594,7 +590,6 @@ private TagListPagingIterable(List firstPage, StateTag secondPageCont, Coder< @Override public Iterator iterator() { return new AbstractIterator() { - private int numPagesRead = 1; private Iterator currentPage = firstPage.iterator(); private StateTag nextPageCont = secondPageCont; private Future> pendingNextPage = @@ -613,7 +608,6 @@ protected T computeNext() { ValuesAndContToken valuesAndContToken; try { valuesAndContToken = pendingNextPage.get(); - numPagesRead++; } catch (InterruptedException | ExecutionException e) { throw new RuntimeException("Unable to read value from state", e); } From 40e53d701722285d0e702291995f5842d252cfa5 Mon Sep 17 00:00:00 2001 From: kirpichov Date: Wed, 9 Dec 2015 15:17:51 -0800 Subject: [PATCH 1220/1541] Updates google-api-services-dataflow dependency ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109838205 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 099352d7cf0b9..734eaf8775291 100644 --- a/pom.xml +++ b/pom.xml @@ -69,7 +69,7 @@ 1.7.7 v2-rev248-1.21.0 - v1b3-rev12-1.21.0 + v1b3-rev13-1.21.0 v1beta2-rev1-3.0.2 1.21.0 18.0 From 1afaf46c93b5d2e654a4d44ee94260e53ad374a1 Mon Sep 17 00:00:00 2001 From: klk Date: Wed, 9 Dec 2015 19:23:26 -0800 Subject: [PATCH 1221/1541] Return a future for WindmillState.persist() Previously, WindmillState.persist() was a purely local operation. Recent additions made it necessary to make remote calls, which could be serialized in unfortunate ways. This change makes the need for those calls to be asynchronous explicit, by making them into Futures. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109857899 --- .../worker/WindmillStateInternals.java | 111 +++++++++++++----- 1 file changed, 81 insertions(+), 30 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java index 9f79ad6463839..63a0256e072da 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java @@ -17,6 +17,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; +import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill.WorkItemCommitRequest; import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; @@ -34,6 +35,7 @@ import com.google.cloud.dataflow.sdk.util.state.ValueState; import com.google.cloud.dataflow.sdk.util.state.WatermarkStateInternal; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.Futures; @@ -146,6 +148,8 @@ public WindmillStateInternals(String prefix, boolean useStateFamilies, } public void persist(final Windmill.WorkItemCommitRequest.Builder commitBuilder) { + List> commitsToMerge = new ArrayList<>(); + // Call persist on each first, which may schedule some futures for reading. for (State location : inMemoryState.values()) { if (!(location instanceof WindmillState)) { @@ -156,7 +160,7 @@ public void persist(final Windmill.WorkItemCommitRequest.Builder commitBuilder) } try { - ((WindmillState) location).persist(commitBuilder); + commitsToMerge.add(((WindmillState) location).persist()); } catch (IOException e) { throw new RuntimeException("Unable to persist state", e); } @@ -168,6 +172,14 @@ public void persist(final Windmill.WorkItemCommitRequest.Builder commitBuilder) // Clear out the map of already retrieved state instances. inMemoryState.clear(); + + try { + for (Future commitFuture : commitsToMerge) { + commitBuilder.mergeFrom(commitFuture.get()); + } + } catch (ExecutionException | InterruptedException exc) { + throw new RuntimeException("Failed to retrieve Windmill state during persist()", exc); + } } private ByteString encodeKey(StateNamespace namespace, StateTag address) { @@ -185,8 +197,34 @@ private ByteString encodeKey(StateNamespace namespace, StateTag address) { } } + /** + * Anything that can provide a {@link WorkItemCommitRequest} to persist its state; it may need + * to read some state in order to build the commit request. + */ private interface WindmillState { - void persist(Windmill.WorkItemCommitRequest.Builder commitBuilder) throws IOException; + /** + * Return an asynchronously computed {@link WorkItemCommitRequest}. The request should + * be of a form that can be merged with others (only add to repeated fields). + */ + Future persist() + throws IOException; + } + + /** + * Base class for implementations of {@link WindmillState} where the {@link #persist} call does + * not require any asynchronous reading. + */ + private abstract static class SimpleWindmillState implements WindmillState { + @Override + public final Future persist() throws IOException{ + return Futures.immediateFuture(persistDirectly()); + } + + /** + * Returns a {@link WorkItemCommitRequest} that can be used to persist this state to + * Windmill. + */ + protected abstract WorkItemCommitRequest persistDirectly() throws IOException; } @Override @@ -194,7 +232,8 @@ public T state(StateNamespace namespace, StateTag address) return inMemoryState.get(namespace, address); } - private static class WindmillValue implements ValueState, WindmillState { + private static class WindmillValue extends SimpleWindmillState + implements ValueState, WindmillState { private final ByteString stateKey; private final String stateFamily; @@ -244,11 +283,10 @@ public void set(T value) { } @Override - public void persist( - Windmill.WorkItemCommitRequest.Builder commitBuilder) throws IOException { + protected WorkItemCommitRequest persistDirectly() throws IOException { if (!modified) { // No in-memory changes. - return; + return WorkItemCommitRequest.newBuilder().buildPartial(); } // We can't write without doing a read, so we need to kick off a read if we get here. @@ -261,6 +299,7 @@ public void persist( coder.encode(modifiedValue, stream, Coder.Context.OUTER); } + WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); commitBuilder .addValueUpdatesBuilder() .setTag(stateKey) @@ -268,10 +307,12 @@ public void persist( .getValueBuilder() .setData(stream.toByteString()) .setTimestamp(Long.MAX_VALUE); + return commitBuilder.buildPartial(); } } - private static class WindmillBag implements BagState, WindmillState { + private static class WindmillBag extends SimpleWindmillState + implements BagState, WindmillState { private final ByteString stateKey; private final String stateFamily; @@ -351,7 +392,9 @@ public void add(T input) { } @Override - public void persist(Windmill.WorkItemCommitRequest.Builder commitBuilder) throws IOException { + public WorkItemCommitRequest persistDirectly() throws IOException { + WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); + if (cleared) { // If we do a delete, we need to have done a read to prevent Windmill complaining about // blind deletes. We use the underlying reader, because we normally skip the actual read @@ -363,7 +406,6 @@ public void persist(Windmill.WorkItemCommitRequest.Builder commitBuilder) throws .setEndTimestamp(Long.MAX_VALUE); } - if (!localAdditions.isEmpty()) { byte[] zero = {0x0}; Windmill.TagList.Builder listUpdatesBuilder = @@ -382,6 +424,7 @@ public void persist(Windmill.WorkItemCommitRequest.Builder commitBuilder) throws .setTimestamp(Long.MAX_VALUE); } } + return commitBuilder.buildPartial(); } } @@ -478,27 +521,30 @@ public void add(Instant outputTime) { } @Override - public void persist(Windmill.WorkItemCommitRequest.Builder commitBuilder) { + public Future persist() { if (!cleared && localAdditions == null) { // Nothing to do - return; + return Futures.immediateFuture(WorkItemCommitRequest.newBuilder().buildPartial()); } else if (cleared && localAdditions == null) { // Just clearing the persisted state; blind delete + WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); commitBuilder.addWatermarkHoldsBuilder() .setTag(stateKey) .setStateFamily(stateFamily) .setReset(true); - + return Futures.immediateFuture(commitBuilder.buildPartial()); } else if (cleared && localAdditions != null) { // Since we cleared before adding, we can do a blind overwrite of persisted state + WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); commitBuilder.addWatermarkHoldsBuilder() .setTag(stateKey) .setStateFamily(stateFamily) .setReset(true) .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); + return Futures.immediateFuture(commitBuilder.buildPartial()); } else if (!cleared && localAdditions != null){ // Otherwise, we need to combine the local additions with the already persisted data - combineWithPersisted(commitBuilder); + return combineWithPersisted(); } else { throw new IllegalStateException("Unreachable condition"); } @@ -508,7 +554,7 @@ public void persist(Windmill.WorkItemCommitRequest.Builder commitBuilder) { * Combines local additions with persisted data and mutates the {@code commitBuilder} * to write the result. */ - private void combineWithPersisted(Windmill.WorkItemCommitRequest.Builder commitBuilder) { + private Future combineWithPersisted() { boolean windmillCanCombine = false; // If the combined output time depends only on the window, then we are just blindly adding @@ -525,29 +571,34 @@ private void combineWithPersisted(Windmill.WorkItemCommitRequest.Builder commitB if (windmillCanCombine) { // We do a blind write and let Windmill take the MIN + WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); commitBuilder.addWatermarkHoldsBuilder() .setTag(stateKey) .setStateFamily(stateFamily) .addTimestamps( WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); + return Futures.immediateFuture(commitBuilder.buildPartial()); } else { // The non-fast path does a read-modify-write - Instant priorHold; - try { - priorHold = reader.watermarkFuture(stateKey, stateFamily).get(); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException("Unable to read state", e); - } + return Futures.lazyTransform(reader.watermarkFuture(stateKey, stateFamily), + new Function() { - Instant combinedHold = (priorHold == null) ? localAdditions - : outputTimeFn.combine(priorHold, localAdditions); + @Override + public WorkItemCommitRequest apply(Instant priorHold) { - commitBuilder.addWatermarkHoldsBuilder() - .setTag(stateKey) - .setStateFamily(stateFamily) - .setReset(true) - .addTimestamps( - WindmillTimeUtils.harnessToWindmillTimestamp(combinedHold)); + Instant combinedHold = (priorHold == null) ? localAdditions + : outputTimeFn.combine(priorHold, localAdditions); + + WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); + commitBuilder.addWatermarkHoldsBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .setReset(true) + .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(combinedHold)); + + return commitBuilder.buildPartial(); + } + }); } } } @@ -599,7 +650,7 @@ public void clear() { } @Override - public void persist(Windmill.WorkItemCommitRequest.Builder commitBuilder) throws IOException { + public Future persist() throws IOException { if (hasLocalAdditions) { // TODO: Take into account whether it's in the cache. if (COMPACT_NOW.get().get()) { @@ -610,7 +661,7 @@ public void persist(Windmill.WorkItemCommitRequest.Builder commitBuilder) throws localAdditionsAccum = combineFn.createAccumulator(); hasLocalAdditions = false; } - bag.persist(commitBuilder); + return bag.persist(); } @Override From 8dff8fb53c1ea14a1e4458946fee8772c20c9cdb Mon Sep 17 00:00:00 2001 From: robertwb Date: Thu, 10 Dec 2015 10:43:22 -0800 Subject: [PATCH 1222/1541] Minor javadoc updates. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109913802 --- .../cloud/dataflow/sdk/transforms/Create.java | 40 +++++++++++-------- .../dataflow/sdk/transforms/Flatten.java | 4 +- .../dataflow/sdk/transforms/GroupByKey.java | 19 +++++---- 3 files changed, 37 insertions(+), 26 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java index 556f3010d13ef..a74e5bff7f65a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Create.java @@ -94,8 +94,9 @@ public class Create { * elements. * *

By default, {@code Create.Values} can automatically determine the {@code Coder} to use - * if all elements have the same run-time class, and a default coder is registered for that - * class. See {@link CoderRegistry} for details on how defaults are determined. + * if all elements have the same non-parameterized run-time class, and a default coder is + * registered for that class. See {@link CoderRegistry} for details on how defaults are + * determined. * Otherwise, use {@link Create.Values#withCoder} to set the coder explicitly. */ public static Values of(Iterable elems) { @@ -110,11 +111,12 @@ public static Values of(Iterable elems) { * {@link Create#timestamped} for a way of creating a {@code PCollection} * with timestamped elements. * - *

The argument should not be modified after this is called. + *

The arguments should not be modified after this is called. * *

By default, {@code Create.Values} can automatically determine the {@code Coder} to use - * if all elements have the same run-time class, and a default coder is registered for that - * class. See {@link CoderRegistry} for details on how defaults are determined. + * if all elements have the same non-parameterized run-time class, and a default coder is + * registered for that class. See {@link CoderRegistry} for details on how defaults are + * determined. * Otherwise, use {@link Create.Values#withCoder} to set the coder explicitly. */ @SafeVarargs @@ -132,8 +134,9 @@ public static Values of(T... elems) { * with timestamped elements. * *

By default, {@code Create.Values} can automatically determine the {@code Coder} to use - * if all elements have the same run-time class, and a default coder is registered for that - * class. See {@link CoderRegistry} for details on how defaults are determined. + * if all elements have the same non-parameterized run-time class, and a default coder is + * registered for that class. See {@link CoderRegistry} for details on how defaults are + * determined. * Otherwise, use {@link Create.Values#withCoder} to set the coder explicitly. */ public static Values> of(Map elems) { @@ -152,8 +155,9 @@ public static Values> of(Map elems) { *

The argument should not be modified after this is called. * *

By default, {@code Create.TimestampedValues} can automatically determine the {@code Coder} - * to use if all elements have the same run-time class, and a default coder is registered for - * that class. See {@link CoderRegistry} for details on how defaults are determined. + * to use if all elements have the same non-parameterized run-time class, and a default coder is + * registered for that class. See {@link CoderRegistry} for details on how defaults are + * determined. * Otherwise, use {@link Create.TimestampedValues#withCoder} to set the coder explicitly. */ public static TimestampedValues timestamped(Iterable> elems) { @@ -164,7 +168,7 @@ public static TimestampedValues timestamped(Iterable> * Returns a new {@link Create.TimestampedValues} transform that produces a {@link PCollection} * containing the specified elements with the specified timestamps. * - *

The argument should not be modified after this is called. + *

The arguments should not be modified after this is called. */ @SafeVarargs public static TimestampedValues timestamped( @@ -179,8 +183,9 @@ public static TimestampedValues timestamped( *

The arguments should not be modified after this is called. * *

By default, {@code Create.TimestampedValues} can automatically determine the {@code Coder} - * to use if all elements have the same run-time class, and a default coder is registered for - * that class. See {@link CoderRegistry} for details on how defaults are determined. + * to use if all elements have the same non-parameterized run-time class, and a default coder + * is registered for that class. See {@link CoderRegistry} for details on how defaults are + * determined. * Otherwise, use {@link Create.TimestampedValues#withCoder} to set the coder explicitly. * @throws IllegalArgumentException if there are a different number of values @@ -212,8 +217,9 @@ public static class Values extends PTransform> { * value of type {@code T}. * *

By default, {@code Create.Values} can automatically determine the {@code Coder} to use - * if all elements have the same run-time class, and a default coder is registered for that - * class. See {@link CoderRegistry} for details on how defaults are determined. + * if all elements have the same non-parameterized run-time class, and a default coder is + * registered for that class. See {@link CoderRegistry} for details on how defaults are + * determined. * *

Note that for {@link Create.Values} with no elements, the {@link VoidCoder} is used. */ @@ -327,9 +333,9 @@ public static class TimestampedValues extends Values { * value of type {@code T}. * *

By default, {@code Create.TimestampedValues} can automatically determine the - * {@code Coder} to use if all elements have the same run-time class, and a default coder is - * registered for that class. See {@link CoderRegistry} for details on how defaults are - * determined. + * {@code Coder} to use if all elements have the same non-parameterized run-time class, + * and a default coder is registered for that class. See {@link CoderRegistry} for details + * on how defaults are determined. * *

Note that for {@link Create.TimestampedValues with no elements}, the {@link VoidCoder} * is used. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java index 2e1a3ab04f85e..de6add0ea3c64 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Flatten.java @@ -58,8 +58,7 @@ public class Flatten { * into a {@link PCollection} containing all the elements of all * the {@link PCollection}s in its input. * - *

If any of the inputs to {@code Flatten} require window merging, - * all inputs must have equal {@link WindowFn}s. + *

All inputs must have equal {@link WindowFn}s. * The output elements of {@code Flatten} are in the same windows and * have the same timestamps as their corresponding input elements. The output * {@code PCollection} will have the same @@ -98,6 +97,7 @@ public static FlattenIterables iterables() { * A {@link PTransform} that flattens a {@link PCollectionList} * into a {@link PCollection} containing all the elements of all * the {@link PCollection}s in its input. + * Implements {@link #pCollections}. * * @param the type of the elements in the input and output * {@code PCollection}s. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index ca125545d90d9..6c9defadfe55b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -28,6 +28,7 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn; import com.google.cloud.dataflow.sdk.util.ReifyTimestampAndWindowsDoFn; @@ -53,8 +54,9 @@ * {@code PCollection>>} representing a map from * each distinct key and window of the input {@code PCollection} to an * {@code Iterable} over all the values associated with that key in - * the input. Each key in the output {@code PCollection} is unique within - * each window. + * the input per window. Absent repeatedly-firing + * {@link Window#triggering triggering}, each key in the output + * {@code PCollection} is unique within each window. * *

{@code GroupByKey} is analogous to converting a multi-map into * a uni-map, and related to {@code GROUP BY} in SQL. It corresponds @@ -68,7 +70,7 @@ * encoded bytes. This admits efficient parallel evaluation. Note that * this requires that the {@code Coder} of the keys be deterministic (see * {@link Coder#verifyDeterministic()}). If the key {@code Coder} is not - * deterministic, an exception is thrown at runtime. + * deterministic, an exception is thrown at pipeline construction time. * *

By default, the {@code Coder} of the keys of the output * {@code PCollection} is the same as that of the keys of the input, @@ -109,18 +111,21 @@ * {@link com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark} * for details on the estimation. * - *

The timestamp for each emitted pane is the earliest event time among all elements in - * the pane. The output {@code PCollection} will have the same {@link WindowFn} + *

The timestamp for each emitted pane is determined by the + * {@link Window.Bound#withOutputTimeFn windowing operation}. + * The output {@code PCollection} will have the same {@link WindowFn} * as the input. * *

If the input {@code PCollection} contains late data (see * {@link com.google.cloud.dataflow.sdk.io.PubsubIO.Read.Bound#timestampLabel} - * for an example of how this can occur), then there may be multiple elements + * for an example of how this can occur) or the + * {@link Window#triggering requested TriggerFn} can fire before + * the watermark, then there may be multiple elements * output by a {@code GroupByKey} that correspond to the same key and window. * *

If the {@link WindowFn} of the input requires merging, it is not * valid to apply another {@code GroupByKey} without first applying a new - * {@link WindowFn}. + * {@link WindowFn} or applying {@link Window#remerge()}. * * @param the type of the keys of the input and output * {@code PCollection}s From bceb7d60204b60736b18ebcd456a9cdd60176a06 Mon Sep 17 00:00:00 2001 From: dhalperi Date: Thu, 10 Dec 2015 16:42:16 -0800 Subject: [PATCH 1223/1541] CombineTest: remove unused import, suppress a Java warning ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109948609 --- .../com/google/cloud/dataflow/sdk/transforms/CombineTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java index 987b97e9ae248..99193a9ac6100 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java @@ -46,7 +46,6 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.transforms.windowing.Window.ClosingBehavior; import com.google.cloud.dataflow.sdk.util.PropertyNames; -import com.google.cloud.dataflow.sdk.util.common.Counter; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -562,6 +561,7 @@ public static SetCoder of( return of((Coder) components.get(0)); } + @SuppressWarnings("unused") // required for coder instantiation public static List getInstanceComponents(Set exampleValue) { return IterableCoder.getInstanceComponents(exampleValue); } From 8746470d941e8f8f3ee81fbaba338451ca8f871f Mon Sep 17 00:00:00 2001 From: klk Date: Thu, 10 Dec 2015 16:50:07 -0800 Subject: [PATCH 1224/1541] Rename TriggerTester to ReduceFnTester This name was a legacy holdover. The TriggerTester actually tests the whole stack from ReduceFnRunner down. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109949189 --- .../sdk/util/TriggerContextFactory.java | 9 +- .../transforms/windowing/AfterAllTest.java | 10 +- .../transforms/windowing/AfterEachTest.java | 10 +- .../transforms/windowing/AfterFirstTest.java | 10 +- .../transforms/windowing/AfterPaneTest.java | 8 +- .../windowing/AfterProcessingTimeTest.java | 8 +- .../AfterSynchronizedProcessingTimeTest.java | 10 +- .../windowing/AfterWatermarkTest.java | 28 +- .../windowing/DefaultTriggerTest.java | 10 +- .../windowing/OrFinallyTriggerTest.java | 10 +- .../transforms/windowing/RepeatedlyTest.java | 6 +- ...TriggerTester.java => ReduceFnTester.java} | 33 +- .../sdk/util/TriggerExecutorTest.java | 409 +++++++++--------- 13 files changed, 284 insertions(+), 277 deletions(-) rename sdk/src/test/java/com/google/cloud/dataflow/sdk/util/{TriggerTester.java => ReduceFnTester.java} (95%) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerContextFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerContextFactory.java index 9e574c27e9151..7627c5c3cf505 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerContextFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerContextFactory.java @@ -40,16 +40,17 @@ import java.util.Map; /** - * Factory for creating instances of the various {@link Trigger} contexts. + * Factory for creating instances of the various {@link Trigger} contexts from the corresponding + * {@link ReduceFn} contexts. */ -class TriggerContextFactory { +public class TriggerContextFactory { private final WindowingStrategy windowingStrategy; private StateInternals stateInternals; private ActiveWindowSet activeWindows; - TriggerContextFactory(WindowingStrategy windowingStrategy, StateInternals stateInternals, - ActiveWindowSet activeWindows) { + public TriggerContextFactory(WindowingStrategy windowingStrategy, + StateInternals stateInternals, ActiveWindowSet activeWindows) { this.windowingStrategy = windowingStrategy; this.stateInternals = stateInternals; this.activeWindows = activeWindows; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java index e4855fb1f7084..d34e788c336c9 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java @@ -27,8 +27,8 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.MergeResult; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.TriggerResult; +import com.google.cloud.dataflow.sdk.util.ReduceFnTester; import com.google.cloud.dataflow.sdk.util.TimeDomain; -import com.google.cloud.dataflow.sdk.util.TriggerTester; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; import com.google.cloud.dataflow.sdk.values.TimestampedValue; @@ -50,12 +50,12 @@ public class AfterAllTest { @Mock private OnceTrigger mockTrigger1; @Mock private OnceTrigger mockTrigger2; - private TriggerTester, IntervalWindow> tester; + private ReduceFnTester, IntervalWindow> tester; private IntervalWindow firstWindow; public void setUp(WindowFn windowFn) throws Exception { MockitoAnnotations.initMocks(this); - tester = TriggerTester.nonCombining( + tester = ReduceFnTester.nonCombining( windowFn, AfterAll.of(mockTrigger1, mockTrigger2), AccumulationMode.DISCARDING_FIRED_PANES, @@ -219,7 +219,7 @@ public void testFireDeadline() throws Exception { @Test public void testAfterAllRealTriggersFixedWindow() throws Exception { - tester = TriggerTester.nonCombining(FixedWindows.of(Duration.millis(50)), + tester = ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(50)), Repeatedly.forever( AfterAll.of( AfterPane.elementCountAtLeast(5), @@ -267,7 +267,7 @@ public void testAfterAllRealTriggersFixedWindow() throws Exception { @Test public void testAfterAllMergingWindowSomeFinished() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(windowDuration), AfterAll.of( AfterProcessingTime.pastFirstElementInPane() diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEachTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEachTest.java index 93084bfe0fde0..d28515999cbf8 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEachTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEachTest.java @@ -28,8 +28,8 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnTimerContext; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.TriggerResult; +import com.google.cloud.dataflow.sdk.util.ReduceFnTester; import com.google.cloud.dataflow.sdk.util.TimeDomain; -import com.google.cloud.dataflow.sdk.util.TriggerTester; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; import com.google.cloud.dataflow.sdk.values.TimestampedValue; @@ -52,12 +52,12 @@ public class AfterEachTest { @Mock private Trigger mockTrigger1; @Mock private Trigger mockTrigger2; - private TriggerTester, IntervalWindow> tester; + private ReduceFnTester, IntervalWindow> tester; private IntervalWindow firstWindow; public void setUp(WindowFn windowFn) throws Exception { MockitoAnnotations.initMocks(this); - tester = TriggerTester.nonCombining( + tester = ReduceFnTester.nonCombining( windowFn, AfterEach.inOrder(mockTrigger1, mockTrigger2), AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); @@ -219,7 +219,7 @@ public void testFireDeadline() throws Exception { @Test public void testSequenceRealTriggersFixedWindow() throws Exception { - tester = TriggerTester.nonCombining(FixedWindows.of(Duration.millis(50)), + tester = ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(50)), AfterEach.inOrder( AfterPane.elementCountAtLeast(5), AfterPane.elementCountAtLeast(5), @@ -249,7 +249,7 @@ public void testSequenceRealTriggersFixedWindow() throws Exception { @Test public void testAfterEachMergingWindowSomeFinished() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(windowDuration), AfterEach.inOrder( AfterProcessingTime.pastFirstElementInPane() diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirstTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirstTest.java index 63a240a466a27..11c34771b4a85 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirstTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirstTest.java @@ -27,8 +27,8 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.MergeResult; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.TriggerResult; +import com.google.cloud.dataflow.sdk.util.ReduceFnTester; import com.google.cloud.dataflow.sdk.util.TimeDomain; -import com.google.cloud.dataflow.sdk.util.TriggerTester; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; import com.google.cloud.dataflow.sdk.values.TimestampedValue; @@ -51,12 +51,12 @@ public class AfterFirstTest { @Mock private OnceTrigger mockTrigger1; @Mock private OnceTrigger mockTrigger2; - private TriggerTester, IntervalWindow> tester; + private ReduceFnTester, IntervalWindow> tester; private IntervalWindow firstWindow; public void setUp(WindowFn windowFn) throws Exception { MockitoAnnotations.initMocks(this); - tester = TriggerTester.nonCombining( + tester = ReduceFnTester.nonCombining( windowFn, AfterFirst.of(mockTrigger1, mockTrigger2), AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); @@ -206,7 +206,7 @@ public void testFireDeadline() throws Exception { @Test public void testAfterFirstRealTriggersFixedWindow() throws Exception { - tester = TriggerTester.nonCombining(FixedWindows.of(Duration.millis(50)), + tester = ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(50)), Repeatedly.forever( AfterFirst.of( AfterPane.elementCountAtLeast(5), @@ -261,7 +261,7 @@ public void testAfterFirstRealTriggersFixedWindow() throws Exception { @Test public void testAfterFirstMergingWindowSomeFinished() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(windowDuration), AfterFirst.of( AfterProcessingTime.pastFirstElementInPane() diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterPaneTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterPaneTest.java index 9da3b4bde8826..9f094d916b91e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterPaneTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterPaneTest.java @@ -24,7 +24,7 @@ import com.google.cloud.dataflow.sdk.WindowMatchers; import com.google.cloud.dataflow.sdk.coders.VarIntCoder; import com.google.cloud.dataflow.sdk.transforms.Sum.SumIntegerFn; -import com.google.cloud.dataflow.sdk.util.TriggerTester; +import com.google.cloud.dataflow.sdk.util.ReduceFnTester; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; import com.google.cloud.dataflow.sdk.values.TimestampedValue; @@ -43,7 +43,7 @@ public class AfterPaneTest { @Test public void testAfterPaneWithGlobalWindowsAndCombining() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester tester = TriggerTester.combining( + ReduceFnTester tester = ReduceFnTester.combining( FixedWindows.of(windowDuration), AfterPane.elementCountAtLeast(2), AccumulationMode.DISCARDING_FIRED_PANES, @@ -74,7 +74,7 @@ public void testAfterPaneWithGlobalWindowsAndCombining() throws Exception { @Test public void testAfterPaneWithFixedWindow() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(windowDuration), AfterPane.elementCountAtLeast(2), AccumulationMode.DISCARDING_FIRED_PANES, @@ -103,7 +103,7 @@ public void testAfterPaneWithFixedWindow() throws Exception { @Test public void testAfterPaneWithMerging() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(windowDuration), AfterPane.elementCountAtLeast(2), AccumulationMode.DISCARDING_FIRED_PANES, diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java index cada240cd0e70..db041290b9e37 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java @@ -21,8 +21,8 @@ import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.WindowMatchers; +import com.google.cloud.dataflow.sdk.util.ReduceFnTester; import com.google.cloud.dataflow.sdk.util.TimeDomain; -import com.google.cloud.dataflow.sdk.util.TriggerTester; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; import com.google.cloud.dataflow.sdk.values.TimestampedValue; @@ -41,7 +41,7 @@ public class AfterProcessingTimeTest { @Test public void testAfterProcessingTimeIgnoresTimer() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(windowDuration), AfterProcessingTime .pastFirstElementInPane() @@ -59,7 +59,7 @@ public void testAfterProcessingTimeIgnoresTimer() throws Exception { @Test public void testAfterProcessingTimeWithFixedWindow() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(windowDuration), AfterProcessingTime .pastFirstElementInPane() @@ -107,7 +107,7 @@ public void testAfterProcessingTimeWithFixedWindow() throws Exception { @Test public void testAfterProcessingTimeWithMergingWindow() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(windowDuration), AfterProcessingTime .pastFirstElementInPane() diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java index 3270d073e4abf..074c8dbfec7d2 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java @@ -20,8 +20,8 @@ import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.WindowMatchers; +import com.google.cloud.dataflow.sdk.util.ReduceFnTester; import com.google.cloud.dataflow.sdk.util.TimeDomain; -import com.google.cloud.dataflow.sdk.util.TriggerTester; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; import com.google.cloud.dataflow.sdk.values.TimestampedValue; @@ -44,7 +44,7 @@ public class AfterSynchronizedProcessingTimeTest { @Test public void testAfterProcessingTimeWithFixedWindow() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(windowDuration), underTest, AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); @@ -73,7 +73,7 @@ public void testAfterProcessingTimeWithFixedWindow() throws Exception { @Test public void testAfterProcessingTimeWithMergingWindow() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(windowDuration), underTest, AccumulationMode.DISCARDING_FIRED_PANES, @@ -96,7 +96,7 @@ public void testAfterProcessingTimeWithMergingWindow() throws Exception { @Test public void testAfterProcessingTimeWithMergingWindowAlreadyFired() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(windowDuration), underTest, AccumulationMode.DISCARDING_FIRED_PANES, @@ -129,7 +129,7 @@ public void testAfterProcessingTimeWithMergingWindowAlreadyFired() throws Except @Test public void testAfterSynchronizedProcessingTimeIgnoresTimer() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(windowDuration), new AfterSynchronizedProcessingTime(), AccumulationMode.DISCARDING_FIRED_PANES, diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermarkTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermarkTest.java index 9f34a9c920e14..06974323e13f5 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermarkTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermarkTest.java @@ -27,8 +27,8 @@ import com.google.cloud.dataflow.sdk.WindowMatchers; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.TriggerResult; +import com.google.cloud.dataflow.sdk.util.ReduceFnTester; import com.google.cloud.dataflow.sdk.util.TimeDomain; -import com.google.cloud.dataflow.sdk.util.TriggerTester; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; import com.google.cloud.dataflow.sdk.values.TimestampedValue; @@ -65,7 +65,7 @@ public void setUp() { @Test public void testFirstInPaneWithFixedWindow() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(windowDuration), AfterWatermark.pastFirstElementInPane().plusDelayOf(Duration.millis(5)), AccumulationMode.DISCARDING_FIRED_PANES, @@ -96,7 +96,7 @@ public void testFirstInPaneWithFixedWindow() throws Exception { @Test public void testAlignAndDelay() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(Duration.standardMinutes(1)), AfterWatermark.pastFirstElementInPane() .alignedTo(Duration.standardMinutes(1)) @@ -130,7 +130,7 @@ public void testAlignAndDelay() throws Exception { @Test public void testFirstInPaneWithMerging() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(windowDuration), AfterWatermark.pastFirstElementInPane().plusDelayOf(Duration.millis(5)), AccumulationMode.DISCARDING_FIRED_PANES, @@ -156,7 +156,7 @@ public void testFirstInPaneWithMerging() throws Exception { @Test public void testEndOfWindowFixedWindow() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(windowDuration), AfterWatermark.pastEndOfWindow(), AccumulationMode.DISCARDING_FIRED_PANES, @@ -190,7 +190,7 @@ public void testEndOfWindowFixedWindow() throws Exception { @Test public void testEndOfWindowWithMerging() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(windowDuration), AfterWatermark.pastEndOfWindow(), AccumulationMode.DISCARDING_FIRED_PANES, @@ -222,7 +222,7 @@ public void testEndOfWindowWithMerging() throws Exception { @Test public void testEndOfWindowIgnoresTimer() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(windowDuration), AfterWatermark.pastEndOfWindow(), AccumulationMode.DISCARDING_FIRED_PANES, @@ -270,7 +270,7 @@ public void testContinuation() throws Exception { @Test public void testEarlyAndAtWatermarkProcessElement() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(Duration.millis(100)), AfterWatermark.pastEndOfWindow() .withEarlyFirings(mockEarly), @@ -296,7 +296,7 @@ public void testEarlyAndAtWatermarkProcessElement() throws Exception { @Test public void testLateAndAtWatermarkProcessElement() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(Duration.millis(100)), AfterWatermark.pastEndOfWindow() .withLateFirings(mockLate), @@ -336,7 +336,7 @@ public void testLateAndAtWatermarkProcessElement() throws Exception { @Test public void testEarlyLateAndAtWatermarkProcessElement() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(Duration.millis(100)), AfterWatermark.pastEndOfWindow() .withEarlyFirings(mockEarly) @@ -382,7 +382,7 @@ public void testEarlyLateAndAtWatermarkProcessElement() throws Exception { @Test public void testEarlyAndAtWatermarkSessions() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(Duration.millis(20)), AfterWatermark.pastEndOfWindow() .withEarlyFirings(AfterPane.elementCountAtLeast(2)), @@ -406,7 +406,7 @@ public void testEarlyAndAtWatermarkSessions() throws Exception { @Test public void testLateAndAtWatermarkSessionsProcessingTime() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(Duration.millis(20)), AfterWatermark.pastEndOfWindow() .withLateFirings(AfterProcessingTime @@ -438,7 +438,7 @@ public void testLateAndAtWatermarkSessionsProcessingTime() throws Exception { @Test public void testLateAndAtWatermarkSessions() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(Duration.millis(20)), AfterWatermark.pastEndOfWindow() .withLateFirings(AfterPane.elementCountAtLeast(2)), @@ -468,7 +468,7 @@ public void testLateAndAtWatermarkSessions() throws Exception { @Test public void testEarlyLateAndAtWatermarkSessions() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(Duration.millis(20)), AfterWatermark.pastEndOfWindow() .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane() diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/DefaultTriggerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/DefaultTriggerTest.java index ab9c868a5e5a1..5c068c75d8e16 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/DefaultTriggerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/DefaultTriggerTest.java @@ -21,7 +21,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; -import com.google.cloud.dataflow.sdk.util.TriggerTester; +import com.google.cloud.dataflow.sdk.util.ReduceFnTester; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; import com.google.cloud.dataflow.sdk.values.TimestampedValue; @@ -43,7 +43,7 @@ public class DefaultTriggerTest { @Test public void testDefaultTriggerWithFixedWindow() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(Duration.millis(10)), DefaultTrigger.of(), AccumulationMode.DISCARDING_FIRED_PANES, @@ -82,7 +82,7 @@ public void testDefaultTriggerWithFixedWindow() throws Exception { @Test public void testDefaultTriggerWithSessionWindow() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(Duration.millis(10)), DefaultTrigger.of(), AccumulationMode.DISCARDING_FIRED_PANES, @@ -113,7 +113,7 @@ public void testDefaultTriggerWithSessionWindow() throws Exception { @Test public void testDefaultTriggerWithSlidingWindow() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( SlidingWindows.of(Duration.millis(10)).every(Duration.millis(5)), DefaultTrigger.of(), AccumulationMode.DISCARDING_FIRED_PANES, @@ -148,7 +148,7 @@ public void testDefaultTriggerWithSlidingWindow() throws Exception { @Test public void testDefaultTriggerWithContainedSessionWindow() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(Duration.millis(10)), DefaultTrigger.of(), AccumulationMode.DISCARDING_FIRED_PANES, diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTriggerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTriggerTest.java index e5be23527fa63..60d15473cb8cb 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTriggerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTriggerTest.java @@ -26,8 +26,8 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.MergeResult; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.TriggerResult; +import com.google.cloud.dataflow.sdk.util.ReduceFnTester; import com.google.cloud.dataflow.sdk.util.TimeDomain; -import com.google.cloud.dataflow.sdk.util.TriggerTester; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; import com.google.cloud.dataflow.sdk.values.TimestampedValue; @@ -49,7 +49,7 @@ public class OrFinallyTriggerTest { @Mock private Trigger mockActual; @Mock private OnceTrigger mockUntil; - private TriggerTester, IntervalWindow> tester; + private ReduceFnTester, IntervalWindow> tester; private IntervalWindow firstWindow; public void setUp(WindowFn windowFn) throws Exception { @@ -58,7 +58,7 @@ public void setUp(WindowFn windowFn) throws Exception { Trigger underTest = new OrFinallyTrigger(mockActual, mockUntil); - tester = TriggerTester.nonCombining( + tester = ReduceFnTester.nonCombining( windowFn, underTest, AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); firstWindow = new IntervalWindow(new Instant(0), new Instant(10)); @@ -233,7 +233,7 @@ public void testFireDeadline() throws Exception { @Test public void testOrFinallyRealTriggersFixedWindow() throws Exception { // Test an orFinally with a composite trigger, and make sure it properly resets state, etc. - tester = TriggerTester.nonCombining(FixedWindows.of(Duration.millis(50)), + tester = ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(50)), Repeatedly.forever( // This element count should never fire because the orFinally fires sooner, every time AfterPane.elementCountAtLeast(12) @@ -286,7 +286,7 @@ public void testOrFinallyRealTriggersFixedWindow() throws Exception { @Test public void testOrFinallyMergingWindowSomeFinished() throws Exception { Duration windowDuration = Duration.millis(10); - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(windowDuration), AfterProcessingTime.pastFirstElementInPane() .plusDelayOf(Duration.millis(5)) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/RepeatedlyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/RepeatedlyTest.java index 413d1bccd6625..cd25b2f82009c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/RepeatedlyTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/RepeatedlyTest.java @@ -24,8 +24,8 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.MergeResult; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.TriggerResult; +import com.google.cloud.dataflow.sdk.util.ReduceFnTester; import com.google.cloud.dataflow.sdk.util.TimeDomain; -import com.google.cloud.dataflow.sdk.util.TriggerTester; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; import com.google.cloud.dataflow.sdk.values.TimestampedValue; @@ -46,13 +46,13 @@ public class RepeatedlyTest { @Mock private Trigger mockRepeated; - private TriggerTester, IntervalWindow> tester; + private ReduceFnTester, IntervalWindow> tester; private IntervalWindow firstWindow; public void setUp(WindowFn windowFn) throws Exception { MockitoAnnotations.initMocks(this); Trigger underTest = Repeatedly.forever(mockRepeated); - tester = TriggerTester.nonCombining( + tester = ReduceFnTester.nonCombining( windowFn, underTest, AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerTester.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReduceFnTester.java similarity index 95% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerTester.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReduceFnTester.java index 5d119196086b1..4413d0630890a 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerTester.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReduceFnTester.java @@ -60,8 +60,6 @@ import org.joda.time.Duration; import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.ArrayList; @@ -77,20 +75,19 @@ import javax.annotation.Nullable; /** - * Test utility that runs a {@link WindowFn}, {@link Trigger} using in-memory stub implementations - * to provide the {@link TimerInternals} and {@link WindowingInternals} needed to run - * {@code Trigger}s and {@code ReduceFn}s. + * Test utility that runs a {@link ReduceFn}, {@link WindowFn}, {@link Trigger} using in-memory stub + * implementations to provide the {@link TimerInternals} and {@link WindowingInternals} needed to + * run {@code Trigger}s and {@code ReduceFn}s. * * @param The element types. * @param The final type for elements in the window (for instance, * {@code Iterable}) * @param The type of windows being used. */ -public class TriggerTester { - private static final Logger LOG = LoggerFactory.getLogger(TriggerTester.class); - +public class ReduceFnTester { private final TestInMemoryStateInternals stateInternals = new TestInMemoryStateInternals(); private final TestTimerInternals timerInternals = new TestTimerInternals(); + private final WindowFn windowFn; private final TestWindowingInternals windowingInternals; private final Coder outputCoder; @@ -105,17 +102,17 @@ public class TriggerTester { private final InMemoryLongSumAggregator droppedDueToLateness = new InMemoryLongSumAggregator(ReduceFnRunner.DROPPED_DUE_TO_LATENESS_COUNTER); - public static TriggerTester, W> nonCombining( - WindowingStrategy windowingStrategy) throws Exception { - return new TriggerTester, W>( + public static ReduceFnTester, W> + nonCombining(WindowingStrategy windowingStrategy) throws Exception { + return new ReduceFnTester, W>( windowingStrategy, SystemReduceFn.buffering(VarIntCoder.of()).create(KEY), IterableCoder.of(VarIntCoder.of())); } - public static TriggerTester, W> nonCombining( - WindowFn windowFn, TriggerBuilder trigger, AccumulationMode mode, - Duration allowedDataLateness) throws Exception { + public static ReduceFnTester, W> + nonCombining(WindowFn windowFn, TriggerBuilder trigger, AccumulationMode mode, + Duration allowedDataLateness) throws Exception { WindowingStrategy strategy = WindowingStrategy.of(windowFn) .withTrigger(trigger.buildTrigger()) @@ -124,7 +121,7 @@ public static TriggerTester return nonCombining(strategy); } - public static TriggerTester + public static ReduceFnTester combining(WindowFn windowFn, Trigger trigger, AccumulationMode mode, KeyedCombineFn combineFn, Coder outputCoder, Duration allowedDataLateness) throws Exception { @@ -138,14 +135,14 @@ public static TriggerTester AppliedCombineFn.withInputCoder( combineFn, registry, KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())); - return new TriggerTester( + return new ReduceFnTester( strategy, SystemReduceFn.combining(StringUtf8Coder.of(), fn) .create(KEY), outputCoder); } - private TriggerTester(WindowingStrategy wildcardStrategy, + private ReduceFnTester(WindowingStrategy wildcardStrategy, ReduceFn reduceFn, Coder outputCoder) throws Exception { @SuppressWarnings("unchecked") WindowingStrategy objectStrategy = (WindowingStrategy) wildcardStrategy; @@ -318,7 +315,7 @@ public WindowedValue apply(TimestampedValue input) { try { InputT value = input.getValue(); Instant timestamp = input.getTimestamp(); - Collection windows = windowFn.assignWindows(new TriggerTester.TestAssignContext( + Collection windows = windowFn.assignWindows(new TestAssignContext( windowFn, value, timestamp, Arrays.asList(GlobalWindow.INSTANCE))); return WindowedValue.of(value, timestamp, windows, PaneInfo.NO_FIRING); } catch (Exception e) { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerExecutorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerExecutorTest.java index c6d56bc15d887..164ab3f65183a 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerExecutorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerExecutorTest.java @@ -71,8 +71,8 @@ */ @RunWith(JUnit4.class) public class TriggerExecutorTest { - - @Mock private Trigger mockTrigger; + @Mock + private Trigger mockTrigger; private IntervalWindow firstWindow; @Before @@ -82,11 +82,9 @@ public void setUp() { firstWindow = new IntervalWindow(new Instant(0), new Instant(10)); } - private void injectElement(TriggerTester tester, - int element, TriggerResult result) - throws Exception { - when(mockTrigger.onElement( - Mockito..OnElementContext>any())) + private void injectElement(ReduceFnTester tester, int element, + TriggerResult result) throws Exception { + when(mockTrigger.onElement(Mockito..OnElementContext>any())) .thenReturn(result); tester.injectElements(TimestampedValue.of(element, new Instant(element))); } @@ -94,11 +92,9 @@ private void injectElement(TriggerTester tester, @Test public void testOnElementBufferingDiscarding() throws Exception { // Test basic execution of a trigger using a non-combining window set and discarding mode. - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( - FixedWindows.of(Duration.millis(10)), - mockTrigger, - AccumulationMode.DISCARDING_FIRED_PANES, - Duration.millis(100)); + ReduceFnTester, IntervalWindow> tester = + ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger, + AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); injectElement(tester, 1, TriggerResult.CONTINUE); injectElement(tester, 2, TriggerResult.FIRE); @@ -108,9 +104,11 @@ public void testOnElementBufferingDiscarding() throws Exception { // This element shouldn't be seen, because the trigger has finished injectElement(tester, 4, null); - assertThat(tester.extractOutput(), Matchers.contains( - isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 0, 10), - isSingleWindowedValue(Matchers.containsInAnyOrder(3), 3, 0, 10))); + assertThat( + tester.extractOutput(), + Matchers.contains( + isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 0, 10), + isSingleWindowedValue(Matchers.containsInAnyOrder(3), 3, 0, 10))); assertTrue(tester.isMarkedFinished(firstWindow)); tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow); @@ -121,11 +119,9 @@ public void testOnElementBufferingDiscarding() throws Exception { @Test public void testOnElementBufferingAccumulating() throws Exception { // Test basic execution of a trigger using a non-combining window set and accumulating mode. - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( - FixedWindows.of(Duration.millis(10)), - mockTrigger, - AccumulationMode.ACCUMULATING_FIRED_PANES, - Duration.millis(100)); + ReduceFnTester, IntervalWindow> tester = + ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger, + AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(100)); injectElement(tester, 1, TriggerResult.CONTINUE); injectElement(tester, 2, TriggerResult.FIRE); @@ -134,9 +130,11 @@ public void testOnElementBufferingAccumulating() throws Exception { // This element shouldn't be seen, because the trigger has finished injectElement(tester, 4, null); - assertThat(tester.extractOutput(), Matchers.contains( - isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 0, 10), - isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2, 3), 3, 0, 10))); + assertThat( + tester.extractOutput(), + Matchers.contains( + isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 0, 10), + isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2, 3), 3, 0, 10))); assertTrue(tester.isMarkedFinished(firstWindow)); tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow); } @@ -144,13 +142,9 @@ public void testOnElementBufferingAccumulating() throws Exception { @Test public void testOnElementCombiningDiscarding() throws Exception { // Test basic execution of a trigger using a non-combining window set and discarding mode. - TriggerTester tester = TriggerTester.combining( - FixedWindows.of(Duration.millis(10)), - mockTrigger, - AccumulationMode.DISCARDING_FIRED_PANES, - new Sum.SumIntegerFn().asKeyedFn(), - VarIntCoder.of(), - Duration.millis(100)); + ReduceFnTester tester = ReduceFnTester.combining( + FixedWindows.of(Duration.millis(10)), mockTrigger, AccumulationMode.DISCARDING_FIRED_PANES, + new Sum.SumIntegerFn().asKeyedFn(), VarIntCoder.of(), Duration.millis(100)); injectElement(tester, 2, TriggerResult.CONTINUE); injectElement(tester, 3, TriggerResult.FIRE); @@ -159,9 +153,11 @@ public void testOnElementCombiningDiscarding() throws Exception { // This element shouldn't be seen, because the trigger has finished injectElement(tester, 6, null); - assertThat(tester.extractOutput(), Matchers.contains( - isSingleWindowedValue(Matchers.equalTo(5), 2, 0, 10), - isSingleWindowedValue(Matchers.equalTo(4), 4, 0, 10))); + assertThat( + tester.extractOutput(), + Matchers.contains( + isSingleWindowedValue(Matchers.equalTo(5), 2, 0, 10), + isSingleWindowedValue(Matchers.equalTo(4), 4, 0, 10))); assertTrue(tester.isMarkedFinished(firstWindow)); tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow); } @@ -169,13 +165,10 @@ public void testOnElementCombiningDiscarding() throws Exception { @Test public void testOnElementCombiningAccumulating() throws Exception { // Test basic execution of a trigger using a non-combining window set and accumulating mode. - TriggerTester tester = TriggerTester.combining( - FixedWindows.of(Duration.millis(10)), - mockTrigger, - AccumulationMode.ACCUMULATING_FIRED_PANES, - new Sum.SumIntegerFn().asKeyedFn(), - VarIntCoder.of(), - Duration.millis(100)); + ReduceFnTester tester = + ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)), mockTrigger, + AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().asKeyedFn(), + VarIntCoder.of(), Duration.millis(100)); injectElement(tester, 1, TriggerResult.CONTINUE); injectElement(tester, 2, TriggerResult.FIRE); @@ -184,9 +177,11 @@ public void testOnElementCombiningAccumulating() throws Exception { // This element shouldn't be seen, because the trigger has finished injectElement(tester, 4, null); - assertThat(tester.extractOutput(), Matchers.contains( - isSingleWindowedValue(Matchers.equalTo(3), 1, 0, 10), - isSingleWindowedValue(Matchers.equalTo(6), 3, 0, 10))); + assertThat( + tester.extractOutput(), + Matchers.contains( + isSingleWindowedValue(Matchers.equalTo(3), 1, 0, 10), + isSingleWindowedValue(Matchers.equalTo(6), 3, 0, 10))); assertTrue(tester.isMarkedFinished(firstWindow)); tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow); } @@ -194,8 +189,8 @@ public void testOnElementCombiningAccumulating() throws Exception { @Test public void testWatermarkHoldAndLateData() throws Exception { // Test handling of late data. Specifically, ensure the watermark hold is correct. - TriggerTester, IntervalWindow> tester = - TriggerTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger, + ReduceFnTester, IntervalWindow> tester = + ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger, AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(10)); // Input watermark -> null @@ -305,23 +300,25 @@ public void testWatermarkHoldAndLateData() throws Exception { @Test public void testPaneInfoAllStates() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( - FixedWindows.of(Duration.millis(10)), - mockTrigger, - AccumulationMode.DISCARDING_FIRED_PANES, - Duration.millis(100)); + ReduceFnTester, IntervalWindow> tester = + ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger, + AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); when(mockTrigger.onTimer(Mockito..OnTimerContext>any())) .thenReturn(TriggerResult.CONTINUE); tester.advanceInputWatermark(new Instant(0)); injectElement(tester, 1, TriggerResult.FIRE); - assertThat(tester.extractOutput(), Matchers.contains( - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY)))); + assertThat( + tester.extractOutput(), + Matchers.contains( + WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY)))); injectElement(tester, 2, TriggerResult.FIRE); - assertThat(tester.extractOutput(), Matchers.contains( - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.EARLY, 1, -1)))); + assertThat( + tester.extractOutput(), + Matchers.contains(WindowMatchers.valueWithPaneInfo( + PaneInfo.createPane(false, false, Timing.EARLY, 1, -1)))); tester.advanceInputWatermark(new Instant(15)); injectElement(tester, 3, TriggerResult.FIRE); @@ -345,106 +342,127 @@ public void testPaneInfoAllStates() throws Exception { @Test public void testPaneInfoAllStatesAfterWatermark() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) - .withTrigger(Repeatedly.forever( - AfterFirst.of( - AfterPane.elementCountAtLeast(2), - AfterWatermark.pastEndOfWindow()))) + .withTrigger(Repeatedly.forever(AfterFirst.of( + AfterPane.elementCountAtLeast(2), + AfterWatermark.pastEndOfWindow()))) .withMode(AccumulationMode.DISCARDING_FIRED_PANES) .withAllowedLateness(Duration.millis(100)) .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS)); tester.advanceInputWatermark(new Instant(0)); tester.injectElements( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(2, new Instant(2))); + TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); List>> output = tester.extractOutput(); - assertThat(output, Matchers.contains( - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)))); - assertThat(output, Matchers.contains( - WindowMatchers.isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 0, 10))); + assertThat( + output, + Matchers.contains(WindowMatchers.valueWithPaneInfo( + PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)))); + assertThat( + output, + Matchers.contains( + WindowMatchers.isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 0, 10))); tester.advanceInputWatermark(new Instant(50)); // We should get the ON_TIME pane even though it is empty, // because we have an AfterWatermark.pastEndOfWindow() trigger. output = tester.extractOutput(); - assertThat(output, Matchers.contains( - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0)))); - assertThat(output, Matchers.contains( - WindowMatchers.isSingleWindowedValue(Matchers.emptyIterable(), 9, 0, 10))); + assertThat( + output, + Matchers.contains(WindowMatchers.valueWithPaneInfo( + PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0)))); + assertThat( + output, + Matchers.contains( + WindowMatchers.isSingleWindowedValue(Matchers.emptyIterable(), 9, 0, 10))); // We should get the final pane even though it is empty. tester.advanceInputWatermark(new Instant(150)); output = tester.extractOutput(); - assertThat(output, Matchers.contains( - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1)))); - assertThat(output, Matchers.contains( - WindowMatchers.isSingleWindowedValue(Matchers.emptyIterable(), 9, 0, 10))); + assertThat( + output, + Matchers.contains( + WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1)))); + assertThat( + output, + Matchers.contains( + WindowMatchers.isSingleWindowedValue(Matchers.emptyIterable(), 9, 0, 10))); } @Test public void testPaneInfoAllStatesAfterWatermarkAccumulating() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) - .withTrigger(Repeatedly.forever( - AfterFirst.of( - AfterPane.elementCountAtLeast(2), - AfterWatermark.pastEndOfWindow()))) + .withTrigger(Repeatedly.forever(AfterFirst.of( + AfterPane.elementCountAtLeast(2), + AfterWatermark.pastEndOfWindow()))) .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES) .withAllowedLateness(Duration.millis(100)) .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS)); tester.advanceInputWatermark(new Instant(0)); tester.injectElements( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(2, new Instant(2))); + TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); List>> output = tester.extractOutput(); - assertThat(output, Matchers.contains( - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)))); - assertThat(output, Matchers.contains( - WindowMatchers.isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 0, 10))); + assertThat( + output, + Matchers.contains(WindowMatchers.valueWithPaneInfo( + PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)))); + assertThat( + output, + Matchers.contains( + WindowMatchers.isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 0, 10))); tester.advanceInputWatermark(new Instant(50)); // We should get the ON_TIME pane even though it is empty, // because we have an AfterWatermark.pastEndOfWindow() trigger. output = tester.extractOutput(); - assertThat(output, Matchers.contains( - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0)))); - assertThat(output, Matchers.contains( - WindowMatchers.isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 9, 0, 10))); + assertThat( + output, + Matchers.contains(WindowMatchers.valueWithPaneInfo( + PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0)))); + assertThat( + output, + Matchers.contains( + WindowMatchers.isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 9, 0, 10))); // We should get the final pane even though it is empty. tester.advanceInputWatermark(new Instant(150)); output = tester.extractOutput(); - assertThat(output, Matchers.contains( - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1)))); - assertThat(output, Matchers.contains( - WindowMatchers.isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 9, 0, 10))); + assertThat( + output, + Matchers.contains( + WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1)))); + assertThat( + output, + Matchers.contains( + WindowMatchers.isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 9, 0, 10))); } @Test public void testPaneInfoFinalAndOnTime() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( + ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining( WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) .withTrigger( Repeatedly.forever(AfterPane.elementCountAtLeast(2)) - .orFinally(AfterWatermark.pastEndOfWindow())) + .orFinally(AfterWatermark.pastEndOfWindow())) .withMode(AccumulationMode.DISCARDING_FIRED_PANES) .withAllowedLateness(Duration.millis(100)) .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS)); tester.advanceInputWatermark(new Instant(0)); tester.injectElements( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(2, new Instant(2))); + TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); - assertThat(tester.extractOutput(), Matchers.contains( - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)))); + assertThat( + tester.extractOutput(), + Matchers.contains(WindowMatchers.valueWithPaneInfo( + PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)))); tester.advanceInputWatermark(new Instant(150)); assertThat(tester.extractOutput(), Matchers.contains( @@ -453,41 +471,39 @@ public void testPaneInfoFinalAndOnTime() throws Exception { @Test public void testPaneInfoSkipToFinish() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( - FixedWindows.of(Duration.millis(10)), - mockTrigger, - AccumulationMode.DISCARDING_FIRED_PANES, - Duration.millis(100)); + ReduceFnTester, IntervalWindow> tester = + ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger, + AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); tester.advanceInputWatermark(new Instant(0)); injectElement(tester, 1, TriggerResult.FIRE_AND_FINISH); - assertThat(tester.extractOutput(), Matchers.contains( - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.EARLY)))); + assertThat( + tester.extractOutput(), + Matchers.contains( + WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.EARLY)))); } @Test public void testPaneInfoSkipToNonSpeculativeAndFinish() throws Exception { - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( - FixedWindows.of(Duration.millis(10)), - mockTrigger, - AccumulationMode.DISCARDING_FIRED_PANES, - Duration.millis(100)); + ReduceFnTester, IntervalWindow> tester = + ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger, + AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); tester.advanceInputWatermark(new Instant(15)); injectElement(tester, 1, TriggerResult.FIRE_AND_FINISH); - assertThat(tester.extractOutput(), Matchers.contains( - WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.LATE)))); + assertThat( + tester.extractOutput(), + Matchers.contains( + WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.LATE)))); } @Test public void testMergeBeforeFinalizing() throws Exception { // Verify that we merge windows before producing output so users don't see undesired // unmerged windows. - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( - Sessions.withGapDuration(Duration.millis(10)), - mockTrigger, - AccumulationMode.DISCARDING_FIRED_PANES, - Duration.millis(0)); + ReduceFnTester, IntervalWindow> tester = + ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger, + AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(0)); // All on time data, verify watermark hold. when(mockTrigger.onMerge(Mockito..OnMergeContext>any())) @@ -495,8 +511,7 @@ public void testMergeBeforeFinalizing() throws Exception { when(mockTrigger.onElement(Mockito..OnElementContext>any())) .thenReturn(TriggerResult.CONTINUE, TriggerResult.CONTINUE); tester.injectElements( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(10, new Instant(10))); + TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(10, new Instant(10))); when(mockTrigger.onTimer(Mockito..OnTimerContext>any())) .thenReturn(TriggerResult.CONTINUE); @@ -513,17 +528,15 @@ public void testMergeBeforeFinalizing() throws Exception { @Test public void testDropDataMultipleWindows() throws Exception { - TriggerTester tester = TriggerTester.combining( + ReduceFnTester tester = ReduceFnTester.combining( SlidingWindows.of(Duration.millis(100)).every(Duration.millis(30)), - AfterWatermark.pastEndOfWindow(), - AccumulationMode.ACCUMULATING_FIRED_PANES, - new Sum.SumIntegerFn().asKeyedFn(), - VarIntCoder.of(), - Duration.millis(20)); + AfterWatermark.pastEndOfWindow(), AccumulationMode.ACCUMULATING_FIRED_PANES, + new Sum.SumIntegerFn().asKeyedFn(), VarIntCoder.of(), Duration.millis(20)); tester.injectElements( - TimestampedValue.of(10, new Instant(23)), // [-60, 40), [-30, 70), [0, 100) - TimestampedValue.of(12, new Instant(40))); // [-30, 70), [0, 100), [30, 130) + TimestampedValue.of(10, new Instant(23)), // [-60, 40), [-30, 70), [0, 100) + TimestampedValue.of(12, new Instant(40))); + // [-30, 70), [0, 100), [30, 130) assertEquals(0, tester.getElementsDroppedDueToLateness()); assertEquals(0, tester.getElementsDroppedDueToClosedWindow()); @@ -535,8 +548,8 @@ public void testDropDataMultipleWindows() throws Exception { assertEquals(0, tester.getElementsDroppedDueToLateness()); assertEquals(1, tester.getElementsDroppedDueToClosedWindow()); - tester.injectElements( - TimestampedValue.of(16, new Instant(40))); // dropped b/c lateness, assigned to 3 windows + tester.injectElements(TimestampedValue.of(16, new Instant(40))); + // dropped b/c lateness, assigned to 3 windows assertEquals(3, tester.getElementsDroppedDueToLateness()); assertEquals(1, tester.getElementsDroppedDueToClosedWindow()); @@ -546,11 +559,9 @@ public void testDropDataMultipleWindows() throws Exception { public void testIdempotentEmptyPanes() throws Exception { // Test uninteresting (empty) panes don't increment the index or otherwise // modify PaneInfo. - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( - FixedWindows.of(Duration.millis(10)), - mockTrigger, - AccumulationMode.DISCARDING_FIRED_PANES, - Duration.millis(100)); + ReduceFnTester, IntervalWindow> tester = + ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger, + AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); // Inject a couple of on-time elements and fire at the window end. injectElement(tester, 1, TriggerResult.CONTINUE); @@ -580,8 +591,8 @@ public void testIdempotentEmptyPanes() throws Exception { // The late pane has the correct indices. assertThat(output.get(1).getValue(), contains(3)); - assertThat(output.get(1).getPane(), - equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1))); + assertThat( + output.get(1).getPane(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1))); assertTrue(tester.isMarkedFinished(firstWindow)); tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow); @@ -594,11 +605,9 @@ public void testIdempotentEmptyPanes() throws Exception { public void testIdempotentEmptyPanesAccumulating() throws Exception { // Test uninteresting (empty) panes don't increment the index or otherwise // modify PaneInfo. - TriggerTester, IntervalWindow> tester = TriggerTester.nonCombining( - FixedWindows.of(Duration.millis(10)), - mockTrigger, - AccumulationMode.ACCUMULATING_FIRED_PANES, - Duration.millis(100)); + ReduceFnTester, IntervalWindow> tester = + ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger, + AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(100)); // Inject a couple of on-time elements and fire at the window end. injectElement(tester, 1, TriggerResult.CONTINUE); @@ -630,8 +639,8 @@ public void testIdempotentEmptyPanesAccumulating() throws Exception { // The late pane has the correct indices. assertThat(output.get(1).getValue(), containsInAnyOrder(1, 2, 3)); - assertThat(output.get(1).getPane(), - equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1))); + assertThat( + output.get(1).getPane(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1))); assertTrue(tester.isMarkedFinished(firstWindow)); tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow); @@ -642,6 +651,7 @@ public void testIdempotentEmptyPanesAccumulating() throws Exception { private class ResultCaptor implements Answer { private T result = null; + public T get() { return result; } @@ -661,27 +671,25 @@ public T answer(InvocationOnMock invocationOnMock) throws Throwable { */ @Test public void testEmptyOnTimeFromOrFinally() throws Exception { - TriggerTester tester = TriggerTester.combining( - FixedWindows.of(Duration.millis(10)), - AfterEach.inOrder( - Repeatedly.forever( - AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(new Duration(5))) - .orFinally(AfterWatermark.pastEndOfWindow()), - Repeatedly.forever( - AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(new Duration(25)))), - AccumulationMode.ACCUMULATING_FIRED_PANES, - new Sum.SumIntegerFn().asKeyedFn(), VarIntCoder.of(), - Duration.millis(100)); + ReduceFnTester tester = + ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)), + AfterEach.inOrder( + Repeatedly + .forever( + AfterProcessingTime.pastFirstElementInPane().plusDelayOf( + new Duration(5))) + .orFinally(AfterWatermark.pastEndOfWindow()), + Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane().plusDelayOf( + new Duration(25)))), + AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().asKeyedFn(), + VarIntCoder.of(), Duration.millis(100)); tester.advanceInputWatermark(new Instant(0)); tester.advanceProcessingTime(new Instant(0)); - tester.injectElements( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(1, new Instant(3)), - TimestampedValue.of(1, new Instant(7)), + tester.injectElements(TimestampedValue.of(1, new Instant(1)), + TimestampedValue.of(1, new Instant(3)), TimestampedValue.of(1, new Instant(7)), TimestampedValue.of(1, new Instant(5))); tester.advanceProcessingTime(new Instant(6)); @@ -693,10 +701,12 @@ public void testEmptyOnTimeFromOrFinally() throws Exception { assertThat(output.get(0), WindowMatchers.isSingleWindowedValue(4, 1, 0, 10)); assertThat(output.get(1), WindowMatchers.isSingleWindowedValue(4, 9, 0, 10)); - assertThat(output.get(0), WindowMatchers.valueWithPaneInfo(PaneInfo.createPane( - true, false, Timing.EARLY, 0, -1))); - assertThat(output.get(1), WindowMatchers.valueWithPaneInfo(PaneInfo.createPane( - false, false, Timing.ON_TIME, 1, 0))); + assertThat( + output.get(0), + WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))); + assertThat( + output.get(1), + WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0))); } /** @@ -706,27 +716,25 @@ public void testEmptyOnTimeFromOrFinally() throws Exception { */ @Test public void testProcessingTime() throws Exception { - TriggerTester tester = TriggerTester.combining( - FixedWindows.of(Duration.millis(10)), - AfterEach.inOrder( - Repeatedly.forever( - AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(new Duration(5))) - .orFinally(AfterWatermark.pastEndOfWindow()), - Repeatedly.forever( - AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(new Duration(25)))), - AccumulationMode.ACCUMULATING_FIRED_PANES, - new Sum.SumIntegerFn().asKeyedFn(), VarIntCoder.of(), - Duration.millis(100)); + ReduceFnTester tester = + ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)), + AfterEach.inOrder( + Repeatedly + .forever( + AfterProcessingTime.pastFirstElementInPane().plusDelayOf( + new Duration(5))) + .orFinally(AfterWatermark.pastEndOfWindow()), + Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane().plusDelayOf( + new Duration(25)))), + AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().asKeyedFn(), + VarIntCoder.of(), Duration.millis(100)); tester.advanceInputWatermark(new Instant(0)); tester.advanceProcessingTime(new Instant(0)); - tester.injectElements( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(1, new Instant(3)), - TimestampedValue.of(1, new Instant(7)), + tester.injectElements(TimestampedValue.of(1, new Instant(1)), + TimestampedValue.of(1, new Instant(3)), TimestampedValue.of(1, new Instant(7)), TimestampedValue.of(1, new Instant(5))); // 4 elements all at processing time 0 @@ -768,29 +776,30 @@ public void testProcessingTime() throws Exception { assertThat(output.get(2), WindowMatchers.isSingleWindowedValue(11, 9, 0, 10)); assertThat(output.get(3), WindowMatchers.isSingleWindowedValue(12, 9, 0, 10)); - assertThat(output.get(0), WindowMatchers.valueWithPaneInfo(PaneInfo.createPane( - true, false, Timing.EARLY, 0, -1))); - assertThat(output.get(1), WindowMatchers.valueWithPaneInfo(PaneInfo.createPane( - false, false, Timing.ON_TIME, 1, 0))); - assertThat(output.get(2), WindowMatchers.valueWithPaneInfo(PaneInfo.createPane( - false, false, Timing.LATE, 2, 1))); - assertThat(output.get(3), WindowMatchers.valueWithPaneInfo(PaneInfo.createPane( - false, true, Timing.LATE, 3, 2))); + assertThat( + output.get(0), + WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))); + assertThat( + output.get(1), + WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0))); + assertThat( + output.get(2), + WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.LATE, 2, 1))); + assertThat( + output.get(3), + WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 3, 2))); } @Test public void testMultipleTimerTypes() throws Exception { - Trigger trigger = spy(Repeatedly.forever( - AfterFirst.of(AfterProcessingTime.pastFirstElementInPane().plusDelayOf( - Duration.millis(10)), - AfterWatermark.pastEndOfWindow()))); - - TriggerTester, IntervalWindow> tester = - TriggerTester.nonCombining( - FixedWindows.of(Duration.millis(10)), - trigger, - AccumulationMode.DISCARDING_FIRED_PANES, - Duration.standardDays(1)); + Trigger trigger = spy(Repeatedly.forever(AfterFirst.of( + AfterProcessingTime.pastFirstElementInPane().plusDelayOf( + Duration.millis(10)), + AfterWatermark.pastEndOfWindow()))); + + ReduceFnTester, IntervalWindow> tester = + ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), trigger, + AccumulationMode.DISCARDING_FIRED_PANES, Duration.standardDays(1)); tester.injectElements(TimestampedValue.of(1, new Instant(1))); From a82abf46e52da2e7941171bafaef968c358508f1 Mon Sep 17 00:00:00 2001 From: kirpichov Date: Thu, 10 Dec 2015 16:56:53 -0800 Subject: [PATCH 1225/1541] Fixes a data race in OffsetBasedSource This race is harmless in practice, because we only read properties of the returned source that don't change across splitAtFraction(), however it is better to be safe. Also clarifies documentation related to Reader.getCurrentSource() and dynamic work rebalancing. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109949629 --- .../cloud/dataflow/sdk/io/BoundedSource.java | 88 ++++++++++++++++++- .../dataflow/sdk/io/OffsetBasedSource.java | 5 +- .../google/cloud/dataflow/sdk/io/Source.java | 57 +++++++----- 3 files changed, 122 insertions(+), 28 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedSource.java index 5f91ef39b863a..be3a415cff93e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedSource.java @@ -80,14 +80,22 @@ public abstract List> splitIntoBundles( * again on this object. * *

Thread safety

- * All methods will be run from the same thread except {@link #splitAtFraction} and - * {@link #getFractionConsumed}, which can be called concurrently from a different thread. There - * will not be multiple concurrent calls to {@link #splitAtFraction} but there can be for - * {@link #getFractionConsumed} if {@link #splitAtFraction} is implemented. + * All methods will be run from the same thread except {@link #splitAtFraction}, + * {@link #getFractionConsumed} and {@link #getCurrentSource}, which can be called concurrently + * from a different thread. There will not be multiple concurrent calls to + * {@link #splitAtFraction} but there can be for {@link #getFractionConsumed} if + * {@link #splitAtFraction} is implemented. + * *

If the source does not implement {@link #splitAtFraction}, you do not need to worry about * thread safety. If implemented, it must be safe to call {@link #splitAtFraction} and * {@link #getFractionConsumed} concurrently with other methods. * + *

Additionally, a successful {@link #splitAtFraction} call must, by definition, cause + * {@link #getCurrentSource} to start returning a different value. + * Callers of {@link #getCurrentSource} need to be aware of the possibility that the returned + * value can change at any time, and must only access the properties of the source returned by + * {@link #getCurrentSource} which do not change between {@link #splitAtFraction} calls. + * *

Implementing {@link #splitAtFraction}

* In the course of dynamic work rebalancing, the method {@link #splitAtFraction} * may be called concurrently with {@link #advance} or {@link #start}. It is critical that @@ -124,6 +132,78 @@ public Double getFractionConsumed() { return null; } + /** + * Returns a {@code Source} describing the same input that this {@code Reader} currently reads + * (including items already read). + * + *

Usage

+ *

Reader subclasses can use this method for convenience to access unchanging properties of + * the source being read. Alternatively, they can cache these properties in the constructor. + *

The framework will call this method in the course of dynamic work rebalancing, e.g. after + * a successful {@link BoundedSource.BoundedReader#splitAtFraction} call. + * + *

Mutability and thread safety

+ * Remember that {@link Source} objects must always be immutable. However, the return value of + * this function may be affected by dynamic work rebalancing, happening asynchronously via + * {@link BoundedSource.BoundedReader#splitAtFraction}, meaning it can return a different + * {@link Source} object. However, the returned object itself will still itself be immutable. + * Callers must take care not to rely on properties of the returned source that may be + * asynchronously changed as a result of this process (e.g. do not cache an end offset when + * reading a file). + * + *

Implementation

+ * For convenience, subclasses should usually return the most concrete subclass of + * {@link Source} possible. + * In practice, the implementation of this method should nearly always be one of the following: + *
    + *
  • Source that inherits from a base class that already implements + * {@link #getCurrentSource}: delegate to base class. In this case, it is almost always + * an error for the subclass to maintain its own copy of the source. + *
    {@code
    +     *   public FooReader(FooSource source) {
    +     *     super(source);
    +     *   }
    +     *
    +     *   public FooSource getCurrentSource() {
    +     *     return (FooSource)super.getCurrentSource();
    +     *   }
    +     * }
    + *
  • Source that does not support dynamic work rebalancing: return a private final variable. + *
    {@code
    +     *   private final FooSource source;
    +     *
    +     *   public FooReader(FooSource source) {
    +     *     this.source = source;
    +     *   }
    +     *
    +     *   public FooSource getCurrentSource() {
    +     *     return source;
    +     *   }
    +     * }
    + *
  • {@link BoundedSource.BoundedReader} that explicitly supports dynamic work rebalancing: + * maintain a variable pointing to an immutable source object, and protect it with + * synchronization. + *
    {@code
    +     *   private FooSource source;
    +     *
    +     *   public FooReader(FooSource source) {
    +     *     this.source = source;
    +     *   }
    +     *
    +     *   public synchronized FooSource getCurrentSource() {
    +     *     return source;
    +     *   }
    +     *
    +     *   public synchronized FooSource splitAtFraction(double fraction) {
    +     *     ...
    +     *     FooSource primary = ...;
    +     *     FooSource residual = ...;
    +     *     this.source = primary;
    +     *     return residual;
    +     *   }
    +     * }
    + *
+ */ @Override public abstract BoundedSource getCurrentSource(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java index ea33d3143ff9a..4527b85f222be 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java @@ -189,6 +189,7 @@ public abstract static class OffsetBasedReader extends BoundedReader { private static final Logger LOG = LoggerFactory.getLogger(OffsetBasedReader.class); private OffsetBasedSource source; + /** * The {@link OffsetRangeTracker} managing the range and current position of the source. * Subclasses MUST use it before returning records from {@link #start} or {@link #advance}: @@ -247,7 +248,7 @@ public final boolean advance() throws IOException { protected abstract boolean advanceImpl() throws IOException; @Override - public OffsetBasedSource getCurrentSource() { + public synchronized OffsetBasedSource getCurrentSource() { return source; } @@ -257,7 +258,7 @@ public Double getFractionConsumed() { } @Override - public final OffsetBasedSource splitAtFraction(double fraction) { + public final synchronized OffsetBasedSource splitAtFraction(double fraction) { if (rangeTracker.getStopPosition() == Long.MAX_VALUE) { LOG.debug( "Refusing to split unbounded OffsetBasedReader {} at fraction {}", diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java index 22cb63a6ed6e2..6aa8ac5f54863 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java @@ -72,10 +72,12 @@ public abstract class Source implements Serializable { * the current model tends to be easier to program and more efficient in practice * for iterating over sources such as files, databases etc. (rather than pure collections). * - *

{@code Reader} implementations do not need to be thread-safe; they may only be accessed - * by a single thread at once. + *

All {@code Reader} functions except {@link #getCurrentSource} do not need to be thread-safe; + * they may only be accessed by a single thread at once. However, {@link #getCurrentSource} needs + * to be thread-safe, and other functions should assume that its returned value can change + * asynchronously. * - *

Callers of {@code Readers} must obey the following access pattern: + *

Reading data from the {@link Reader} must obey the following access pattern: *

    *
  • One call to {@link Reader#start} *
    • If {@link Reader#start} returned true, any number of calls to {@code getCurrent}* @@ -88,27 +90,35 @@ public abstract class Source implements Serializable { * *

      For example, if the reader is reading a fixed set of data: *

      -   * for (boolean available = reader.start(); available; available = reader.advance()) {
      -   *   T item = reader.getCurrent();
      -   *   Instant timestamp = reader.getCurrentTimestamp();
      -   *   ...
      -   * }
      +   *   try {
      +   *     for (boolean available = reader.start(); available; available = reader.advance()) {
      +   *       T item = reader.getCurrent();
      +   *       Instant timestamp = reader.getCurrentTimestamp();
      +   *       ...
      +   *     }
      +   *   } finally {
      +   *     reader.close();
      +   *   }
          * 
      * *

      If the set of data being read is continually growing: *

      -   * boolean available = reader.start();
      -   * while (true) {
      -   *   if (available) {
      -   *     T item = reader.getCurrent();
      -   *     Instant timestamp = reader.getCurrentTimestamp();
      -   *     ...
      -   *     resetExponentialBackoff();
      -   *   } else {
      -   *     exponentialBackoff();
      +   *   try {
      +   *     boolean available = reader.start();
      +   *     while (true) {
      +   *       if (available) {
      +   *         T item = reader.getCurrent();
      +   *         Instant timestamp = reader.getCurrentTimestamp();
      +   *         ...
      +   *         resetExponentialBackoff();
      +   *       } else {
      +   *         exponentialBackoff();
      +   *       }
      +   *       available = reader.advance();
      +   *     }
      +   *   } finally {
      +   *     reader.close();
          *   }
      -   *   available = reader.advance();
      -   * }
          * 
      * *

      Note: this interface is a work-in-progress and may change. @@ -168,11 +178,14 @@ public abstract static class Reader implements AutoCloseable { public abstract void close() throws IOException; /** - * Returns a {@code Source} describing the same input that this {@code Reader} reads + * Returns a {@code Source} describing the same input that this {@code Reader} currently reads * (including items already read). * - *

      A reader created from the result of {@code getCurrentSource}, if consumed, MUST - * return the same data items as the current reader. + *

      Usually, an implementation will simply return the immutable {@link Source} object from + * which the current {@link Reader} was constructed, or delegate to the base class. + * However, when using or implementing this method on a {@link BoundedSource.BoundedReader}, + * special considerations apply, see documentation for + * {@link BoundedSource.BoundedReader#getCurrentSource}. */ public abstract Source getCurrentSource(); } From 7539d5e81e18590881bb5300ab0dba51cdc8be85 Mon Sep 17 00:00:00 2001 From: amyu Date: Mon, 31 Aug 2015 08:30:25 -0700 Subject: [PATCH 1226/1541] A series of batch and streaming pipeline examples A series of batch and streaming pipelines in a 'mobile gaming' domain that illustrate some advanced topics, including windowing and triggers ----Release Notes---- A series of batch and streaming pipelines in a 'mobile gaming' domain that illustrate some advanced topics, including windowing and triggers. [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=109997483 --- examples/README.md | 8 + examples/pom.xml | 4 + .../examples/complete/game/GameStats.java | 360 +++++++++++++++ .../complete/game/HourlyTeamScore.java | 225 ++++++++++ .../examples/complete/game/LeaderBoard.java | 274 ++++++++++++ .../examples/complete/game/UserScore.java | 286 ++++++++++++ .../complete/game/injector/Injector.java | 419 ++++++++++++++++++ .../complete/game/injector/InjectorUtils.java | 101 +++++ .../injector/RetryHttpInitializerWrapper.java | 127 ++++++ .../examples/complete/game/GameStatsTest.java | 99 +++++ .../complete/game/HourlyTeamScoreTest.java | 121 +++++ .../examples/complete/game/UserScoreTest.java | 156 +++++++ 12 files changed, 2180 insertions(+) create mode 100644 examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/GameStats.java create mode 100644 examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScore.java create mode 100644 examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/LeaderBoard.java create mode 100644 examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/UserScore.java create mode 100644 examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/Injector.java create mode 100644 examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/InjectorUtils.java create mode 100644 examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/RetryHttpInitializerWrapper.java create mode 100644 examples/src/test/java8/com/google/cloud/dataflow/examples/complete/game/GameStatsTest.java create mode 100644 examples/src/test/java8/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScoreTest.java create mode 100644 examples/src/test/java8/com/google/cloud/dataflow/examples/complete/game/UserScoreTest.java diff --git a/examples/README.md b/examples/README.md index d334200e858ca..cbcd01fc0f1c5 100644 --- a/examples/README.md +++ b/examples/README.md @@ -85,3 +85,11 @@ directory for some common and useful patterns like joining, filtering, and combi The [`complete`](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete) directory contains a few realistic end-to-end pipelines. + +See the +[Java 8](https://github.com/GoogleCloudPlatform/DataflowJavaSDK/tree/master/examples/src/main/java8/com/google/cloud/dataflow/examples) +examples as well. This directory includes a Java 8 version of the +MinimalWordCount example, as well as series of examples in a simple 'mobile +gaming' domain. This series introduces some advanced concepts and provides +additional examples of using Java 8 syntax. Other than usage of Java 8 lambda +expressions, the concepts that are used apply equally well in Java 7. diff --git a/examples/pom.xml b/examples/pom.xml index d33c610a1a32c..56b76dc54ff90 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -106,6 +106,7 @@ **/*Java8Test.java + **/game/*.java @@ -124,6 +125,7 @@ **/*Java8Test.java + **/game/*.java @@ -142,6 +144,7 @@ **/*Java8*.java + **/game/*.java @@ -160,6 +163,7 @@ **/*Java8*.java + **/game/*.java diff --git a/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/GameStats.java b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/GameStats.java new file mode 100644 index 0000000000000..9dc430f6d912f --- /dev/null +++ b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/GameStats.java @@ -0,0 +1,360 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples.complete.game; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.examples.common.DataflowExampleUtils; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.PipelineResult; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.DefaultCoder; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition; +import com.google.cloud.dataflow.sdk.io.PubsubIO; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess; +import com.google.cloud.dataflow.sdk.transforms.MapElements; +import com.google.cloud.dataflow.sdk.transforms.Mean; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SerializableComparator; +import com.google.cloud.dataflow.sdk.transforms.SimpleFunction; +import com.google.cloud.dataflow.sdk.transforms.Values; +import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterEach; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns; +import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly; +import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; + +import org.apache.avro.reflect.Nullable; +import org.joda.time.DateTimeZone; +import org.joda.time.Duration; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TimeZone; + +/** + * This class is the fourth in a series of four pipelines that tell a story in a 'gaming' + * domain, following {@link UserScore}, {@link HourlyTeamScore}, and {@link LeaderBoard}. + * New concepts: session windows and finding session duration; use of both + * singleton and non-singleton side inputs. + * + *

      This pipeline builds on the {@link LeaderBoard} functionality, and adds some "business + * intelligence" analysis: abuse detection and usage patterns. The pipeline derives the Mean user + * score sum for a window, and uses that information to identify likely spammers/robots. (The robots + * have a higher click rate than the human users). The 'robot' users are then filtered out when + * calculating the team scores. + * + *

      Additionally, user sessions are tracked: that is, we find bursts of user activity using + * session windows. Then, the mean session duration information is recorded in the context of + * subsequent fixed windowing. (This could be used to tell us what games are giving us greater + * user retention). + * + *

      Run {@link injector.Injector} to generate pubsub data for this pipeline. The Injector + * documentation provides more detail. + * + *

      To execute this pipeline using the Dataflow service, specify the pipeline configuration + * like this: + *

      {@code
      + *   --project=YOUR_PROJECT_ID
      + *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
      + *   --runner=BlockingDataflowPipelineRunner
      + *   --dataset=YOUR-DATASET
      + *   --topic=projects/YOUR-PROJECT/topics/YOUR-TOPIC
      + * }
      + * 
      + * where the BigQuery dataset you specify must already exist. The PubSub topic you specify should + * be the same topic to which the Injector is publishing. + */ +public class GameStats extends LeaderBoard { + + private static final String TIMESTAMP_ATTRIBUTE = "timestamp_ms"; + private static final Logger LOG = LoggerFactory.getLogger(GameStats.class); + + private static DateTimeFormatter fmt = + DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS") + .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST"))); + static final Duration FIVE_MINUTES = Duration.standardMinutes(5); + static final Duration TEN_MINUTES = Duration.standardMinutes(10); + + + /** + * Filter out all but those users with a high clickrate, which we will consider as 'spammy' uesrs. + * We do this by finding the mean total score per user, then using that information as a side + * input to filter out all but those user scores that are > (mean * SCORE_WEIGHT) + */ + public static class CalculateSpammyUsers + extends PTransform>, PCollection>> { + private static final Logger LOG = LoggerFactory.getLogger(CalculateSpammyUsers.class); + private static final double SCORE_WEIGHT = 2.5; + + @Override + public PCollection> apply(PCollection> userScores) { + + // Get the sum of scores for each user. + PCollection> sumScores = userScores + .apply("UserSum", Sum.integersPerKey()); + + // Extract the score from each element, and use it to find the global mean. + final PCollectionView globalMeanScore = sumScores.apply(Values.create()) + .apply(Mean.globally().asSingletonView()); + + // Filter the user sums using the global mean. + PCollection> filtered = sumScores + .apply(ParDo + .named("ProcessAndFilter") + // use the derived mean total score as a side input + .withSideInputs(globalMeanScore) + .of(new DoFn, KV>() { + private final Aggregator numSpammerUsers = + createAggregator("SpammerUsers", new Sum.SumLongFn()); + @Override + public void processElement(ProcessContext c) { + Integer score = c.element().getValue(); + Double gmc = c.sideInput(globalMeanScore); + if (score > (gmc * SCORE_WEIGHT)) { + LOG.info("user " + c.element().getKey() + " spammer score " + score + + " with mean " + gmc); + numSpammerUsers.addValue(1L); + c.output(c.element()); + } + } + })); + return filtered; + } + } + + /** + * Calculate and output an element's session duration. + */ + private static class UserSessionInfoFn extends DoFn, Integer> + implements RequiresWindowAccess { + + @Override + public void processElement(ProcessContext c) { + IntervalWindow w = (IntervalWindow) c.window(); + int duration = new Duration( + w.start(), w.end()).toPeriod().toStandardMinutes().getMinutes(); + c.output(duration); + } + } + + + /** + * Options supported by {@link GameStats}. + */ + static interface Options extends LeaderBoard.Options { + @Description("Pub/Sub topic to read from") + @Validation.Required + String getTopic(); + void setTopic(String value); + + @Description("Numeric value of fixed window duration for user analysis, in minutes") + @Default.Integer(60) + Integer getFixedWindowDuration(); + void setFixedWindowDuration(Integer value); + + @Description("Numeric value of gap between user sessions, in minutes") + @Default.Integer(5) + Integer getSessionGap(); + void setSessionGap(Integer value); + + @Description("Numeric value of fixed window for finding mean of user session duration, " + + "in minutes") + @Default.Integer(30) + Integer getUserActivityWindowDuration(); + void setUserActivityWindowDuration(Integer value); + + @Description("Prefix used for the BigQuery table names") + @Default.String("game_stats") + String getTableName(); + void setTableName(String value); + } + + /** + * Format user activity information (objects of type UserActivityInfo) and write to BigQuery. + * The constructor argument indicates the table prefix to use. + */ + public static class WriteAverageSessionLengthToBigQuery + extends PTransform, PDone> { + + private final String tablePrefix; + + public WriteAverageSessionLengthToBigQuery(String tablePrefix) { + this.tablePrefix = tablePrefix; + } + + /** + * Convert the user activity info into a BigQuery TableRow. + */ + private class BuildSessionActivityRowFn extends DoFn + implements RequiresWindowAccess { + + @Override + public void processElement(ProcessContext c) { + + IntervalWindow w = (IntervalWindow) c.window(); + + TableRow row = new TableRow() + .set("window_start", fmt.print(w.start())) + .set("mean_duration", c.element()); + c.output(row); + } + } + + /** Build the output table schema. */ + private TableSchema getMeanSchema() { + List fields = new ArrayList<>(); + fields.add(new TableFieldSchema().setName("window_start").setType("STRING")); + fields.add(new TableFieldSchema().setName("mean_duration").setType("FLOAT")); + return new TableSchema().setFields(fields); + } + + @Override + public PDone apply(PCollection userInfo) { + return userInfo + .apply(ParDo.named("ConvertToUserInfoRow").of(new BuildSessionActivityRowFn())) + .apply(BigQueryIO.Write + .to(getTable(userInfo.getPipeline(), + tablePrefix + "_mean_sessions")) + .withSchema(getMeanSchema()) + .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)); + } + } + + + public static void main(String[] args) throws Exception { + + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + // Enforce that this pipeline is always run in streaming mode. + options.setStreaming(true); + // Allow the pipeline to be cancelled automatically. + options.setRunner(DataflowPipelineRunner.class); + DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options); + Pipeline pipeline = Pipeline.create(options); + + // Read Events from Pub/Sub using custom timestamps + PCollection rawEvents = pipeline + .apply(PubsubIO.Read.timestampLabel(TIMESTAMP_ATTRIBUTE).topic(options.getTopic())) + .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn())); + + // Extract username/score pairs from the event stream + PCollection> userEvents = + rawEvents.apply("ExtractUserScore", + MapElements.via((GameActionInfo gInfo) -> KV.of(gInfo.getUser(), gInfo.getScore())) + .withOutputType(new TypeDescriptor>() {})); + + // Calculate the total score per user over fixed windows, and + // cumulative updates for late data. + final PCollectionView> spammersView = userEvents + .apply(Window.named("FixedWindowsUser") + .>into(FixedWindows.of( + Duration.standardMinutes(options.getFixedWindowDuration()))) + ) + + // Filter out everyone but those with (SCORE_WEIGHT * avg) clickrate. + // These might be robots/spammers. + .apply("CalculateSpammyUsers", new CalculateSpammyUsers()) + // Derive a view from the collection of spammer users. It will be used as a side input + // in calculating the team score sums, below. + .apply("CreateSpammersView", View.asMap()); + + // Calculate the total score per team over fixed windows, + // and emit cumulative updates for late data. Uses the side input derived above-- the set of + // suspected robots-- to filter out scores from those users from the sum. + // Write the results to BigQuery. + rawEvents + .apply(Window.named("WindowIntoFixedWindows") + .into(FixedWindows.of( + Duration.standardMinutes(options.getFixedWindowDuration()))) + ) + // Filter out the detected spammer users, using the side input derived above. + .apply(ParDo.named("FilterOutSpammers") + .withSideInputs(spammersView) + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + // If the user is not in the spammers Map, output the data element. + if (c.sideInput(spammersView).get(c.element().getUser().trim()) == null) { + c.output(c.element()); + }}})) + // Extract and sum teamname/score pairs from the event data. + .apply("ExtractTeamScore", new ExtractAndSumScore("team")) + // Write the result to BigQuery + .apply("WriteTeamSums", + new WriteScoresToBigQuery(options.getTableName(), "team", true, false)); + + // Calculate the total score for the users per session-- that is, a burst of activity + // separated by a gap from further activity. Find and record the mean session lengths. + // This information could help the game designers track the changing user engagement + // as their set of games changes. + userEvents + .apply(Window.named("WindowIntoSessions") + .>into( + Sessions.withGapDuration(Duration.standardMinutes(options.getSessionGap()))) + .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow()) + .withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()) + .apply("UserSessionSum", Sum.integersPerKey()) + // Get the duration per session. + .apply("UserSessionActivity", ParDo.of(new UserSessionInfoFn())) + + // Re-window to process groups of session sums according to when the sessions complete. + .apply(Window.named("WindowToExtractSessionMean") + .into( + FixedWindows.of(Duration.standardMinutes(options.getUserActivityWindowDuration()))) + .withAllowedLateness(Duration.standardMinutes(options.getAllowedLateness())) + .accumulatingFiredPanes()) + // Find the mean session duration in each window. + .apply(Mean.globally().withoutDefaults()) + // Write this info to a BigQuery table. + .apply("WriteAvgSessionLength", + new WriteAverageSessionLengthToBigQuery(options.getTableName())); + + + // Run the pipeline and wait for the pipeline to finish; capture cancellation requests from the + // command line. + PipelineResult result = pipeline.run(); + dataflowUtils.waitToFinish(result); + } +} diff --git a/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScore.java b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScore.java new file mode 100644 index 0000000000000..01d500341d37b --- /dev/null +++ b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScore.java @@ -0,0 +1,225 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples.complete.game; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess; +import com.google.cloud.dataflow.sdk.transforms.Filter; +import com.google.cloud.dataflow.sdk.transforms.MapElements; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SimpleFunction; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.transforms.WithTimestamps; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; + +import org.joda.time.DateTimeZone; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; + +import java.util.ArrayList; +import java.util.List; +import java.util.TimeZone; + +/** + * This class is the second in a series of four pipelines that tell a story in a 'gaming' + * domain, following {@link UserScore}. In addition to the concepts introduced in {@ UserScore}, new + * concepts include: windowing and element timestamps; use of {@code Filter.byPredicate()}. + * + *

      This pipeline processes data collected from gaming events in batch, building on {@link + * UserScore} but using fixed windows. It calculates the sum of scores per team, for each window, + * optionally allowing specification of two timestamps before and after which data is filtered out. + * This allows a model where late data collected after the intended analysis window can be included, + * and any late-arriving data prior to the beginning of the analysis window can be removed as well. + * By using windowing and adding element timestamps, we can do finer-grained analysis than with the + * {@link UserScore} pipeline. However, our batch processing is high-latency, in that we don't get + * results from plays at the beginning of the batch's time period until the batch is processed. + * + *

      To execute this pipeline using the Dataflow service, specify the pipeline configuration + * like this: + *

      {@code
      + *   --project=YOUR_PROJECT_ID
      + *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
      + *   --runner=BlockingDataflowPipelineRunner
      + *   --dataset=YOUR-DATASET
      + * }
      + * 
      + * where the BigQuery dataset you specify must already exist. + * + *

      Optionally include {@code --input} to specify the batch input file path. + * To indicate a time after which the data should be filtered out, include the + * {@code --stopMin} arg. E.g., {@code --stopMin=2015-10-18-23-59} indicates that any data + * timestamped after 23:59 PST on 2015-10-18 should not be included in the analysis. + * To indicate a time before which data should be filtered out, include the {@code --startMin} arg. + * If you're using the default input specified in {@link UserScore}, + * "gs://dataflow-samples/game/gaming_data*.csv", then + * {@code --startMin=2015-11-16-16-10 --stopMin=2015-11-17-16-10} are good values. + */ +public class HourlyTeamScore extends UserScore { + + private static DateTimeFormatter fmt = + DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS") + .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST"))); + private static DateTimeFormatter minFmt = + DateTimeFormat.forPattern("yyyy-MM-dd-HH-mm") + .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST"))); + + + /** Format fixed window information for scores, and write that info to BigQuery. */ + public static class WriteWindowedToBigQuery + extends PTransform>, PDone> { + + private final String tableName; + + public WriteWindowedToBigQuery(String tableName) { + this.tableName = tableName; + } + + /** Convert each key/score pair into a BigQuery TableRow. */ + private class BuildFixedRowFn extends DoFn, TableRow> + implements RequiresWindowAccess { + + @Override + public void processElement(ProcessContext c) { + + IntervalWindow w = (IntervalWindow) c.window(); + + TableRow row = new TableRow() + .set("team", c.element().getKey()) + .set("total_score", c.element().getValue().longValue()) + // Add windowing info to the output. + .set("window_start", fmt.print(w.start())); + c.output(row); + } + } + + /** Build the output table schema. */ + private TableSchema getFixedSchema() { + List fields = new ArrayList<>(); + fields.add(new TableFieldSchema().setName("team").setType("STRING")); + fields.add(new TableFieldSchema().setName("total_score").setType("INTEGER")); + fields.add(new TableFieldSchema().setName("window_start").setType("STRING")); + return new TableSchema().setFields(fields); + } + + @Override + public PDone apply(PCollection> teamAndScore) { + return teamAndScore + .apply(ParDo.named("ConvertToFixedRow").of(new BuildFixedRowFn())) + .apply(BigQueryIO.Write + .to(getTable(teamAndScore.getPipeline(), + tableName)) + .withSchema(getFixedSchema()) + .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)); + } + } + + + /** + * Options supported by {@link HourlyTeamScore}. + */ + static interface Options extends UserScore.Options { + + @Description("Numeric value of fixed window duration, in minutes") + @Default.Integer(60) + Integer getWindowDuration(); + void setWindowDuration(Integer value); + + @Description("String representation of the first minute after which to generate results," + + "in the format: yyyy-MM-dd-HH-mm . This time should be in PST." + + "Any input data timestamped prior to that minute won't be included in the sums.") + @Default.String("1970-01-01-00-00") + String getStartMin(); + void setStartMin(String value); + + @Description("String representation of the first minute for which to not generate results," + + "in the format: yyyy-MM-dd-HH-mm . This time should be in PST." + + "Any input data timestamped after that minute won't be included in the sums.") + @Default.String("2100-01-01-00-00") + String getStopMin(); + void setStopMin(String value); + + @Description("The BigQuery table name. Should not already exist.") + @Default.String("hourly_team_score") + String getTableName(); + void setTableName(String value); + } + + + /** + * Run a batch pipeline to do windowed analysis of the data. + */ + public static void main(String[] args) throws Exception { + // Begin constructing a pipeline configured by commandline flags. + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + Pipeline pipeline = Pipeline.create(options); + + final Instant stopMinTimestamp = new Instant(minFmt.parseMillis(options.getStopMin())); + final Instant startMinTimestamp = new Instant(minFmt.parseMillis(options.getStartMin())); + + // Read 'gaming' events from a text file. + pipeline.apply(TextIO.Read.from(options.getInput())) + // Parse the incoming data. + .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn())) + + // Filter out data before and after the given times so that it is not included + // in the calculations. As we collect data in batches (say, by day), the batch for the day + // that we want to analyze could potentially include some late-arriving data from the previous + // day. If so, we want to weed it out. Similarly, if we include data from the following day + // (to scoop up late-arriving events from the day we're analyzing), we need to weed out events + // that fall after the time period we want to analyze. + .apply("FilterStartTime", Filter.byPredicate( + (GameActionInfo gInfo) + -> gInfo.getTimestamp() > startMinTimestamp.getMillis())) + .apply("FilterEndTime", Filter.byPredicate( + (GameActionInfo gInfo) + -> gInfo.getTimestamp() < stopMinTimestamp.getMillis())) + + // Add an element timestamp based on the event log, and apply fixed windowing. + .apply("AddEventTimestamps", + WithTimestamps.of((GameActionInfo i) -> new Instant(i.getTimestamp()))) + .apply(Window.named("FixedWindowsTeam") + .into(FixedWindows.of( + Duration.standardMinutes(options.getWindowDuration())))) + + // Extract and sum teamname/score pairs from the event data. + .apply("ExtractTeamScore", new ExtractAndSumScore("team")) + .apply("WriteTeamScoreSums", new WriteWindowedToBigQuery(options.getTableName())); + + pipeline.run(); + } + +} diff --git a/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/LeaderBoard.java b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/LeaderBoard.java new file mode 100644 index 0000000000000..6bb236668b9b1 --- /dev/null +++ b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/LeaderBoard.java @@ -0,0 +1,274 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples.complete.game; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.examples.common.DataflowExampleOptions; +import com.google.cloud.dataflow.examples.common.DataflowExampleUtils; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.PipelineResult; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition; +import com.google.cloud.dataflow.sdk.io.PubsubIO; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.StreamingOptions; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess; +import com.google.cloud.dataflow.sdk.transforms.MapElements; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SimpleFunction; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterEach; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; + + +import org.joda.time.DateTimeZone; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; + +import java.util.ArrayList; +import java.util.List; +import java.util.TimeZone; + +/** + * This class is the third in a series of four pipelines that tell a story in a 'gaming' domain, + * following {@link UserScore} and {@link HourlyTeamScore}. Concepts include: processing unbounded + * data using fixed windows; use of custom timestamps and event-time processing; generation of + * early/speculative results; using .accumulatingFiredPanes() to do cumulative processing of late- + * arriving data. + * + *

      This pipeline processes an unbounded stream of 'game events'. The calculation of the team + * scores uses fixed windowing based on event time (the time of the game play event), not + * processing time (the time that an event is processed by the pipeline). The pipeline calculates + * the sum of scores per team, for each window. By default, the team scores are calculated using + * one-hour windows. + * + *

      In contrast-- to demo another windowing option-- the user scores are calculated using a + * global window, which periodically (every ten minutes) emits cumulative user score sums. + * + *

      In contrast to the previous pipelines in the series, which used static, finite input data, + * here we're using an unbounded data source, which lets us provide speculative results, and allows + * handling of late data, at much lower latency. We can use the early/speculative results to keep a + * 'leaderboard' updated in near-realtime. Our handling of late data lets us generate correct + * results, e.g. for 'team prizes'. We're now outputing window results as they're + * calculated, giving us much lower latency than with the previous batch examples. + * + *

      Run {@link injector.Injector} to generate pubsub data for this pipeline. The Injector + * documentation provides more detail on how to do this. + * + *

      To execute this pipeline using the Dataflow service, specify the pipeline configuration + * like this: + *

      {@code
      + *   --project=YOUR_PROJECT_ID
      + *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
      + *   --runner=BlockingDataflowPipelineRunner
      + *   --dataset=YOUR-DATASET
      + *   --topic=projects/YOUR-PROJECT/topics/YOUR-TOPIC
      + * }
      + * 
      + * where the BigQuery dataset you specify must already exist. + * The PubSub topic you specify should be the same topic to which the Injector is publishing. + */ +public class LeaderBoard extends HourlyTeamScore { + + private static final String TIMESTAMP_ATTRIBUTE = "timestamp_ms"; + + private static DateTimeFormatter fmt = + DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS") + .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST"))); + static final Duration FIVE_MINUTES = Duration.standardMinutes(5); + static final Duration TEN_MINUTES = Duration.standardMinutes(10); + + + /** + * Format information for scores, and write that info to BigQuery. + * Optionally include fixed windowing information and timing in the result. + */ + public static class WriteScoresToBigQuery + extends PTransform>, PDone> { + + private final String fieldName; + private final String tablePrefix; + private final boolean writeTiming; // Whether to write timing info to the resultant table. + private final boolean writeWindowStart; // whether to include window start info. + + public WriteScoresToBigQuery(String tablePrefix, String fieldName, + boolean writeWindowStart, boolean writeTiming) { + this.fieldName = fieldName; + this.tablePrefix = tablePrefix; + this.writeWindowStart = writeWindowStart; + this.writeTiming = writeTiming; + } + + /** Convert each key/score pair into a BigQuery TableRow. */ + private class BuildFixedRowFn extends DoFn, TableRow> + implements RequiresWindowAccess { + + @Override + public void processElement(ProcessContext c) { + + // IntervalWindow w = (IntervalWindow) c.window(); + + TableRow row = new TableRow() + .set(fieldName, c.element().getKey()) + .set("total_score", c.element().getValue().longValue()) + .set("processing_time", fmt.print(Instant.now())); + if (writeWindowStart) { + IntervalWindow w = (IntervalWindow) c.window(); + row.set("window_start", fmt.print(w.start())); + } + if (writeTiming) { + row.set("timing", c.pane().getTiming().toString()); + } + c.output(row); + } + } + + /** Build the output table schema. */ + private TableSchema getFixedSchema() { + List fields = new ArrayList<>(); + fields.add(new TableFieldSchema().setName(fieldName).setType("STRING")); + fields.add(new TableFieldSchema().setName("total_score").setType("INTEGER")); + fields.add(new TableFieldSchema().setName("processing_time").setType("STRING")); + if (writeWindowStart) { + fields.add(new TableFieldSchema().setName("window_start").setType("STRING")); + } + if (writeTiming) { + fields.add(new TableFieldSchema().setName("timing").setType("STRING")); + } + return new TableSchema().setFields(fields); + } + + @Override + public PDone apply(PCollection> teamAndScore) { + return teamAndScore + .apply(ParDo.named("ConvertToFixedTriggersRow").of(new BuildFixedRowFn())) + .apply(BigQueryIO.Write + .to(getTable(teamAndScore.getPipeline(), + tablePrefix + "_" + fieldName)) + .withSchema(getFixedSchema()) + .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)); + } + } + + + /** + * Options supported by {@link LeaderBoard}. + */ + static interface Options extends HourlyTeamScore.Options, DataflowExampleOptions { + + @Description("Pub/Sub topic to read from") + @Validation.Required + String getTopic(); + void setTopic(String value); + + @Description("Numeric value of fixed window duration for team analysis, in minutes") + @Default.Integer(60) + Integer getTeamWindowDuration(); + void setTeamWindowDuration(Integer value); + + @Description("Numeric value of allowed data lateness, in minutes") + @Default.Integer(120) + Integer getAllowedLateness(); + void setAllowedLateness(Integer value); + + @Description("Prefix used for the BigQuery table names") + @Default.String("leaderboard") + String getTableName(); + void setTableName(String value); + } + + + public static void main(String[] args) throws Exception { + + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + // Enforce that this pipeline is always run in streaming mode. + options.setStreaming(true); + // For example purposes, allow the pipeline to be easily cancelled instead of running + // continuously. + options.setRunner(DataflowPipelineRunner.class); + DataflowExampleUtils dataflowUtils = new DataflowExampleUtils(options); + Pipeline pipeline = Pipeline.create(options); + + // Read game events from Pub/Sub using custom timestamps, which are extracted from the pubsub + // data elements, and parse the data. + PCollection gameEvents = pipeline + .apply(PubsubIO.Read.timestampLabel(TIMESTAMP_ATTRIBUTE).topic(options.getTopic())) + .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn())); + + // Extract team/score pairs from the event stream, using hour-long windows by default. + gameEvents + .apply(Window.named("LeaderboardTeamFixedWindows") + .into(FixedWindows.of( + Duration.standardMinutes(options.getTeamWindowDuration()))) + // We will get early (speculative) results as well as cumulative + // processing of late data. + .triggering( + AfterWatermark.pastEndOfWindow() + .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(FIVE_MINUTES)) + .withLateFirings(AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(TEN_MINUTES))) + .withAllowedLateness(Duration.standardMinutes(options.getAllowedLateness())) + .accumulatingFiredPanes()) + // Extract and sum teamname/score pairs from the event data. + .apply("ExtractTeamScore", new ExtractAndSumScore("team")) + // Write the results to BigQuery. + .apply("WriteTeamScoreSums", + new WriteScoresToBigQuery(options.getTableName(), "team", true, true)); + + // Extract user/score pairs from the event stream using processing time, via global windowing. + // Get periodic updates on all users' running scores. + gameEvents + .apply(Window.named("LeaderboardUserGlobalWindow") + .into(new GlobalWindows()) + // Get periodic results every ten minutes. + .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(TEN_MINUTES))) + .accumulatingFiredPanes() + .withAllowedLateness(Duration.standardMinutes(options.getAllowedLateness()))) + // Extract and sum username/score pairs from the event data. + .apply("ExtractUserScore", new ExtractAndSumScore("user")) + // Write the results to BigQuery. + .apply("WriteUserScoreSums", + new WriteScoresToBigQuery(options.getTableName(), "user", false, false)); + + // Run the pipeline and wait for the pipeline to finish; capture cancellation requests from the + // command line. + PipelineResult result = pipeline.run(); + dataflowUtils.waitToFinish(result); + } +} diff --git a/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/UserScore.java b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/UserScore.java new file mode 100644 index 0000000000000..78e7cb8c63490 --- /dev/null +++ b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/UserScore.java @@ -0,0 +1,286 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples.complete.game; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.DefaultCoder; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.GcpOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.MapElements; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SimpleFunction; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; + +import org.apache.avro.reflect.Nullable; +import org.joda.time.DateTimeZone; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.TimeZone; + +/** + * This class is the first in a series of four pipelines that tell a story in a 'gaming' domain. + * Concepts: batch processing; reading input from Google Cloud Storage and writing output to + * BigQuery; using standalone DoFns; use of the sum by key transform; examples of + * Java 8 lambda syntax. + * + *

      In this gaming scenario, many users play, as members of different teams, over the course of a + * day, and their actions are logged for processing. Some of the logged game events may be late- + * arriving, if users play on mobile devices and go transiently offline for a period. + * + *

      This pipeline does batch processing of data collected from gaming events. It calculates the + * sum of scores per user, over an entire batch of gaming data (collected, say, for each day). The + * batch processing will not include any late data that arrives after the day's cutoff point. + * + *

      To execute this pipeline using the Dataflow service and static example input data, specify + * the pipeline configuration like this: + *

      {@code
      + *   --project=YOUR_PROJECT_ID
      + *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
      + *   --runner=BlockingDataflowPipelineRunner
      + *   --dataset=YOUR-DATASET
      + * }
      + * 
      + * where the BigQuery dataset you specify must already exist. + * + *

      Optionally include the --input argument to specify a batch input file. + * See the --input default value for example batch data file, or use {@link injector.Injector} to + * generate your own batch data. + */ +public class UserScore { + + /** + * Class to hold info about a game event. + */ + @DefaultCoder(AvroCoder.class) + static class GameActionInfo { + @Nullable String user; + @Nullable String team; + @Nullable Integer score; + @Nullable Long timestamp; + + public GameActionInfo() {} + + public GameActionInfo(String user, String team, Integer score, Long timestamp) { + this.user = user; + this.team = team; + this.score = score; + this.timestamp = timestamp; + } + + public String getUser() { + return this.user; + } + public String getTeam() { + return this.team; + } + public Integer getScore() { + return this.score; + } + public String getKey(String keyname) { + if (keyname.equals("team")) { + return this.team; + } else { // return username as default + return this.user; + } + } + public Long getTimestamp() { + return this.timestamp; + } + } + + + /** + * Parses the raw game event info into GameActionInfo objects. Each event line has the following + * format: username,teamname,score,timestamp_in_ms,readable_time + * e.g.: + * user2_AsparagusPig,AsparagusPig,10,1445230923951,2015-11-02 09:09:28.224 + * The human-readable time string is not used here. + */ + static class ParseEventFn extends DoFn { + + // Log and count parse errors. + private static final Logger LOG = LoggerFactory.getLogger(ParseEventFn.class); + private final Aggregator numParseErrors = + createAggregator("ParseErrors", new Sum.SumLongFn()); + + @Override + public void processElement(ProcessContext c) { + String[] components = c.element().split(","); + try { + String user = components[0].trim(); + String team = components[1].trim(); + Integer score = Integer.parseInt(components[2].trim()); + Long timestamp = Long.parseLong(components[3].trim()); + GameActionInfo gInfo = new GameActionInfo(user, team, score, timestamp); + c.output(gInfo); + } catch (ArrayIndexOutOfBoundsException | NumberFormatException e) { + numParseErrors.addValue(1L); + LOG.info("Parse error on " + c.element() + ", " + e.getMessage()); + } + } + } + + /** + * A transform to extract key/score information from GameActionInfo, and sum the scores. The + * constructor arg determines whether 'team' or 'user' info is extracted. + */ + public static class ExtractAndSumScore + extends PTransform, PCollection>> { + + private final String field; + + ExtractAndSumScore(String field) { + this.field = field; + } + + @Override + public PCollection> apply( + PCollection gameInfo) { + + return gameInfo + .apply(MapElements + .via((GameActionInfo gInfo) -> KV.of(gInfo.getKey(field), gInfo.getScore())) + .withOutputType(new TypeDescriptor>() {})) + .apply(Sum.integersPerKey()); + } + } + + + /** + * Format information for key/total_score pairs, and write that info to BigQuery. + */ + public static class WriteToBigQuery + extends PTransform>, PDone> { + + private final String tableName; + + public WriteToBigQuery(String tableName) { + this.tableName = tableName; + } + + /** Convert each key/score pair into a BigQuery TableRow. */ + private class BuildRowFn extends DoFn, TableRow> { + + @Override + public void processElement(ProcessContext c) { + + TableRow row = new TableRow() + .set("user", c.element().getKey()) + .set("total_score", c.element().getValue().longValue()); + c.output(row); + } + } + + /** Build the output table schema. */ + private TableSchema getSchema() { + List fields = new ArrayList<>(); + fields.add(new TableFieldSchema().setName("user").setType("STRING")); + fields.add(new TableFieldSchema().setName("total_score").setType("INTEGER")); + return new TableSchema().setFields(fields); + } + + @Override + public PDone apply(PCollection> teamAndScore) { + return teamAndScore + .apply(ParDo.named("ConvertToRow").of(new BuildRowFn())) + .apply(BigQueryIO.Write + .to(getTable(teamAndScore.getPipeline(), + tableName)) + .withSchema(getSchema()) + .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)); + } + } + + /** Utility to construct an output table reference. */ + static TableReference getTable(Pipeline pipeline, String tableName) { + PipelineOptions options = pipeline.getOptions(); + TableReference table = new TableReference(); + table.setDatasetId(options.as(Options.class).getDataset()); + table.setProjectId(options.as(GcpOptions.class).getProject()); + table.setTableId(tableName); + return table; + } + + /** + * Options supported by {@link UserScore}. + */ + static interface Options extends PipelineOptions { + + @Description("Path to the data file(s) containing game data.") + // The default maps to two large Google Cloud Storage files (each ~12GB) holding two subsequent + // day's worth (roughly) of data. + @Default.String("gs://dataflow-samples/game/gaming_data*.csv") + String getInput(); + void setInput(String value); + + @Description("BigQuery Dataset to write tables to. Must already exist.") + @Validation.Required + String getDataset(); + void setDataset(String value); + + @Description("The BigQuery table name. Should not already exist.") + @Default.String("user_score") + String getTableName(); + void setTableName(String value); + } + + + + /** + * Run a batch pipeline. + */ + public static void main(String[] args) throws Exception { + // Begin constructing a pipeline configured by commandline flags. + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + Pipeline pipeline = Pipeline.create(options); + + // Read events from a text file and parse them. + pipeline.apply(TextIO.Read.from(options.getInput())) + .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn())) + // Extract and sum username/score pairs from the event data. + .apply("ExtractUserScore", new ExtractAndSumScore("user")) + .apply("WriteUserScoreSums", new WriteToBigQuery(options.getTableName())); + + // Run the batch pipeline. + pipeline.run(); + } + +} diff --git a/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/Injector.java b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/Injector.java new file mode 100644 index 0000000000000..12bd5b63dfe0d --- /dev/null +++ b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/Injector.java @@ -0,0 +1,419 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples.complete.game.injector; + +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.pubsub.model.PublishRequest; +import com.google.api.services.pubsub.model.PubsubMessage; + +import com.google.common.collect.ImmutableMap; +import org.joda.time.DateTimeZone; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; +import java.io.BufferedOutputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.PrintWriter; +import java.security.GeneralSecurityException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Random; +import java.util.TimeZone; +import java.util.UUID; + + +/** + * This is a generator that simulates usage data from a mobile game, and either publishes the data + * to a pubsub topic or writes it to a file. + * + *

      The general model used by the generator is the following. There is a set of teams with team + * members. Each member is scoring points for their team. After some period, a team will dissolve + * and a new one will be created in its place. There is also a set of 'Robots', or spammer users. + * They hop from team to team. The robots are set to have a higher 'click rate' (generate more + * events) than the regular team members. + * + *

      Each generated line of data has the following form: + * username,teamname,score,timestamp_in_ms,readable_time + * e.g.: + * user2_AsparagusPig,AsparagusPig,10,1445230923951,2015-11-02 09:09:28.224 + * + *

      The Injector writes either to a PubSub topic, or a file. It will use the PubSub topic if + * specified. It takes the following arguments: + * {@code Injector project-name (topic-name|none) (filename|none)}. + * + *

      To run the Injector in the mode where it publishes to PubSub, you will need to authenticate + * locally using project-based service account credentials to avoid running over PubSub + * quota. + * See https://developers.google.com/identity/protocols/application-default-credentials + * for more information on using service account credentials. Set the GOOGLE_APPLICATION_CREDENTIALS + * environment variable to point to your downloaded service account credentials before starting the + * program, e.g.: + * {@code export GOOGLE_APPLICATION_CREDENTIALS=/path/to/your/credentials-key.json}. + * If you do not do this, then your injector will only run for a few minutes on your + * 'user account' credentials before you will start to see quota error messages like: + * "Request throttled due to user QPS limit being reached", and see this exception: + * ".com.google.api.client.googleapis.json.GoogleJsonResponseException: 429 Too Many Requests". + * Once you've set up your credentials, run the Injector like this": + *

      {@code
      + * Injector   none
      + * }
      + * 
      + * The pubsub topic will be created if it does not exist. + * + *

      To run the injector in write-to-file-mode, set the topic name to "none" and specify the + * filename: + *

      {@code
      + * Injector  none 
      + * }
      + * 
      + */ +class Injector { + private static Pubsub pubsub; + private static Random random = new Random(); + private static String topic; + private static String project; + private static final String TIMESTAMP_ATTRIBUTE = "timestamp_ms"; + + // QPS ranges from 800 to 1000. + private static final int MIN_QPS = 800; + private static final int QPS_RANGE = 200; + // How long to sleep, in ms, between creation of the threads that make API requests to PubSub. + private static final int THREAD_SLEEP_MS = 500; + + // Lists used to generate random team names. + private static final ArrayList COLORS = + new ArrayList(Arrays.asList( + "Magenta", "AliceBlue", "Almond", "Amaranth", "Amber", + "Amethyst", "AndroidGreen", "AntiqueBrass", "Fuchsia", "Ruby", "AppleGreen", + "Apricot", "Aqua", "ArmyGreen", "Asparagus", "Auburn", "Azure", "Banana", + "Beige", "Bisque", "BarnRed", "BattleshipGrey")); + + private static final ArrayList ANIMALS = + new ArrayList(Arrays.asList( + "Echidna", "Koala", "Wombat", "Marmot", "Quokka", "Kangaroo", "Dingo", "Numbat", "Emu", + "Wallaby", "CaneToad", "Bilby", "Possum", "Cassowary", "Kookaburra", "Platypus", + "Bandicoot", "Cockatoo", "Antechinus")); + + // The list of live teams. + private static ArrayList liveTeams = new ArrayList(); + + private static DateTimeFormatter fmt = + DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS") + .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST"))); + + + // The total number of robots in the system. + private static final int NUM_ROBOTS = 20; + // Determines the chance that a team will have a robot team member. + private static final int ROBOT_PROBABILITY = 3; + private static final int NUM_LIVE_TEAMS = 15; + private static final int BASE_MEMBERS_PER_TEAM = 5; + private static final int MEMBERS_PER_TEAM = 15; + private static final int MAX_SCORE = 20; + private static final int LATE_DATA_RATE = 5 * 60 * 2; // Every 10 minutes + private static final int BASE_DELAY_IN_MILLIS = 5 * 60 * 1000; // 5-10 minute delay + private static final int FUZZY_DELAY_IN_MILLIS = 5 * 60 * 1000; + + // The minimum time a 'team' can live. + private static final int BASE_TEAM_EXPIRATION_TIME_IN_MINS = 20; + private static final int TEAM_EXPIRATION_TIME_IN_MINS = 20; + + + /** + * A class for holding team info: the name of the team, when it started, + * and the current team members. Teams may but need not include one robot team member. + */ + private static class TeamInfo { + String teamName; + long startTimeInMillis; + int expirationPeriod; + // The team might but need not include 1 robot. Will be non-null if so. + String robot; + int numMembers; + + private TeamInfo(String teamName, long startTimeInMillis, String robot) { + this.teamName = teamName; + this.startTimeInMillis = startTimeInMillis; + // How long until this team is dissolved. + this.expirationPeriod = random.nextInt(TEAM_EXPIRATION_TIME_IN_MINS) + + BASE_TEAM_EXPIRATION_TIME_IN_MINS; + this.robot = robot; + // Determine the number of team members. + numMembers = random.nextInt(MEMBERS_PER_TEAM) + BASE_MEMBERS_PER_TEAM; + } + + String getTeamName() { + return teamName; + } + String getRobot() { + return robot; + } + + long getStartTimeInMillis() { + return startTimeInMillis; + } + long getEndTimeInMillis() { + return startTimeInMillis + (expirationPeriod * 60 * 1000); + } + String getRandomUser() { + int userNum = random.nextInt(numMembers); + return "user" + userNum + "_" + teamName; + } + + int numMembers() { + return numMembers; + } + + @Override + public String toString() { + return "(" + teamName + ", num members: " + numMembers() + ", starting at: " + + startTimeInMillis + ", expires in: " + expirationPeriod + ", robot: " + robot + ")"; + } + } + + /** Utility to grab a random element from an array of Strings. */ + private static String randomElement(ArrayList list) { + int index = random.nextInt(list.size()); + return list.get(index); + } + + /** + * Get and return a random team. If the selected team is too old w.r.t its expiration, remove + * it, replacing it with a new team. + */ + private static TeamInfo randomTeam(ArrayList list) { + int index = random.nextInt(list.size()); + TeamInfo team = list.get(index); + // If the selected team is expired, remove it and return a new team. + long currTime = System.currentTimeMillis(); + if ((team.getEndTimeInMillis() < currTime) || team.numMembers() == 0) { + System.out.println("\nteam " + team + " is too old; replacing."); + System.out.println("start time: " + team.getStartTimeInMillis() + + ", end time: " + team.getEndTimeInMillis() + + ", current time:" + currTime); + removeTeam(index); + // Add a new team in its stead. + return (addLiveTeam()); + } else { + return team; + } + } + + /** + * Create and add a team. Possibly add a robot to the team. + */ + private static synchronized TeamInfo addLiveTeam() { + String teamName = randomElement(COLORS) + randomElement(ANIMALS); + String robot = null; + // Decide if we want to add a robot to the team. + if (random.nextInt(ROBOT_PROBABILITY) == 0) { + robot = "Robot-" + random.nextInt(NUM_ROBOTS); + } + long currTime = System.currentTimeMillis(); + // Create the new team. + TeamInfo newTeam = new TeamInfo(teamName, System.currentTimeMillis(), robot); + liveTeams.add(newTeam); + System.out.println("[+" + newTeam + "]"); + return newTeam; + } + + /** + * Remove a specific team. + */ + private static synchronized void removeTeam(int teamIndex) { + TeamInfo removedTeam = liveTeams.remove(teamIndex); + System.out.println("[-" + removedTeam + "]"); + } + + /** Generate a user gaming event. */ + private static String generateEvent(Long currTime, int delayInMillis) { + TeamInfo team = randomTeam(liveTeams); + String teamName = team.getTeamName(); + String user; + int PARSE_ERROR_RATE = 900000; + + String robot = team.getRobot(); + // If the team has an associated robot team member... + if (robot != null) { + // Then use that robot for the message with some probability. + // Set this probability to higher than that used to select any of the 'regular' team + // members, so that if there is a robot on the team, it has a higher click rate. + if (random.nextInt(team.numMembers() / 2) == 0) { + user = robot; + } else { + user = team.getRandomUser(); + } + } else { // No robot. + user = team.getRandomUser(); + } + String event = user + "," + teamName + "," + random.nextInt(MAX_SCORE); + // Randomly introduce occasional parse errors. You can see a custom counter tracking the number + // of such errors in the Dataflow Monitoring UI, as the example pipeline runs. + if (random.nextInt(PARSE_ERROR_RATE) == 0) { + System.out.println("Introducing a parse error."); + event = "THIS LINE REPRESENTS CORRUPT DATA AND WILL CAUSE A PARSE ERROR"; + } + return addTimeInfoToEvent(event, currTime, delayInMillis); + } + + /** + * Add time info to a generated gaming event. + */ + private static String addTimeInfoToEvent(String message, Long currTime, int delayInMillis) { + String eventTimeString = + Long.toString((currTime - delayInMillis) / 1000 * 1000); + // Add a (redundant) 'human-readable' date string to make the data semantics more clear. + String dateString = fmt.print(currTime); + message = message + "," + eventTimeString + "," + dateString; + return message; + } + + /** + * Publish 'numMessages' arbitrary events from live users with the provided delay, to a + * PubSub topic. + */ + public static void publishData(int numMessages, int delayInMillis) + throws IOException { + List pubsubMessages = new ArrayList<>(); + + for (int i = 0; i < Math.max(1, numMessages); i++) { + Long currTime = System.currentTimeMillis(); + String message = generateEvent(currTime, delayInMillis); + PubsubMessage pubsubMessage = new PubsubMessage() + .encodeData(message.getBytes("UTF-8")); + pubsubMessage.setAttributes( + ImmutableMap.of(TIMESTAMP_ATTRIBUTE, + Long.toString((currTime - delayInMillis) / 1000 * 1000))); + if (delayInMillis != 0) { + System.out.println(pubsubMessage.getAttributes()); + System.out.println("late data for: " + message); + } + pubsubMessages.add(pubsubMessage); + } + + PublishRequest publishRequest = new PublishRequest(); + publishRequest.setMessages(pubsubMessages); + pubsub.projects().topics().publish(topic, publishRequest).execute(); + } + + /** + * Publish generated events to a file. + */ + public static void publishDataToFile(String fileName, int numMessages, int delayInMillis) + throws IOException { + List pubsubMessages = new ArrayList<>(); + PrintWriter out = new PrintWriter(new OutputStreamWriter( + new BufferedOutputStream(new FileOutputStream(fileName, true)), "UTF-8")); + + try { + for (int i = 0; i < Math.max(1, numMessages); i++) { + Long currTime = System.currentTimeMillis(); + String message = generateEvent(currTime, delayInMillis); + out.println(message); + } + } catch (Exception e) { + e.printStackTrace(); + } finally { + if (out != null) { + out.flush(); + out.close(); + } + } + } + + + public static void main(String[] args) + throws GeneralSecurityException, IOException, InterruptedException { + if (args.length < 3) { + System.out.println("Usage: Injector project-name (topic-name|none) (filename|none)"); + System.exit(1); + } + boolean writeToFile = false; + boolean writeToPubsub = true; + project = args[0]; + String topicName = args[1]; + String fileName = args[2]; + // The Injector writes either to a PubSub topic, or a file. It will use the PubSub topic if + // specified; otherwise, it will try to write to a file. + if (topicName.equalsIgnoreCase("none")) { + writeToFile = true; + writeToPubsub = false; + } + if (writeToPubsub) { + // Create the PubSub client. + pubsub = InjectorUtils.getClient(); + // Create the PubSub topic as necessary. + topic = InjectorUtils.getFullyQualifiedTopicName(project, topicName); + InjectorUtils.createTopic(pubsub, topic); + System.out.println("Injecting to topic: " + topic); + } else { + if (fileName.equalsIgnoreCase("none")) { + System.out.println("Filename not specified."); + System.exit(1); + } + System.out.println("Writing to file: " + fileName); + } + System.out.println("Starting Injector"); + + // Start off with some random live teams. + while (liveTeams.size() < NUM_LIVE_TEAMS) { + addLiveTeam(); + } + + // Publish messages at a rate determined by the QPS and Thread sleep settings. + for (int i = 0; true; i++) { + if (Thread.activeCount() > 10) { + System.err.println("I'm falling behind!"); + } + + // Decide if this should be a batch of late data. + final int numMessages; + final int delayInMillis; + if (i % LATE_DATA_RATE == 0) { + // Insert delayed data for one user (one message only) + delayInMillis = BASE_DELAY_IN_MILLIS + random.nextInt(FUZZY_DELAY_IN_MILLIS); + numMessages = 1; + System.out.println("DELAY(" + delayInMillis + ", " + numMessages + ")"); + } else { + System.out.print("."); + delayInMillis = 0; + numMessages = MIN_QPS + random.nextInt(QPS_RANGE); + } + + if (writeToFile) { // Won't use threading for the file write. + publishDataToFile(fileName, numMessages, delayInMillis); + } else { // Write to PubSub. + // Start a thread to inject some data. + new Thread(){ + public void run() { + try { + publishData(numMessages, delayInMillis); + } catch (IOException e) { + System.err.println(e); + } + } + }.start(); + } + + // Wait before creating another injector thread. + Thread.sleep(THREAD_SLEEP_MS); + } + } +} diff --git a/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/InjectorUtils.java b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/InjectorUtils.java new file mode 100644 index 0000000000000..06c38646274f2 --- /dev/null +++ b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/InjectorUtils.java @@ -0,0 +1,101 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples.complete.game.injector; + + +import com.google.api.client.googleapis.auth.oauth2.GoogleCredential; +import com.google.api.client.googleapis.json.GoogleJsonResponseException; +import com.google.api.client.googleapis.util.Utils; +import com.google.api.client.http.HttpRequestInitializer; +import com.google.api.client.http.HttpStatusCodes; +import com.google.api.client.http.HttpTransport; +import com.google.api.client.json.JsonFactory; +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.pubsub.PubsubScopes; +import com.google.api.services.pubsub.model.Topic; + +import com.google.common.base.Preconditions; + +import java.io.IOException; + +class InjectorUtils { + + private static final String APP_NAME = "injector"; + + /** + * Builds a new Pubsub client and returns it. + */ + public static Pubsub getClient(final HttpTransport httpTransport, + final JsonFactory jsonFactory) + throws IOException { + Preconditions.checkNotNull(httpTransport); + Preconditions.checkNotNull(jsonFactory); + GoogleCredential credential = + GoogleCredential.getApplicationDefault(httpTransport, jsonFactory); + if (credential.createScopedRequired()) { + credential = credential.createScoped(PubsubScopes.all()); + } + if (credential.getServiceAccountId() == null) { + System.out.println("\n***Warning! You are not using service account credentials to " + + "authenticate.\nYou need to use service account credentials for this example," + + "\nsince user-level credentials do not have enough pubsub quota,\nand so you will run " + + "out of PubSub quota very quickly.\nSee " + + "https://developers.google.com/identity/protocols/application-default-credentials."); + System.exit(1); + } + HttpRequestInitializer initializer = + new RetryHttpInitializerWrapper(credential); + return new Pubsub.Builder(httpTransport, jsonFactory, initializer) + .setApplicationName(APP_NAME) + .build(); + } + + /** + * Builds a new Pubsub client with default HttpTransport and + * JsonFactory and returns it. + */ + public static Pubsub getClient() throws IOException { + return getClient(Utils.getDefaultTransport(), + Utils.getDefaultJsonFactory()); + } + + + /** + * Returns the fully qualified topic name for Pub/Sub. + */ + public static String getFullyQualifiedTopicName( + final String project, final String topic) { + return String.format("projects/%s/topics/%s", project, topic); + } + + /** + * Create a topic if it doesn't exist. + */ + public static void createTopic(Pubsub client, String fullTopicName) + throws IOException { + try { + client.projects().topics().get(fullTopicName).execute(); + } catch (GoogleJsonResponseException e) { + if (e.getStatusCode() == HttpStatusCodes.STATUS_CODE_NOT_FOUND) { + Topic topic = client.projects().topics() + .create(fullTopicName, new Topic()) + .execute(); + System.out.printf("Topic %s was created.\n", topic.getName()); + } + } + } +} diff --git a/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/RetryHttpInitializerWrapper.java b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/RetryHttpInitializerWrapper.java new file mode 100644 index 0000000000000..eeeabcef8bebe --- /dev/null +++ b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/RetryHttpInitializerWrapper.java @@ -0,0 +1,127 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples.complete.game.injector; + +import com.google.api.client.auth.oauth2.Credential; +import com.google.api.client.http.HttpBackOffIOExceptionHandler; +import com.google.api.client.http.HttpBackOffUnsuccessfulResponseHandler; +import com.google.api.client.http.HttpRequest; +import com.google.api.client.http.HttpRequestInitializer; +import com.google.api.client.http.HttpResponse; +import com.google.api.client.http.HttpUnsuccessfulResponseHandler; +import com.google.api.client.util.ExponentialBackOff; +import com.google.api.client.util.Sleeper; +import com.google.common.base.Preconditions; + +import java.io.IOException; +import java.util.logging.Logger; + +/** + * RetryHttpInitializerWrapper will automatically retry upon RPC + * failures, preserving the auto-refresh behavior of the Google + * Credentials. + */ +public class RetryHttpInitializerWrapper implements HttpRequestInitializer { + + /** + * A private logger. + */ + private static final Logger LOG = + Logger.getLogger(RetryHttpInitializerWrapper.class.getName()); + + /** + * One minutes in miliseconds. + */ + private static final int ONEMINITUES = 60000; + + /** + * Intercepts the request for filling in the "Authorization" + * header field, as well as recovering from certain unsuccessful + * error codes wherein the Credential must refresh its token for a + * retry. + */ + private final Credential wrappedCredential; + + /** + * A sleeper; you can replace it with a mock in your test. + */ + private final Sleeper sleeper; + + /** + * A constructor. + * + * @param wrappedCredential Credential which will be wrapped and + * used for providing auth header. + */ + public RetryHttpInitializerWrapper(final Credential wrappedCredential) { + this(wrappedCredential, Sleeper.DEFAULT); + } + + /** + * A protected constructor only for testing. + * + * @param wrappedCredential Credential which will be wrapped and + * used for providing auth header. + * @param sleeper Sleeper for easy testing. + */ + RetryHttpInitializerWrapper( + final Credential wrappedCredential, final Sleeper sleeper) { + this.wrappedCredential = Preconditions.checkNotNull(wrappedCredential); + this.sleeper = sleeper; + } + + /** + * Initializes the given request. + */ + @Override + public final void initialize(final HttpRequest request) { + request.setReadTimeout(2 * ONEMINITUES); // 2 minutes read timeout + final HttpUnsuccessfulResponseHandler backoffHandler = + new HttpBackOffUnsuccessfulResponseHandler( + new ExponentialBackOff()) + .setSleeper(sleeper); + request.setInterceptor(wrappedCredential); + request.setUnsuccessfulResponseHandler( + new HttpUnsuccessfulResponseHandler() { + @Override + public boolean handleResponse( + final HttpRequest request, + final HttpResponse response, + final boolean supportsRetry) throws IOException { + if (wrappedCredential.handleResponse( + request, response, supportsRetry)) { + // If credential decides it can handle it, + // the return code or message indicated + // something specific to authentication, + // and no backoff is desired. + return true; + } else if (backoffHandler.handleResponse( + request, response, supportsRetry)) { + // Otherwise, we defer to the judgement of + // our internal backoff handler. + LOG.info("Retrying " + + request.getUrl().toString()); + return true; + } else { + return false; + } + } + }); + request.setIOExceptionHandler( + new HttpBackOffIOExceptionHandler(new ExponentialBackOff()) + .setSleeper(sleeper)); + } +} + diff --git a/examples/src/test/java8/com/google/cloud/dataflow/examples/complete/game/GameStatsTest.java b/examples/src/test/java8/com/google/cloud/dataflow/examples/complete/game/GameStatsTest.java new file mode 100644 index 0000000000000..4795de2fc32d9 --- /dev/null +++ b/examples/src/test/java8/com/google/cloud/dataflow/examples/complete/game/GameStatsTest.java @@ -0,0 +1,99 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples.complete.game; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.dataflow.examples.complete.game.GameStats.CalculateSpammyUsers; +import com.google.cloud.dataflow.examples.complete.game.UserScore.GameActionInfo; +import com.google.cloud.dataflow.examples.complete.game.UserScore.ParseEventFn; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.RunnableOnService; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFnTester; +import com.google.cloud.dataflow.sdk.transforms.Filter; +import com.google.cloud.dataflow.sdk.transforms.MapElements; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.transforms.WithTimestamps; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; + +import org.hamcrest.CoreMatchers; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; + +/** + * Tests of GameStats. + * Because the pipeline was designed for easy readability and explanations, it lacks good + * modularity for testing. See our testing documentation for better ideas: + * https://cloud.google.com/dataflow/pipelines/testing-your-pipeline. + */ +@RunWith(JUnit4.class) +public class GameStatsTest implements Serializable { + + // User scores + static final KV[] USER_SCORES_ARRAY = new KV[] { + KV.of("Robot-2", 66), KV.of("Robot-1", 116), KV.of("user7_AndroidGreenKookaburra", 23), + KV.of("user7_AndroidGreenKookaburra", 1), + KV.of("user19_BisqueBilby", 14), KV.of("user13_ApricotQuokka", 15), + KV.of("user18_BananaEmu", 25), KV.of("user6_AmberEchidna", 8), + KV.of("user2_AmberQuokka", 6), KV.of("user0_MagentaKangaroo", 4), + KV.of("user0_MagentaKangaroo", 3), KV.of("user2_AmberCockatoo", 13), + KV.of("user7_AlmondWallaby", 15), KV.of("user6_AmberNumbat", 11), + KV.of("user6_AmberQuokka", 4) + }; + + static final List> USER_SCORES = Arrays.asList(USER_SCORES_ARRAY); + + // The expected list of 'spammers'. + static final KV[] SPAMMERS = new KV[] { + KV.of("Robot-2", 66), KV.of("Robot-1", 116) + }; + + + /** Test the calculation of 'spammy users'. */ + @Test + @Category(RunnableOnService.class) + public void testCalculateSpammyUsers() throws Exception { + Pipeline p = TestPipeline.create(); + + PCollection> input = p.apply(Create.of(USER_SCORES)); + PCollection> output = input.apply(new CalculateSpammyUsers()); + + // Check the set of spammers. + DataflowAssert.that(output).containsInAnyOrder(SPAMMERS); + + p.run(); + } + +} diff --git a/examples/src/test/java8/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScoreTest.java b/examples/src/test/java8/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScoreTest.java new file mode 100644 index 0000000000000..fe163037f63c5 --- /dev/null +++ b/examples/src/test/java8/com/google/cloud/dataflow/examples/complete/game/HourlyTeamScoreTest.java @@ -0,0 +1,121 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples.complete.game; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.dataflow.examples.complete.game.UserScore.ExtractAndSumScore; +import com.google.cloud.dataflow.examples.complete.game.UserScore.GameActionInfo; +import com.google.cloud.dataflow.examples.complete.game.UserScore.ParseEventFn; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.RunnableOnService; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFnTester; +import com.google.cloud.dataflow.sdk.transforms.Filter; +import com.google.cloud.dataflow.sdk.transforms.MapElements; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.transforms.WithTimestamps; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; + +import org.hamcrest.CoreMatchers; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; + +/** + * Tests of HourlyTeamScore. + * Because the pipeline was designed for easy readability and explanations, it lacks good + * modularity for testing. See our testing documentation for better ideas: + * https://cloud.google.com/dataflow/pipelines/testing-your-pipeline. + */ +@RunWith(JUnit4.class) +public class HourlyTeamScoreTest implements Serializable { + + static final String[] GAME_EVENTS_ARRAY = new String[] { + "user0_MagentaKangaroo,MagentaKangaroo,3,1447955630000,2015-11-19 09:53:53.444", + "user13_ApricotQuokka,ApricotQuokka,15,1447955630000,2015-11-19 09:53:53.444", + "user6_AmberNumbat,AmberNumbat,11,1447955630000,2015-11-19 09:53:53.444", + "user7_AlmondWallaby,AlmondWallaby,15,1447955630000,2015-11-19 09:53:53.444", + "user7_AndroidGreenKookaburra,AndroidGreenKookaburra,12,1447955630000,2015-11-19 09:53:53.444", + "user7_AndroidGreenKookaburra,AndroidGreenKookaburra,11,1447955630000,2015-11-19 09:53:53.444", + "user19_BisqueBilby,BisqueBilby,6,1447955630000,2015-11-19 09:53:53.444", + "user19_BisqueBilby,BisqueBilby,8,1447955630000,2015-11-19 09:53:53.444", + // time gap... + "user0_AndroidGreenEchidna,AndroidGreenEchidna,0,1447965690000,2015-11-19 12:41:31.053", + "user0_MagentaKangaroo,MagentaKangaroo,4,1447965690000,2015-11-19 12:41:31.053", + "user2_AmberCockatoo,AmberCockatoo,13,1447965690000,2015-11-19 12:41:31.053", + "user18_BananaEmu,BananaEmu,7,1447965690000,2015-11-19 12:41:31.053", + "user3_BananaEmu,BananaEmu,17,1447965690000,2015-11-19 12:41:31.053", + "user18_BananaEmu,BananaEmu,1,1447965690000,2015-11-19 12:41:31.053", + "user18_ApricotCaneToad,ApricotCaneToad,14,1447965690000,2015-11-19 12:41:31.053" + }; + + + static final List GAME_EVENTS = Arrays.asList(GAME_EVENTS_ARRAY); + + + // Used to check the filtering. + static final KV[] FILTERED_EVENTS = new KV[] { + KV.of("user0_AndroidGreenEchidna", 0), KV.of("user0_MagentaKangaroo", 4), + KV.of("user2_AmberCockatoo", 13), + KV.of("user18_BananaEmu", 7), KV.of("user3_BananaEmu", 17), + KV.of("user18_BananaEmu", 1), KV.of("user18_ApricotCaneToad", 14) + }; + + + /** Test the filtering. */ + @Test + @Category(RunnableOnService.class) + public void testUserScoresFilter() throws Exception { + Pipeline p = TestPipeline.create(); + + final Instant startMinTimestamp = new Instant(1447965680000L); + + PCollection input = p.apply(Create.of(GAME_EVENTS).withCoder(StringUtf8Coder.of())); + + PCollection> output = input + .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn())) + + .apply("FilterStartTime", Filter.byPredicate( + (GameActionInfo gInfo) + -> gInfo.getTimestamp() > startMinTimestamp.getMillis())) + // run a map to access the fields in the result. + .apply(MapElements + .via((GameActionInfo gInfo) -> KV.of(gInfo.getUser(), gInfo.getScore())) + .withOutputType(new TypeDescriptor>() {})); + + DataflowAssert.that(output).containsInAnyOrder(FILTERED_EVENTS); + + p.run(); + } + +} diff --git a/examples/src/test/java8/com/google/cloud/dataflow/examples/complete/game/UserScoreTest.java b/examples/src/test/java8/com/google/cloud/dataflow/examples/complete/game/UserScoreTest.java new file mode 100644 index 0000000000000..69601be1bb4d0 --- /dev/null +++ b/examples/src/test/java8/com/google/cloud/dataflow/examples/complete/game/UserScoreTest.java @@ -0,0 +1,156 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.examples.complete.game; + +import com.google.cloud.dataflow.examples.complete.game.UserScore.ExtractAndSumScore; +import com.google.cloud.dataflow.examples.complete.game.UserScore.GameActionInfo; +import com.google.cloud.dataflow.examples.complete.game.UserScore.ParseEventFn; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.RunnableOnService; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFnTester; +import com.google.cloud.dataflow.sdk.transforms.MapElements; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; + +/** + * Tests of UserScore. + */ +@RunWith(JUnit4.class) +public class UserScoreTest implements Serializable { + + static final String[] GAME_EVENTS_ARRAY = new String[] { + "user0_MagentaKangaroo,MagentaKangaroo,3,1447955630000,2015-11-19 09:53:53.444", + "user13_ApricotQuokka,ApricotQuokka,15,1447955630000,2015-11-19 09:53:53.444", + "user6_AmberNumbat,AmberNumbat,11,1447955630000,2015-11-19 09:53:53.444", + "user7_AlmondWallaby,AlmondWallaby,15,1447955630000,2015-11-19 09:53:53.444", + "user7_AndroidGreenKookaburra,AndroidGreenKookaburra,12,1447955630000,2015-11-19 09:53:53.444", + "user6_AliceBlueDingo,AliceBlueDingo,4,xxxxxxx,2015-11-19 09:53:53.444", + "user7_AndroidGreenKookaburra,AndroidGreenKookaburra,11,1447955630000,2015-11-19 09:53:53.444", + "THIS IS A PARSE ERROR,2015-11-19 09:53:53.444", + "user19_BisqueBilby,BisqueBilby,6,1447955630000,2015-11-19 09:53:53.444", + "user19_BisqueBilby,BisqueBilby,8,1447955630000,2015-11-19 09:53:53.444" + }; + + static final String[] GAME_EVENTS_ARRAY2 = new String[] { + "user6_AliceBlueDingo,AliceBlueDingo,4,xxxxxxx,2015-11-19 09:53:53.444", + "THIS IS A PARSE ERROR,2015-11-19 09:53:53.444", + "user13_BisqueBilby,BisqueBilby,xxx,1447955630000,2015-11-19 09:53:53.444" + }; + + static final List GAME_EVENTS = Arrays.asList(GAME_EVENTS_ARRAY); + static final List GAME_EVENTS2 = Arrays.asList(GAME_EVENTS_ARRAY2); + + static final KV[] USER_SUMS = new KV[] { + KV.of("user0_MagentaKangaroo", 3), KV.of("user13_ApricotQuokka", 15), + KV.of("user6_AmberNumbat", 11), KV.of("user7_AlmondWallaby", 15), + KV.of("user7_AndroidGreenKookaburra", 23), + KV.of("user19_BisqueBilby", 14) }; + + static final KV[] TEAM_SUMS = new KV[] { + KV.of("MagentaKangaroo", 3), KV.of("ApricotQuokka", 15), + KV.of("AmberNumbat", 11), KV.of("AlmondWallaby", 15), + KV.of("AndroidGreenKookaburra", 23), + KV.of("BisqueBilby", 14) }; + + /** Test the ParseEventFn DoFn. */ + @Test + public void testParseEventFn() { + DoFnTester parseEventFn = + DoFnTester.of(new ParseEventFn()); + + List results = parseEventFn.processBatch(GAME_EVENTS_ARRAY); + Assert.assertEquals(results.size(), 8); + Assert.assertEquals(results.get(0).getUser(), "user0_MagentaKangaroo"); + Assert.assertEquals(results.get(0).getTeam(), "MagentaKangaroo"); + Assert.assertEquals(results.get(0).getScore(), new Integer(3)); + } + + /** Tests ExtractAndSumScore("user"). */ + @Test + @Category(RunnableOnService.class) + public void testUserScoreSums() throws Exception { + Pipeline p = TestPipeline.create(); + + PCollection input = p.apply(Create.of(GAME_EVENTS).withCoder(StringUtf8Coder.of())); + + PCollection> output = input + .apply(ParDo.of(new ParseEventFn())) + // Extract and sum username/score pairs from the event data. + .apply("ExtractUserScore", new ExtractAndSumScore("user")); + + // Check the user score sums. + DataflowAssert.that(output).containsInAnyOrder(USER_SUMS); + + p.run(); + } + + /** Tests ExtractAndSumScore("team"). */ + @Test + @Category(RunnableOnService.class) + public void testTeamScoreSums() throws Exception { + Pipeline p = TestPipeline.create(); + + PCollection input = p.apply(Create.of(GAME_EVENTS).withCoder(StringUtf8Coder.of())); + + PCollection> output = input + .apply(ParDo.of(new ParseEventFn())) + // Extract and sum teamname/score pairs from the event data. + .apply("ExtractTeamScore", new ExtractAndSumScore("team")); + + // Check the team score sums. + DataflowAssert.that(output).containsInAnyOrder(TEAM_SUMS); + + p.run(); + } + + /** Test that bad input data is dropped appropriately. */ + @Test + @Category(RunnableOnService.class) + public void testUserScoresBadInput() throws Exception { + Pipeline p = TestPipeline.create(); + + PCollection input = p.apply(Create.of(GAME_EVENTS2).withCoder(StringUtf8Coder.of())); + + PCollection> extract = input + .apply(ParDo.of(new ParseEventFn())) + .apply( + MapElements.via((GameActionInfo gInfo) -> KV.of(gInfo.getUser(), gInfo.getScore())) + .withOutputType(new TypeDescriptor>() {})); + + DataflowAssert.that(extract).empty(); + + p.run(); + } +} From 3c90d44dbaea86f93a51b010b08b249195b277f4 Mon Sep 17 00:00:00 2001 From: kirpichov Date: Sun, 13 Dec 2015 23:33:48 -0800 Subject: [PATCH 1227/1541] Uses the new progress/split request classes ApproximateProgress has been split into: * ApproximateReportedProgress for progress reporting; * ApproximateSplitRequest for dynamic split requests. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110127317 --- .../sdk/runners/dataflow/CustomSources.java | 15 ++--- .../sdk/runners/worker/AvroReader.java | 13 ++-- .../sdk/runners/worker/ConcatReader.java | 9 +-- .../worker/DataflowWorkProgressUpdater.java | 4 +- .../sdk/runners/worker/DataflowWorker.java | 2 +- .../runners/worker/GroupingShuffleReader.java | 9 +-- .../sdk/runners/worker/InMemoryReader.java | 8 +-- .../worker/SourceTranslationUtils.java | 28 ++++----- .../sdk/runners/worker/TextReader.java | 21 ++++--- .../runners/dataflow/CustomSourcesTest.java | 8 ++- .../sdk/runners/worker/AvroReaderTest.java | 6 +- .../sdk/runners/worker/ConcatReaderTest.java | 5 +- .../DataflowWorkProgressUpdaterTest.java | 28 +++++---- .../worker/GroupingShuffleReaderTest.java | 12 ++-- .../runners/worker/InMemoryReaderTest.java | 5 +- .../sdk/runners/worker/ReaderTestUtils.java | 61 ++++++++++++++----- .../sdk/runners/worker/TextReaderTest.java | 12 ++-- .../common/worker/MapTaskExecutorTest.java | 10 +-- .../util/common/worker/ReadOperationTest.java | 25 +++++--- 19 files changed, 163 insertions(+), 118 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSources.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSources.java index 3e14ca615544a..668c33dcd5498 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSources.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSources.java @@ -28,7 +28,8 @@ import com.google.api.client.util.BackOff; import com.google.api.client.util.Base64; -import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.ApproximateReportedProgress; +import com.google.api.services.dataflow.model.ApproximateSplitRequest; import com.google.api.services.dataflow.model.DerivedSource; import com.google.api.services.dataflow.model.DynamicSourceSplit; import com.google.api.services.dataflow.model.SourceMetadata; @@ -531,10 +532,10 @@ public void close() throws IOException { @Override public Reader.Progress getProgress() { if (reader instanceof BoundedSource.BoundedReader) { - ApproximateProgress progress = new ApproximateProgress(); + ApproximateReportedProgress progress = new ApproximateReportedProgress(); Double fractionConsumed = reader.getFractionConsumed(); if (fractionConsumed != null) { - progress.setPercentComplete(fractionConsumed.floatValue()); + progress.setFractionConsumed(fractionConsumed); } return SourceTranslationUtils.cloudProgressToReaderProgress(progress); } else { @@ -545,15 +546,15 @@ public Reader.Progress getProgress() { @Override public Reader.DynamicSplitResult requestDynamicSplit(Reader.DynamicSplitRequest request) { - ApproximateProgress stopPosition = - SourceTranslationUtils.splitRequestToApproximateProgress(request); - Float fractionConsumed = stopPosition.getPercentComplete(); + ApproximateSplitRequest stopPosition = + SourceTranslationUtils.splitRequestToApproximateSplitRequest(request); + Double fractionConsumed = stopPosition.getFractionConsumed(); if (fractionConsumed == null) { // Only truncating at a fraction is currently supported. return null; } BoundedSource original = reader.getCurrentSource(); - BoundedSource residual = reader.splitAtFraction(fractionConsumed.doubleValue()); + BoundedSource residual = reader.splitAtFraction(fractionConsumed); if (residual == null) { return null; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java index ebad4ab948301..f957b16f6043e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReader.java @@ -18,7 +18,8 @@ import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.ApproximateReportedProgress; +import com.google.api.services.dataflow.model.ApproximateSplitRequest; import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.io.AvroSource; import com.google.cloud.dataflow.sdk.io.BoundedSource; @@ -135,8 +136,8 @@ public Progress getProgress() { if (readerProgress == null) { return null; } - ApproximateProgress progress = new ApproximateProgress(); - progress.setPercentComplete(readerProgress.floatValue()); + ApproximateReportedProgress progress = new ApproximateReportedProgress(); + progress.setFractionConsumed(readerProgress); return cloudProgressToReaderProgress(progress); } @@ -147,9 +148,9 @@ public void close() throws IOException { @Override public DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest) { - ApproximateProgress splitProgress = - SourceTranslationUtils.splitRequestToApproximateProgress(splitRequest); - double splitAtFraction = splitProgress.getPercentComplete(); + ApproximateSplitRequest splitProgress = + SourceTranslationUtils.splitRequestToApproximateSplitRequest(splitRequest); + double splitAtFraction = splitProgress.getFractionConsumed(); LOG.info("Received request for dynamic split at {}", splitAtFraction); OffsetBasedSource residual = reader.splitAtFraction(splitAtFraction); if (residual == null) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ConcatReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ConcatReader.java index a1dd49f078643..2e3bf6cf9a300 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ConcatReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ConcatReader.java @@ -18,10 +18,11 @@ import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateSplitRequest; import static com.google.common.base.Preconditions.checkNotNull; -import com.google.api.services.dataflow.model.ApproximateProgress; +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.cloud.dataflow.sdk.io.range.OffsetRangeTracker; @@ -200,7 +201,7 @@ public Progress getProgress() { concatPosition.setPosition(positionOfCurrentIterator); } - ApproximateProgress progress = new ApproximateProgress(); + ApproximateReportedProgress progress = new ApproximateReportedProgress(); com.google.api.services.dataflow.model.Position currentPosition = new com.google.api.services.dataflow.model.Position(); currentPosition.setConcatPosition(concatPosition); @@ -213,7 +214,7 @@ public Progress getProgress() { public DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest) { checkNotNull(splitRequest); - ApproximateProgress splitProgress = splitRequestToApproximateProgress(splitRequest); + ApproximateSplitRequest splitProgress = 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); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java index f9b62c3e4a282..c1e99bd671b46 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdater.java @@ -23,7 +23,7 @@ import static com.google.cloud.dataflow.sdk.util.TimeUtil.fromCloudTime; import static com.google.cloud.dataflow.sdk.util.TimeUtil.toCloudDuration; -import com.google.api.services.dataflow.model.ApproximateProgress; +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.api.services.dataflow.model.WorkItemStatus; @@ -99,7 +99,7 @@ protected void reportProgressHelper() throws Exception { fromCloudDuration(result.getReportStatusInterval()).getMillis(), leaseRemainingTime(getLeaseExpirationTimestamp(result))); - ApproximateProgress suggestedStopPoint = result.getSuggestedStopPoint(); + ApproximateSplitRequest suggestedStopPoint = result.getSplitRequest(); if (suggestedStopPoint != null) { LOG.info("Proposing dynamic split of work unit {} at {}", workString(), suggestedStopPoint); dynamicSplitResultToReport = worker.requestDynamicSplit( diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java index 6c24cdb394bf7..e3536a92a5273 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java @@ -369,7 +369,7 @@ static WorkItemStatus buildStatus(WorkItem workItem, boolean completed, } if (progress != null) { - status.setProgress(readerProgressToCloudProgress(progress)); + status.setReportedProgress(readerProgressToCloudProgress(progress)); } if (dynamicSplitResult instanceof Reader.DynamicSplitResultWithPosition) { Reader.DynamicSplitResultWithPosition asPosition = diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java index f20b43ca9824c..ff856fe9324a9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReader.java @@ -19,10 +19,11 @@ import static com.google.api.client.util.Preconditions.checkNotNull; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateSplitRequest; import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; -import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.ApproximateReportedProgress; +import com.google.api.services.dataflow.model.ApproximateSplitRequest; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.IterableCoder; import com.google.cloud.dataflow.sdk.coders.KvCoder; @@ -274,7 +275,7 @@ public double getRemainingParallelism() { public Progress getProgress() { com.google.api.services.dataflow.model.Position position = new com.google.api.services.dataflow.model.Position(); - ApproximateProgress progress = new ApproximateProgress(); + ApproximateReportedProgress progress = new ApproximateReportedProgress(); ByteArrayShufflePosition groupStart = rangeTracker.getLastGroupStart(); if (groupStart != null) { position.setShufflePosition(groupStart.encodeBase64()); @@ -291,7 +292,7 @@ public Progress getProgress() { @Override public DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest) { checkNotNull(splitRequest); - ApproximateProgress splitProgress = splitRequestToApproximateProgress( + ApproximateSplitRequest splitProgress = splitRequestToApproximateSplitRequest( splitRequest); com.google.api.services.dataflow.model.Position splitPosition = splitProgress.getPosition(); if (splitPosition == null) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java index d6ca259e6279f..b9a60c16ee498 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java @@ -19,10 +19,10 @@ import static com.google.api.client.util.Preconditions.checkNotNull; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateSplitRequest; import static java.lang.Math.min; -import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.ApproximateReportedProgress; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.range.OffsetRangeTracker; import com.google.cloud.dataflow.sdk.util.CoderUtils; @@ -122,7 +122,7 @@ public Progress getProgress() { new com.google.api.services.dataflow.model.Position(); currentPosition.setRecordIndex((long) nextIndex); - ApproximateProgress progress = new ApproximateProgress(); + ApproximateReportedProgress progress = new ApproximateReportedProgress(); progress.setPosition(currentPosition); return cloudProgressToReaderProgress(progress); @@ -138,7 +138,7 @@ public DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest) checkNotNull(splitRequest); com.google.api.services.dataflow.model.Position splitPosition = - splitRequestToApproximateProgress(splitRequest).getPosition(); + splitRequestToApproximateSplitRequest(splitRequest).getPosition(); if (splitPosition == null) { LOG.warn("InMemoryReader only supports split at a Position. Requested: {}", splitRequest); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java index 8506dad152ad7..7b9fff9d2f24f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SourceTranslationUtils.java @@ -21,7 +21,8 @@ import static com.google.cloud.dataflow.sdk.util.Structs.addLong; import static com.google.cloud.dataflow.sdk.util.Structs.getDictionary; -import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.ApproximateReportedProgress; +import com.google.api.services.dataflow.model.ApproximateSplitRequest; import com.google.api.services.dataflow.model.Position; import com.google.api.services.dataflow.model.Source; import com.google.api.services.dataflow.model.SourceMetadata; @@ -39,7 +40,7 @@ */ public class SourceTranslationUtils { public static Reader.Progress cloudProgressToReaderProgress( - @Nullable ApproximateProgress cloudProgress) { + @Nullable ApproximateReportedProgress cloudProgress) { return cloudProgress == null ? null : new DataflowReaderProgress(cloudProgress); } @@ -47,7 +48,7 @@ public static Reader.Position cloudPositionToReaderPosition(@Nullable Position c return cloudPosition == null ? null : new DataflowReaderPosition(cloudPosition); } - public static ApproximateProgress readerProgressToCloudProgress( + public static ApproximateReportedProgress readerProgressToCloudProgress( @Nullable Reader.Progress readerProgress) { return readerProgress == null ? null : ((DataflowReaderProgress) readerProgress).cloudProgress; } @@ -56,22 +57,21 @@ public static Position toCloudPosition(@Nullable Reader.Position readerPosition) return readerPosition == null ? null : ((DataflowReaderPosition) readerPosition).cloudPosition; } - public static ApproximateProgress splitRequestToApproximateProgress( + public static ApproximateSplitRequest splitRequestToApproximateSplitRequest( @Nullable Reader.DynamicSplitRequest splitRequest) { return (splitRequest == null) - ? null : ((DataflowDynamicSplitRequest) splitRequest).approximateProgress; + ? null : ((DataflowDynamicSplitRequest) splitRequest).splitRequest; } public static Reader.DynamicSplitRequest toDynamicSplitRequest( - @Nullable ApproximateProgress approximateProgress) { - return (approximateProgress == null) - ? null : new DataflowDynamicSplitRequest(approximateProgress); + @Nullable ApproximateSplitRequest splitRequest) { + return (splitRequest == null) ? null : new DataflowDynamicSplitRequest(splitRequest); } static class DataflowReaderProgress implements Reader.Progress { - public final ApproximateProgress cloudProgress; + public final ApproximateReportedProgress cloudProgress; - public DataflowReaderProgress(ApproximateProgress cloudProgress) { + public DataflowReaderProgress(ApproximateReportedProgress cloudProgress) { this.cloudProgress = cloudProgress; } @@ -135,15 +135,15 @@ public static Source dictionaryToCloudSource(Map params) throws } private static class DataflowDynamicSplitRequest implements Reader.DynamicSplitRequest { - public final ApproximateProgress approximateProgress; + public final ApproximateSplitRequest splitRequest; - private DataflowDynamicSplitRequest(ApproximateProgress approximateProgress) { - this.approximateProgress = approximateProgress; + private DataflowDynamicSplitRequest(ApproximateSplitRequest splitRequest) { + this.splitRequest = splitRequest; } @Override public String toString() { - return String.valueOf(approximateProgress); + return String.valueOf(splitRequest); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReader.java index 04bde2c353652..d39faad28cd53 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/TextReader.java @@ -19,9 +19,10 @@ import static com.google.api.client.util.Preconditions.checkNotNull; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateSplitRequest; -import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.ApproximateReportedProgress; +import com.google.api.services.dataflow.model.ApproximateSplitRequest; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.io.range.OffsetRangeTracker; @@ -369,12 +370,12 @@ public Progress getProgress() { new com.google.api.services.dataflow.model.Position(); currentPosition.setByteOffset(offset); - ApproximateProgress progress = new ApproximateProgress(); + ApproximateReportedProgress progress = new ApproximateReportedProgress(); progress.setPosition(currentPosition); // If endOffset is unspecified, we don't know the fraction consumed. if (rangeTracker.getStopPosition() != Long.MAX_VALUE) { - progress.setPercentComplete((float) rangeTracker.getFractionConsumed()); + progress.setFractionConsumed(rangeTracker.getFractionConsumed()); } return cloudProgressToReaderProgress(progress); @@ -385,14 +386,14 @@ public DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest) checkNotNull(splitRequest); // Currently, file-based Reader only supports split at a byte offset. - ApproximateProgress splitProgress = splitRequestToApproximateProgress(splitRequest); + ApproximateSplitRequest splitProgress = splitRequestToApproximateSplitRequest(splitRequest); com.google.api.services.dataflow.model.Position splitPosition = splitProgress.getPosition(); if (splitPosition == null) { - if (splitProgress.getPercentComplete() != null) { - float percentageComplete = splitProgress.getPercentComplete().floatValue(); - if (percentageComplete <= 0 || percentageComplete >= 1) { + if (splitProgress.getFractionConsumed() != null) { + float fractionConsumed = splitProgress.getFractionConsumed().floatValue(); + if (fractionConsumed <= 0 || fractionConsumed >= 1) { LOG.warn( - "TextReader cannot be split since the provided percentage of " + "TextReader cannot be split since the provided fraction of " + "work to be completed is out of the valid range (0, 1). Requested: {}", splitRequest); } @@ -407,7 +408,7 @@ public DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest) } splitPosition.setByteOffset( - rangeTracker.getPositionForFractionConsumed(percentageComplete)); + rangeTracker.getPositionForFractionConsumed(fractionConsumed)); } else { LOG.warn( "TextReader requires either a position or percentage of work to be complete to" diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java index e7541f2397622..7b3ddde901333 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java @@ -361,16 +361,18 @@ public void testProgressAndSourceSplitTranslation() throws Exception { try (Reader.ReaderIterator> iterator = reader.iterator()) { assertTrue(iterator.hasNext()); assertEquals( - 0, readerProgressToCloudProgress(iterator.getProgress()).getPercentComplete().intValue()); + 0.1, + readerProgressToCloudProgress(iterator.getProgress()).getFractionConsumed().doubleValue(), + 1e-6); assertEquals(valueInGlobalWindow(10), iterator.next()); assertEquals( 0.1, - readerProgressToCloudProgress(iterator.getProgress()).getPercentComplete().doubleValue(), + readerProgressToCloudProgress(iterator.getProgress()).getFractionConsumed().doubleValue(), 1e-6); assertEquals(valueInGlobalWindow(11), iterator.next()); assertEquals( 0.2, - readerProgressToCloudProgress(iterator.getProgress()).getPercentComplete().doubleValue(), + readerProgressToCloudProgress(iterator.getProgress()).getFractionConsumed().doubleValue(), 1e-6); assertEquals(valueInGlobalWindow(12), iterator.next()); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderTest.java index 88b3e3314dd35..29f14e9cecdc8 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/AvroReaderTest.java @@ -163,15 +163,15 @@ private List readElems(String filename, @Nullable Long startOffset, AvroReader avroReader = new AvroReader<>(filename, startOffset, endOffset, coder, null); new ExecutorTestUtils.TestReaderObserver(avroReader, actualSizes); - float progressReported = 0; + double progressReported = 0; List actualElems = new ArrayList<>(); try (Reader.ReaderIterator> iterator = avroReader.iterator()) { while (iterator.hasNext()) { actualElems.add(iterator.next().getValue()); - float progress = 0.0f; + double progress = 0.0; Progress readerProgress = iterator.getProgress(); if (readerProgress != null) { - progress = readerProgressToCloudProgress(iterator.getProgress()).getPercentComplete(); + progress = readerProgressToCloudProgress(iterator.getProgress()).getFractionConsumed(); } // Make sure that the reported progress is monotonous. Assert.assertThat(progress, greaterThanOrEqualTo(progressReported)); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ConcatReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ConcatReaderTest.java index a8adae4148565..6ed712a0e1f24 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ConcatReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ConcatReaderTest.java @@ -28,7 +28,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; -import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.ApproximateReportedProgress; import com.google.api.services.dataflow.model.Source; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; @@ -371,7 +371,8 @@ private void runProgressTest(int... sizes) throws Exception { for (int readerIndex = 0; readerIndex < sizes.length; readerIndex++) { for (int recordIndex = 0; recordIndex < sizes[readerIndex]; recordIndex++) { iterator.next(); - ApproximateProgress progress = readerProgressToCloudProgress(iterator.getProgress()); + ApproximateReportedProgress progress = + readerProgressToCloudProgress(iterator.getProgress()); assertEquals( readerIndex, progress.getPosition().getConcatPosition().getIndex().intValue()); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java index b46d34eab3531..a8c0315cbd22f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java @@ -17,11 +17,11 @@ package com.google.cloud.dataflow.sdk.runners.worker; import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.approximateProgressAtIndex; -import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.approximateProgressAtPosition; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.approximateSplitRequestAtPosition; import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.positionAtIndex; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateSplitRequest; import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; import static com.google.cloud.dataflow.sdk.util.CloudCounterUtils.extractCounter; import static com.google.cloud.dataflow.sdk.util.CloudMetricUtils.extractCloudMetric; @@ -40,7 +40,8 @@ import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; -import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.ApproximateReportedProgress; +import com.google.api.services.dataflow.model.ApproximateSplitRequest; import com.google.api.services.dataflow.model.MetricUpdate; import com.google.api.services.dataflow.model.Position; import com.google.api.services.dataflow.model.WorkItem; @@ -83,7 +84,7 @@ @RunWith(JUnit4.class) public class DataflowWorkProgressUpdaterTest { static class TestMapTaskExecutor extends MapTaskExecutor { - ApproximateProgress progress = null; + ApproximateReportedProgress progress = null; public TestMapTaskExecutor(CounterSet counters) { super(new ArrayList(), counters, @@ -98,15 +99,15 @@ public Reader.Progress getWorkerProgress() { @Override public Reader.DynamicSplitResult requestDynamicSplit(Reader.DynamicSplitRequest splitRequest) { @Nullable - ApproximateProgress progress = splitRequestToApproximateProgress(splitRequest); - if (progress == null) { + ApproximateSplitRequest split = splitRequestToApproximateSplitRequest(splitRequest); + if (split == null) { return null; } return new Reader.DynamicSplitResultWithPosition( - cloudPositionToReaderPosition(progress.getPosition())); + cloudPositionToReaderPosition(split.getPosition())); } - public void setWorkerProgress(ApproximateProgress progress) { + public void setWorkerProgress(ApproximateReportedProgress progress) { this.progress = progress; } } @@ -331,7 +332,7 @@ private void setUpMetrics(int n) { } } - private void setUpProgress(ApproximateProgress progress) { + private void setUpProgress(ApproximateReportedProgress progress) { worker.setWorkerProgress(progress); } @@ -346,7 +347,7 @@ private WorkItemServiceState generateServiceState(long leaseExpirationTimestamp, responseState.setNextReportIndex(nextReportIndex); if (suggestedStopPosition != null) { - responseState.setSuggestedStopPoint(approximateProgressAtPosition(suggestedStopPosition)); + responseState.setSplitRequest(approximateSplitRequestAtPosition(suggestedStopPosition)); } return responseState; @@ -361,7 +362,7 @@ private static final class ExpectedDataflowWorkItemStatus Integer metricCount; @Nullable - ApproximateProgress expectedProgress; + ApproximateReportedProgress expectedProgress; @Nullable Position expectedSplitPosition; @@ -379,7 +380,8 @@ public ExpectedDataflowWorkItemStatus withMetrics(Integer metricCount) { return this; } - public ExpectedDataflowWorkItemStatus withProgress(ApproximateProgress expectedProgress) { + public ExpectedDataflowWorkItemStatus withProgress( + ApproximateReportedProgress expectedProgress) { this.expectedProgress = expectedProgress; return this; } @@ -459,7 +461,7 @@ private boolean matchProgress(WorkItemStatus status) { if (expectedProgress == null) { return true; } - ApproximateProgress progress = status.getProgress(); + ApproximateReportedProgress progress = status.getReportedProgress(); return expectedProgress.equals(progress); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java index ce6281fb308a5..4d5ca3f97aa63 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/GroupingShuffleReaderTest.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.runners.worker; import static com.google.api.client.util.Base64.encodeBase64URLSafeString; +import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.approximateSplitRequestAtPosition; import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.positionFromSplitResult; import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.splitRequestAtPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.readerProgressToCloudProgress; @@ -29,7 +30,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.ApproximateReportedProgress; import com.google.api.services.dataflow.model.Position; import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; import com.google.cloud.dataflow.sdk.coders.Coder; @@ -642,7 +643,8 @@ public void testGetApproximateProgress() throws Exception { Integer i = 0; while (readerIterator.hasNext()) { assertTrue(readerIterator.hasNext()); - ApproximateProgress progress = readerProgressToCloudProgress(readerIterator.getProgress()); + ApproximateReportedProgress progress = readerProgressToCloudProgress( + readerIterator.getProgress()); assertNotNull(progress.getPosition().getShufflePosition()); // Compare returned position with the expected position. @@ -662,11 +664,7 @@ public void testGetApproximateProgress() throws Exception { proposedSplitPosition.setShufflePosition(stop); assertNull( readerIterator.requestDynamicSplit( - toDynamicSplitRequest(createApproximateProgress(proposedSplitPosition)))); + toDynamicSplitRequest(approximateSplitRequestAtPosition(proposedSplitPosition)))); } } - - private ApproximateProgress createApproximateProgress(Position position) { - return new ApproximateProgress().setPosition(position); - } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java index 6b561d0e2b9da..f69d95a5d1cf6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java @@ -29,7 +29,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.ApproximateSplitRequest; import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.util.common.worker.ExecutorTestUtils; @@ -145,7 +145,8 @@ public void testDynamicSplit() throws Exception { try (Reader.ReaderIterator iterator = inMemoryReader.iterator()) { // Poke the iterator so that we can test dynamic splitting. assertTrue(iterator.hasNext()); - assertNull(iterator.requestDynamicSplit(toDynamicSplitRequest(new ApproximateProgress()))); + assertNull(iterator.requestDynamicSplit(toDynamicSplitRequest( + new ApproximateSplitRequest()))); assertNull(iterator.requestDynamicSplit(splitRequestAtIndex(null))); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderTestUtils.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderTestUtils.java index fb3e0a2f9289c..0c9e4e53a9df8 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderTestUtils.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/ReaderTestUtils.java @@ -19,7 +19,8 @@ import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.toCloudPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.toDynamicSplitRequest; -import com.google.api.services.dataflow.model.ApproximateProgress; +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.Position; import com.google.cloud.dataflow.sdk.util.WindowedValue; @@ -51,42 +52,72 @@ public static Position positionAtConcatPosition( new ConcatPosition().setIndex(index).setPosition(innerPosition)); } - public static ApproximateProgress approximateProgressAtPosition(@Nullable Position position) { - return new ApproximateProgress().setPosition(position); + public static ApproximateReportedProgress approximateProgressAtPosition( + @Nullable Position position) { + return new ApproximateReportedProgress().setPosition(position); } - public static ApproximateProgress approximateProgressAtIndex(@Nullable Long index) { + public static ApproximateSplitRequest approximateSplitRequestAtPosition( + @Nullable Position position) { + return new ApproximateSplitRequest().setPosition(position); + } + + public static ApproximateReportedProgress approximateProgressAtIndex( + @Nullable Long index) { return approximateProgressAtPosition(positionAtIndex(index)); } - public static ApproximateProgress approximateProgressAtByteOffset(@Nullable Long byteOffset) { + public static ApproximateSplitRequest approximateSplitRequestAtIndex( + @Nullable Long index) { + return approximateSplitRequestAtPosition(positionAtIndex(index)); + } + + public static ApproximateReportedProgress approximateProgressAtByteOffset( + @Nullable Long byteOffset) { return approximateProgressAtPosition(positionAtByteOffset(byteOffset)); } - public static ApproximateProgress approximateProgressAtConcatPosition( + public static ApproximateSplitRequest approximateSplitRequestAtByteOffset( + @Nullable Long byteOffset) { + return approximateSplitRequestAtPosition(positionAtByteOffset(byteOffset)); + } + + public static ApproximateReportedProgress approximateProgressAtConcatPosition( @Nullable Integer index, @Nullable Position innerPosition) { return approximateProgressAtPosition(positionAtConcatPosition(index, innerPosition)); } - public static ApproximateProgress approximateProgressAtFraction(@Nullable Float fraction) { - return new ApproximateProgress().setPercentComplete(fraction); + public static ApproximateSplitRequest approximateSplitRequestAtConcatPosition( + @Nullable Integer index, @Nullable Position innerPosition) { + return approximateSplitRequestAtPosition(positionAtConcatPosition(index, innerPosition)); + } + + public static ApproximateReportedProgress approximateProgressAtFraction( + @Nullable Double fraction) { + return new ApproximateReportedProgress().setFractionConsumed(fraction); + } + + public static ApproximateSplitRequest approximateSplitRequestAtFraction( + @Nullable Double fraction) { + return new ApproximateSplitRequest().setFractionConsumed(fraction); } - public static Reader.DynamicSplitRequest splitRequestAtPosition(@Nullable Position position) { - return toDynamicSplitRequest(approximateProgressAtPosition(position)); + public static Reader.DynamicSplitRequest splitRequestAtPosition( + @Nullable Position position) { + return toDynamicSplitRequest(approximateSplitRequestAtPosition(position)); } public static Reader.DynamicSplitRequest splitRequestAtIndex(@Nullable Long index) { - return toDynamicSplitRequest(approximateProgressAtIndex(index)); + return toDynamicSplitRequest(approximateSplitRequestAtIndex(index)); } public static Reader.DynamicSplitRequest splitRequestAtByteOffset(@Nullable Long byteOffset) { - return toDynamicSplitRequest(approximateProgressAtByteOffset(byteOffset)); + return toDynamicSplitRequest(approximateSplitRequestAtByteOffset(byteOffset)); } public static Reader.DynamicSplitRequest splitRequestAtConcatPosition( @Nullable Integer index, @Nullable Position innerPosition) { - return toDynamicSplitRequest(approximateProgressAtConcatPosition(index, innerPosition)); + return toDynamicSplitRequest(approximateSplitRequestAtConcatPosition(index, innerPosition)); } public static Position positionFromSplitResult(Reader.DynamicSplitResult dynamicSplitResult) { @@ -98,8 +129,8 @@ public static Position positionFromProgress(Reader.Progress progress) { return readerProgressToCloudProgress(progress).getPosition(); } - public static Reader.DynamicSplitRequest splitRequestAtFraction(float fraction) { - return toDynamicSplitRequest(approximateProgressAtFraction(fraction)); + public static Reader.DynamicSplitRequest splitRequestAtFraction(double fraction) { + return toDynamicSplitRequest(approximateSplitRequestAtFraction(fraction)); } /** diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java index abfe2342d7525..188ce9d20fda3 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/TextReaderTest.java @@ -31,7 +31,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.ApproximateReportedProgress; import com.google.api.services.dataflow.model.Position; import com.google.cloud.dataflow.sdk.TestUtils; import com.google.cloud.dataflow.sdk.coders.AtomicCoder; @@ -382,7 +382,7 @@ public void testGetProgressNoEndOffset() throws Exception { new WholeLineVerifyingCoder(), TextIO.CompressionType.UNCOMPRESSED); try (Reader.ReaderIterator iterator = textReader.iterator()) { - ApproximateProgress progress = readerProgressToCloudProgress(iterator.getProgress()); + ApproximateReportedProgress progress = readerProgressToCloudProgress(iterator.getProgress()); assertEquals(0L, progress.getPosition().getByteOffset().longValue()); iterator.next(); progress = readerProgressToCloudProgress(iterator.getProgress()); @@ -391,7 +391,7 @@ public void testGetProgressNoEndOffset() throws Exception { progress = readerProgressToCloudProgress(iterator.getProgress()); assertEquals(28L, progress.getPosition().getByteOffset().longValue()); // Since end position is not specified, percentComplete should be null. - assertNull(progress.getPercentComplete()); + assertNull(progress.getFractionConsumed()); iterator.next(); progress = readerProgressToCloudProgress(iterator.getProgress()); @@ -408,14 +408,14 @@ public void testGetProgressWithEndOffset() throws Exception { try (Reader.ReaderIterator iterator = textReader.iterator()) { iterator.next(); - ApproximateProgress progress = readerProgressToCloudProgress(iterator.getProgress()); + ApproximateReportedProgress progress = readerProgressToCloudProgress(iterator.getProgress()); // Returned a record that starts at position 0 of 40 - 1/40 fraction consumed. - assertEquals(1.0f / 40, progress.getPercentComplete(), 1e-6); + assertEquals(1.0 / 40, progress.getFractionConsumed(), 1e-6); iterator.next(); iterator.next(); progress = readerProgressToCloudProgress(iterator.getProgress()); // Returned a record that starts at position 28 - 29/40 consumed. - assertEquals(1.0f * 29 / 40, progress.getPercentComplete(), 1e-6); + assertEquals(1.0 * 29 / 40, progress.getFractionConsumed(), 1e-6); assertFalse(iterator.hasNext()); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java index ba428df4ce431..a232427315f04 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/MapTaskExecutorTest.java @@ -23,10 +23,10 @@ import static com.google.cloud.dataflow.sdk.runners.worker.ReaderTestUtils.splitRequestAtIndex; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateSplitRequest; import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind.SUM; -import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.ApproximateReportedProgress; import com.google.cloud.dataflow.sdk.util.common.Counter; import com.google.cloud.dataflow.sdk.util.common.CounterSet; import com.google.cloud.dataflow.sdk.util.common.CounterSet.AddCounterMutator; @@ -92,7 +92,7 @@ public void finish() throws Exception { // A mock ReadOperation fed to a MapTaskExecutor in test. static class TestReadOperation extends ReadOperation { - private ApproximateProgress progress = null; + private ApproximateReportedProgress progress = null; TestReadOperation(OutputReceiver outputReceiver, String counterPrefix, AddCounterMutator addCounterMutator, StateSampler stateSampler) { @@ -109,10 +109,10 @@ public Reader.Progress getProgress() { public Reader.DynamicSplitResult requestDynamicSplit(Reader.DynamicSplitRequest splitRequest) { // Fakes the return with the same position as proposed. return new Reader.DynamicSplitResultWithPosition(cloudPositionToReaderPosition( - splitRequestToApproximateProgress(splitRequest).getPosition())); + splitRequestToApproximateSplitRequest(splitRequest).getPosition())); } - public void setProgress(ApproximateProgress progress) { + public void setProgress(ApproximateReportedProgress progress) { this.progress = progress; } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java index 72c24de508b4d..03eaeef261419 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/common/worker/ReadOperationTest.java @@ -21,7 +21,7 @@ import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudPositionToReaderPosition; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.cloudProgressToReaderProgress; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.readerProgressToCloudProgress; -import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateProgress; +import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.splitRequestToApproximateSplitRequest; import static com.google.cloud.dataflow.sdk.runners.worker.SourceTranslationUtils.toCloudPosition; import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; import static com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind; @@ -37,7 +37,8 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import com.google.api.services.dataflow.model.ApproximateProgress; +import com.google.api.services.dataflow.model.ApproximateReportedProgress; +import com.google.api.services.dataflow.model.ApproximateSplitRequest; import com.google.api.services.dataflow.model.Position; import com.google.cloud.dataflow.sdk.io.range.OffsetRangeTracker; import com.google.cloud.dataflow.sdk.util.common.Counter; @@ -127,7 +128,8 @@ public void testGetProgress() throws Exception { for (int i = 0; i < 5; ++i) { sleepMillis(500); // Wait for the operation to start and block. // Ensure that getProgress() doesn't block while the next() method is blocked. - ApproximateProgress progress = readerProgressToCloudProgress(readOperation.getProgress()); + ApproximateReportedProgress progress = readerProgressToCloudProgress( + readOperation.getProgress()); long observedIndex = progress.getPosition().getRecordIndex().longValue(); assertTrue("Actual: " + observedIndex, i == observedIndex || i == observedIndex + 1); iterator.offerNext(i); @@ -161,9 +163,10 @@ public void testDynamicSplit() throws Exception { assertEquals(positionAtIndex(8L), toCloudPosition(split.getAcceptedPosition())); // Check that the progress has been recomputed. - ApproximateProgress progress = readerProgressToCloudProgress(readOperation.getProgress()); + ApproximateReportedProgress progress = readerProgressToCloudProgress( + readOperation.getProgress()); assertEquals(2, progress.getPosition().getRecordIndex().longValue()); - assertEquals(2.0f / 8.0f, progress.getPercentComplete(), 0.001f); + assertEquals(2.0f / 8.0, progress.getFractionConsumed(), 0.001); receiver.unblockProcess(); iterator.offerNext(2); @@ -314,21 +317,23 @@ protected Integer nextImpl() throws IOException { public Reader.Progress getProgress() { Preconditions.checkState(!isClosed); return cloudProgressToReaderProgress( - new ApproximateProgress().setPosition(new Position().setRecordIndex((long) current)) - .setPercentComplete((float) tracker.getFractionConsumed())); + new ApproximateReportedProgress() + .setPosition(new Position().setRecordIndex((long) current)) + .setFractionConsumed(tracker.getFractionConsumed())); } @Override public Reader.DynamicSplitResult requestDynamicSplit( Reader.DynamicSplitRequest splitRequest) { Preconditions.checkState(!isClosed); - ApproximateProgress progress = splitRequestToApproximateProgress(splitRequest); - int index = progress.getPosition().getRecordIndex().intValue(); + ApproximateSplitRequest approximateSplitRequest = splitRequestToApproximateSplitRequest( + splitRequest); + int index = approximateSplitRequest.getPosition().getRecordIndex().intValue(); if (!tracker.trySplitAtPosition(index)) { return null; } return new Reader.DynamicSplitResultWithPosition( - cloudPositionToReaderPosition(progress.getPosition())); + cloudPositionToReaderPosition(approximateSplitRequest.getPosition())); } public int offerNext(int next) { From d7643e6c5651d92f1187cce608104763d7af27ee Mon Sep 17 00:00:00 2001 From: dhalperi Date: Mon, 14 Dec 2015 15:12:34 -0800 Subject: [PATCH 1228/1541] Update OffsetBasedSource javadoc ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110197208 --- .../dataflow/sdk/io/OffsetBasedSource.java | 87 ++++++++++++------- 1 file changed, 58 insertions(+), 29 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java index 4527b85f222be..df244f28f2cf5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java @@ -28,22 +28,27 @@ import java.util.NoSuchElementException; /** - * A {@link Source} that uses offsets to define starting and ending positions. + * A {@link BoundedSource} that uses offsets to define starting and ending positions. * - *

      Extend this class to implement your own offset based custom source. + *

      {@link OffsetBasedSource} is a common base class for all bounded sources where the input can + * be represented as a single range, and an input can be efficiently processed in parallel by + * splitting the range into a set of disjoint ranges whose union is the original range. This class + * should be used for sources that can be cheaply read starting at any given offset. + * {@link OffsetBasedSource} stores the range and implements splitting into bundles. + * + *

      Extend {@link OffsetBasedSource} to implement your own offset-based custom source. * {@link FileBasedSource}, which is a subclass of this, adds additional functionality useful for * custom sources that are based on files. If possible implementors should start from - * {@code FileBasedSource} instead of {@code OffsetBasedSource}. - * - *

      This is a common base class for all sources that use an offset range. It stores the range - * and implements splitting into bundles. This should be used for sources that can be cheaply read - * starting at any given offset. + * {@link FileBasedSource} instead of {@link OffsetBasedSource}. * *

      Consult {@link RangeTracker} for important semantics common to all sources defined by a range * of positions of a certain type, including the semantics of split points * ({@link OffsetBasedReader#isAtSplitPoint}). * * @param Type of records represented by the source. + * @see BoundedSource + * @see FileBasedSource + * @see RangeTracker */ public abstract class OffsetBasedSource extends BoundedSource { private final long startOffset; @@ -53,13 +58,13 @@ public abstract class OffsetBasedSource extends BoundedSource { /** * @param startOffset starting offset (inclusive) of the source. Must be non-negative. * - * @param endOffset ending offset (exclusive) of the source. Any - * {@code offset >= getMaxEndOffset()}, e.g., {@code Long.MAX_VALUE}, means the same as - * {@code getMaxEndOffset()}. Must be {@code >= startOffset}. + * @param endOffset ending offset (exclusive) of the source. Use {@link Long#MAX_VALUE} to + * indicate that the entire source after {@code startOffset} should be read. Must be + * {@code > startOffset}. * * @param minBundleSize minimum bundle size in offset units that should be used when splitting the - * source into sub-sources. This will not be respected if the total range of - * the source is smaller than the specified {@code minBundleSize}. + * source into sub-sources. This value may not be respected if the total + * range of the source is smaller than the specified {@code minBundleSize}. * Must be non-negative. */ public OffsetBasedSource(long startOffset, long endOffset, long minBundleSize) { @@ -76,8 +81,9 @@ public long getStartOffset() { } /** - * Returns the specified ending offset of the source. If this is {@code >= getMaxEndOffset()}, - * e.g. Long.MAX_VALUE, this implies {@code getMaxEndOffset()}. + * Returns the specified ending offset of the source. Any returned value greater than or equal to + * {@link #getMaxEndOffset(PipelineOptions)} should be treated as + * {@link #getMaxEndOffset(PipelineOptions)}. */ public long getEndOffset() { return endOffset; @@ -85,7 +91,7 @@ public long getEndOffset() { /** * Returns the minimum bundle size that should be used when splitting the source into sub-sources. - * This will not be respected if the total range of the source is smaller than the specified + * This value may not be respected if the total range of the source is smaller than the specified * {@code minBundleSize}. */ public long getMinBundleSize() { @@ -155,20 +161,29 @@ public String toString() { * Returns approximately how many bytes of data correspond to a single offset in this source. * Used for translation between this source's range and methods defined in terms of bytes, such * as {@link #getEstimatedSizeBytes} and {@link #splitIntoBundles}. + * + *

      Defaults to {@code 1} byte, which is the common case for, e.g., file sources. */ public long getBytesPerOffset() { return 1L; } /** - * Returns the exact ending offset of the current source. This will be used if the source was - * constructed with an endOffset value {@code Long.MAX_VALUE}. + * Returns the actual ending offset of the current source. The value returned by this function + * will be used to clip the end of the range {@code [startOffset, endOffset)} such that the + * range used is {@code [startOffset, min(endOffset, maxEndOffset))}. + * + *

      As an example in which {@link OffsetBasedSource} is used to implement a file source, suppose + * that this source was constructed with an {@code endOffset} of {@link Long#MAX_VALUE} to + * indicate that a file should be read to the end. Then {@link #getMaxEndOffset} should determine + * the actual, exact size of the file in bytes and return it. */ public abstract long getMaxEndOffset(PipelineOptions options) throws Exception; /** - * Returns an {@code OffsetBasedSource} for a subrange of the current source. [start, end) will - * be within the range [startOffset, endOffset] of the current source. + * Returns an {@link OffsetBasedSource} for a subrange of the current source. The + * subrange {@code [start, end)} must be within the range {@code [startOffset, endOffset)} of + * the current source, i.e. {@code startOffset <= start < end <= endOffset}. */ public abstract OffsetBasedSource createSourceForSubrange(long start, long end); @@ -190,15 +205,11 @@ public abstract static class OffsetBasedReader extends BoundedReader { private OffsetBasedSource source; - /** - * The {@link OffsetRangeTracker} managing the range and current position of the source. - * Subclasses MUST use it before returning records from {@link #start} or {@link #advance}: - * see documentation of {@link RangeTracker}. - */ + /** The {@link OffsetRangeTracker} managing the range and current position of the source. */ private final OffsetRangeTracker rangeTracker; /** - * @param source the {@code OffsetBasedSource} to be read by the current reader. + * @param source the {@link OffsetBasedSource} to be read by the current reader. */ public OffsetBasedReader(OffsetBasedSource source) { this.source = source; @@ -236,14 +247,32 @@ public final boolean advance() throws IOException { } /** - * Same as {@link BoundedReader#start}, except {@link OffsetBasedReader} base class - * takes care of coordinating against concurrent calls to {@link #splitAtFraction}. + * Initializes the {@link OffsetBasedSource.OffsetBasedReader} and advances to the first record, + * returning {@code true} if there is a record available to be read. This method will be + * invoked exactly once and may perform expensive setup operations that are needed to + * initialize the reader. + * + *

      This function is the {@code OffsetBasedReader} implementation of + * {@link BoundedReader#start}. The key difference is that the implementor can ignore the + * possibility that it should no longer produce the first record, either because it has exceeded + * the original {@code endOffset} assigned to the reader, or because a concurrent call to + * {@link #splitAtFraction} has changed the source to shrink the offset range being read. + * + * @see BoundedReader#start */ protected abstract boolean startImpl() throws IOException; /** - * Same as {@link BoundedReader#advance}, except {@link OffsetBasedReader} base class - * takes care of coordinating against concurrent calls to {@link #splitAtFraction}. + * Advances to the next record and returns {@code true}, or returns false if there is no next + * record. + * + *

      This function is the {@code OffsetBasedReader} implementation of + * {@link BoundedReader#advance}. The key difference is that the implementor can ignore the + * possibility that it should no longer produce the next record, either because it has exceeded + * the original {@code endOffset} assigned to the reader, or because a concurrent call to + * {@link #splitAtFraction} has changed the source to shrink the offset range being read. + * + * @see BoundedReader#advance */ protected abstract boolean advanceImpl() throws IOException; From 5d6dfdddb020f5aba30dbe8be3139c5e169b26d3 Mon Sep 17 00:00:00 2001 From: lcwik Date: Mon, 14 Dec 2015 18:07:09 -0800 Subject: [PATCH 1229/1541] Improve error messaging when applying unsupported transform in batch ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110215096 --- .../sdk/runners/DataflowPipelineRunner.java | 43 +++++++++++------- .../runners/DataflowPipelineRunnerTest.java | 45 +++++++++++++------ 2 files changed, 58 insertions(+), 30 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index 0f9f8ac6710c5..b1a53446ba3ac 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -43,6 +43,7 @@ import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator; +import com.google.cloud.dataflow.sdk.options.StreamingOptions; import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.JobSpecification; import com.google.cloud.dataflow.sdk.runners.dataflow.CustomSources; import com.google.cloud.dataflow.sdk.runners.dataflow.DataflowAggregatorTransforms; @@ -256,15 +257,16 @@ public static DataflowPipelineRunner fromOptions(PipelineOptions options) { .put(Write.Bound.class, StreamingWrite.class) .put(PubsubIO.Write.Bound.class, StreamingPubsubIOWrite.class) .put(Read.Unbounded.class, StreamingUnboundedRead.class) - .put(Read.Bounded.class, StreamingUnsupportedIO.class) - .put(AvroIO.Read.Bound.class, StreamingUnsupportedIO.class) - .put(AvroIO.Write.Bound.class, StreamingUnsupportedIO.class) - .put(BigQueryIO.Read.Bound.class, StreamingUnsupportedIO.class) - .put(TextIO.Read.Bound.class, StreamingUnsupportedIO.class) - .put(TextIO.Write.Bound.class, StreamingUnsupportedIO.class) + .put(Read.Bounded.class, UnsupportedIO.class) + .put(AvroIO.Read.Bound.class, UnsupportedIO.class) + .put(AvroIO.Write.Bound.class, UnsupportedIO.class) + .put(BigQueryIO.Read.Bound.class, UnsupportedIO.class) + .put(TextIO.Read.Bound.class, UnsupportedIO.class) + .put(TextIO.Write.Bound.class, UnsupportedIO.class) .build(); } else { overrides = ImmutableMap., Class>builder() + .put(Read.Unbounded.class, UnsupportedIO.class) .build(); } } @@ -1036,7 +1038,7 @@ public Coder> getDefaultOutputCoder(CoderRegistry registry, Coder inp /** * Specialized expansion for unsupported IO transforms that throws an error. */ - private static class StreamingUnsupportedIO + private static class UnsupportedIO extends PTransform { private PTransform transform; @@ -1044,7 +1046,7 @@ private static class StreamingUnsupportedIO transform) { + public UnsupportedIO(AvroIO.Read.Bound transform) { this.transform = transform; } @@ -1052,7 +1054,7 @@ public StreamingUnsupportedIO(AvroIO.Read.Bound transform) { * Builds an instance of this class from the overridden transform. */ @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply() - public StreamingUnsupportedIO(BigQueryIO.Read.Bound transform) { + public UnsupportedIO(BigQueryIO.Read.Bound transform) { this.transform = transform; } @@ -1060,7 +1062,7 @@ public StreamingUnsupportedIO(BigQueryIO.Read.Bound transform) { * Builds an instance of this class from the overridden transform. */ @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply() - public StreamingUnsupportedIO(TextIO.Read.Bound transform) { + public UnsupportedIO(TextIO.Read.Bound transform) { this.transform = transform; } @@ -1068,7 +1070,7 @@ public StreamingUnsupportedIO(TextIO.Read.Bound transform) { * Builds an instance of this class from the overridden transform. */ @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply() - public StreamingUnsupportedIO(Read.Bounded transform) { + public UnsupportedIO(Read.Bounded transform) { this.transform = transform; } @@ -1076,7 +1078,7 @@ public StreamingUnsupportedIO(Read.Bounded transform) { * Builds an instance of this class from the overridden transform. */ @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply() - public StreamingUnsupportedIO(AvroIO.Write.Bound transform) { + public UnsupportedIO(Read.Unbounded transform) { this.transform = transform; } @@ -1084,17 +1086,26 @@ public StreamingUnsupportedIO(AvroIO.Write.Bound transform) { * Builds an instance of this class from the overridden transform. */ @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply() - public StreamingUnsupportedIO(TextIO.Write.Bound transform) { + public UnsupportedIO(AvroIO.Write.Bound transform) { + this.transform = transform; + } + + /** + * Builds an instance of this class from the overridden transform. + */ + @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply() + public UnsupportedIO(TextIO.Write.Bound transform) { this.transform = transform; } @Override public OutputT apply(InputT input) { + String mode = input.getPipeline().getOptions().as(StreamingOptions.class).isStreaming() + ? "streaming" : "batch"; throw new UnsupportedOperationException( - "The DataflowPipelineRunner in streaming mode does not support " - + approximatePTransformName(transform.getClass())); + String.format("The DataflowPipelineRunner in %s mode does not support %s.", + mode, approximatePTransformName(transform.getClass()))); } - } @Override diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java index d1287af33719f..3a29d82040954 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java @@ -41,12 +41,14 @@ import com.google.cloud.dataflow.sdk.io.AvroIO; import com.google.cloud.dataflow.sdk.io.AvroSource; import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions; import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptions.CheckEnabled; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.dataflow.CountingSource; import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.util.DataflowReleaseInfo; @@ -811,10 +813,10 @@ public void testToString() { DataflowPipelineRunner.fromOptions(options).toString()); } - private static PipelineOptions makeStreamingOptions() { + private static PipelineOptions makeOptions(boolean streaming) { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setRunner(DataflowPipelineRunner.class); - options.setStreaming(true); + options.setStreaming(streaming); options.setJobName("TestJobName"); options.setProject("test-project"); options.setTempLocation("gs://test/temp/location"); @@ -823,55 +825,70 @@ private static PipelineOptions makeStreamingOptions() { return options; } - private void testUnsupportedSource(PTransform source, String name) throws Exception { + private void testUnsupportedSource(PTransform source, String name, boolean streaming) + throws Exception { + String mode = streaming ? "streaming" : "batch"; thrown.expect(UnsupportedOperationException.class); thrown.expectMessage( - "The DataflowPipelineRunner in streaming mode does not support " + name); + "The DataflowPipelineRunner in " + mode + " mode does not support " + name); - Pipeline p = Pipeline.create(makeStreamingOptions()); + Pipeline p = Pipeline.create(makeOptions(streaming)); p.apply(source); p.run(); } @Test public void testBoundedSourceUnsupportedInStreaming() throws Exception { - testUnsupportedSource(AvroSource.readFromFileWithClass("foo", String.class), "Read.Bounded"); + testUnsupportedSource( + AvroSource.readFromFileWithClass("foo", String.class), "Read.Bounded", true); } @Test public void testBigQueryIOSourceUnsupportedInStreaming() throws Exception { testUnsupportedSource( - BigQueryIO.Read.from("project:bar.baz").withoutValidation(), "BigQueryIO.Read"); + BigQueryIO.Read.from("project:bar.baz").withoutValidation(), "BigQueryIO.Read", true); } @Test public void testAvroIOSourceUnsupportedInStreaming() throws Exception { - testUnsupportedSource(AvroIO.Read.from("foo"), "AvroIO.Read"); + testUnsupportedSource( + AvroIO.Read.from("foo"), "AvroIO.Read", true); } @Test public void testTextIOSourceUnsupportedInStreaming() throws Exception { - testUnsupportedSource(TextIO.Read.from("foo"), "TextIO.Read"); + testUnsupportedSource(TextIO.Read.from("foo"), "TextIO.Read", true); } - private void testUnsupportedSink(PTransform, PDone> sink, String name) - throws Exception { + @Test + public void testReadBoundedSourceUnsupportedInStreaming() throws Exception { + testUnsupportedSource(Read.from(AvroSource.from("/tmp/test")), "Read.Bounded", true); + } + + @Test + public void testReadUnboundedUnsupportedInBatch() throws Exception { + testUnsupportedSource(Read.from(new CountingSource(1)), "Read.Unbounded", false); + } + + private void testUnsupportedSink( + PTransform, PDone> sink, String name, boolean streaming) + throws Exception { thrown.expect(UnsupportedOperationException.class); thrown.expectMessage( "The DataflowPipelineRunner in streaming mode does not support " + name); - Pipeline p = Pipeline.create(makeStreamingOptions()); + Pipeline p = Pipeline.create(makeOptions(streaming)); p.apply(Create.of("foo")).apply(sink); p.run(); } @Test public void testAvroIOSinkUnsupportedInStreaming() throws Exception { - testUnsupportedSink(AvroIO.Write.to("foo").withSchema(String.class), "AvroIO.Write"); + testUnsupportedSink(AvroIO.Write.to("foo").withSchema(String.class), "AvroIO.Write", true); } @Test public void testTextIOSinkUnsupportedInStreaming() throws Exception { - testUnsupportedSink(TextIO.Write.to("foo"), "TextIO.Write"); + testUnsupportedSink(TextIO.Write.to("foo"), "TextIO.Write", true); } } From 37e7fcf7a9038ea91c6f62a9d38e5eca33c81dae Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 14 Dec 2015 19:48:41 -0800 Subject: [PATCH 1230/1541] Drop data only for expired windows In streaming mode, a GroupByKey operation now drops late data only when it is in a window that has expired -- the end of the window is behind the input watermark by more than the allowed lateness. When an element is assigned to multiple windows, each occurrence is treated independently. Previously, data was dropped if the element's timestamp was beyond the allowed lateness. The new condition drops strictly less data. ----Release Notes---- - Only drop late data if the window it is assigned to has expired (the end of the window is passed by more than the allowed lateness) [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110220195 --- .../dataflow/sdk/util/ReduceFnRunner.java | 58 +++++++++------ .../dataflow/sdk/util/WatermarkHold.java | 18 ----- .../dataflow/sdk/util/ReduceFnTester.java | 21 ++++-- .../sdk/util/TriggerExecutorTest.java | 70 +++++++++++++++---- 4 files changed, 108 insertions(+), 59 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java index 742806db48110..4d11e8a708110 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java @@ -34,6 +34,8 @@ import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; import com.google.common.base.Throwables; import com.google.common.collect.FluentIterable; import com.google.common.collect.Maps; @@ -247,12 +249,15 @@ public void onMerge(Collection mergedWindows, W resultWindow, boolean isResul return sourceWindowsToResultWindows; } - /** Is {@code value} late w.r.t. the garbage collection watermark? */ - private boolean canDropDueToLateness(WindowedValue value) { - Instant inputWM = timerInternals.currentInputWatermarkTime(); - return inputWM != null - && value.getTimestamp().isBefore(inputWM.minus(windowingStrategy.getAllowedLateness())); - } + /** Is the {@code window} expired w.r.t. the garbage collection watermark? */ + private Predicate canDropDueToExpiredWindow = new Predicate() { + @Override + public boolean apply(W window) { + Instant inputWM = timerInternals.currentInputWatermarkTime(); + return inputWM != null + && window.maxTimestamp().plus(windowingStrategy.getAllowedLateness()).isBefore(inputWM); + } + }; /** * Add the initial windows from each of the values to the active window set. Returns the set of @@ -261,15 +266,17 @@ private boolean canDropDueToLateness(WindowedValue value) { private Set addToActiveWindows(Iterable> values) { Set newWindows = new HashSet<>(); for (WindowedValue value : values) { - if (canDropDueToLateness(value)) { - // This value will be dropped (and reported in a counter) by processElement. - // Hence it won't contribute to any new window. - continue; - } for (BoundedWindow untypedWindow : value.getWindows()) { @SuppressWarnings("unchecked") W window = (W) untypedWindow; + + if (canDropDueToExpiredWindow.apply(window)) { + // This value will be dropped (and reported in a counter) by processElement. + // Hence it won't contribute to any new window. + continue; + } + ReduceFn.Context context = contextFactory.base(window); if (!triggerRunner.isClosed(context.state())) { if (activeWindows.add(window)) { @@ -291,22 +298,27 @@ private Set addToActiveWindows(Iterable> values) { */ private void processElement( Function windowMapping, Map results, WindowedValue value) { - if (canDropDueToLateness(value)) { - // Drop the element in all assigned windows if it is past the allowed lateness limit. - droppedDueToLateness.addValue((long) value.getWindows().size()); - WindowTracing.debug( - "processElement: Dropping element at {} for key:{} since too far " - + "behind inputWatermark:{}; outputWatermark:{}", - value.getTimestamp(), key, timerInternals.currentInputWatermarkTime(), - timerInternals.currentOutputWatermarkTime()); - return; - } // Only consider representative windows from among all windows in equivalence classes // induced by window merging. @SuppressWarnings("unchecked") - Iterable windows = - FluentIterable.from((Collection) value.getWindows()).transform(windowMapping); + FluentIterable mappedWindows = + FluentIterable.from((Collection) value.getWindows()) + .transform(windowMapping); + + // Some windows may be expired + Iterable windows = mappedWindows.filter(Predicates.not(canDropDueToExpiredWindow)); + + // Count the number of elements that are dropped + for (W expiredWindow : mappedWindows.filter(canDropDueToExpiredWindow)) { + droppedDueToLateness.addValue(1L); + WindowTracing.debug( + "processElement: Dropping element at {} for key:{} and window:{} since window is " + + "too far behind inputWatermark:{}; outputWatermark:{}", + value.getTimestamp(), key, expiredWindow, + timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + } // Prefetch in each of the windows if we're going to need to process triggers for (W window : windows) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java index cee3eeb0b95ed..bac3af11f7b58 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java @@ -116,18 +116,6 @@ public WatermarkHold(TimerInternals timerInternals, WindowingStrategy wind * The element is very locally late. The window has been garbage collected, thus there * is no target pane E could be assigned to. We discard E. * - *

    • (Discard - beyond allowed lateness) - *
      -   *               |                            |
      -   *      [     E  |     ]                      |
      -   *               |                            |
      -   *             GCWM  <-getAllowedLateness->  IWM
      -   * 
      - * The element is very locally late, and the window is very close to being garbage collected, at - * which point a final {@code LATE} pane could be emitted. We *could* attempt to capture E within - * that pane, however that requires checking against all possible windows which may contain E. - * We instead discard E. - * *
    • (Unobservably late) *
          *          |    |
      @@ -226,12 +214,6 @@ private boolean addElementHold(ReduceFn.ProcessValueContext context)
           Instant outputWM = timerInternals.currentOutputWatermarkTime();
           Instant inputWM = timerInternals.currentInputWatermarkTime();
       
      -    Instant garbageWM =
      -        inputWM == null ? null : inputWM.minus(windowingStrategy.getAllowedLateness());
      -    Preconditions.checkState(garbageWM == null || !elementHold.isBefore(garbageWM),
      -        "Shifted timestamp %s cannot be beyond garbage collection watermark %s", elementHold,
      -        garbageWM);
      -
           // Only add the hold if we can be sure the backend will be able to respect it.
           boolean tooLate;
           if (outputWM != null && elementHold.isBefore(outputWM)) {
      diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReduceFnTester.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReduceFnTester.java
      index 4413d0630890a..b789d6bd33613 100644
      --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReduceFnTester.java
      +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReduceFnTester.java
      @@ -122,12 +122,9 @@ public class ReduceFnTester {
         }
       
         public static  ReduceFnTester
      -      combining(WindowFn windowFn, Trigger trigger, AccumulationMode mode,
      -          KeyedCombineFn combineFn, Coder outputCoder,
      -          Duration allowedDataLateness) throws Exception {
      -    WindowingStrategy strategy =
      -        WindowingStrategy.of(windowFn).withTrigger(trigger).withMode(mode).withAllowedLateness(
      -            allowedDataLateness);
      +      combining(WindowingStrategy strategy,
      +          KeyedCombineFn combineFn,
      +          Coder outputCoder) throws Exception {
       
           CoderRegistry registry = new CoderRegistry();
           registry.registerStandardCoders();
      @@ -142,6 +139,18 @@ public class ReduceFnTester {
               outputCoder);
         }
       
      +  public static  ReduceFnTester
      +      combining(WindowFn windowFn, Trigger trigger, AccumulationMode mode,
      +          KeyedCombineFn combineFn, Coder outputCoder,
      +          Duration allowedDataLateness) throws Exception {
      +
      +    WindowingStrategy strategy =
      +        WindowingStrategy.of(windowFn).withTrigger(trigger).withMode(mode).withAllowedLateness(
      +            allowedDataLateness);
      +
      +    return combining(strategy, combineFn, outputCoder);
      +  }
      +
         private ReduceFnTester(WindowingStrategy wildcardStrategy,
             ReduceFn reduceFn, Coder outputCoder) throws Exception {
           @SuppressWarnings("unchecked")
      diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerExecutorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerExecutorTest.java
      index 164ab3f65183a..5e7681064c780 100644
      --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerExecutorTest.java
      +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/TriggerExecutorTest.java
      @@ -526,33 +526,79 @@ public void testMergeBeforeFinalizing() throws Exception {
               Matchers.equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0)));
         }
       
      +  /**
      +   * Tests that when data is assigned to multiple windows but some of those windows have expired,
      +   * then the data is dropped and counted accurately.
      +   */
         @Test
      -  public void testDropDataMultipleWindows() throws Exception {
      -    ReduceFnTester tester = ReduceFnTester.combining(
      -        SlidingWindows.of(Duration.millis(100)).every(Duration.millis(30)),
      -        AfterWatermark.pastEndOfWindow(), AccumulationMode.ACCUMULATING_FIRED_PANES,
      -        new Sum.SumIntegerFn().asKeyedFn(), VarIntCoder.of(), Duration.millis(20));
      +  public void testDropDataMultipleWindowsExpiredWindow() throws Exception {
      +    ReduceFnTester, IntervalWindow> tester = ReduceFnTester.nonCombining(
      +        WindowingStrategy.of(
      +            SlidingWindows.of(Duration.millis(100)).every(Duration.millis(30)))
      +        .withAllowedLateness(Duration.millis(10)));
       
           tester.injectElements(
      -        TimestampedValue.of(10, new Instant(23)), // [-60, 40), [-30, 70), [0, 100)
      +        // assigned to [-60, 40), [-30, 70), [0, 100)
      +        TimestampedValue.of(10, new Instant(23)),
      +        // assigned to [-30, 70), [0, 100), [30, 130)
               TimestampedValue.of(12, new Instant(40)));
      -        // [-30, 70), [0, 100), [30, 130)
       
           assertEquals(0, tester.getElementsDroppedDueToLateness());
      -    assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
       
           tester.advanceInputWatermark(new Instant(70));
      +
      +
           tester.injectElements(
      -        TimestampedValue.of(14, new Instant(60))); // [-30, 70) = closed, [0, 100), [30, 130)
      +        // assigned to [-30, 70), [0, 100), [30, 130)
      +        // but [-30, 70) has past but is not is expired
      +        TimestampedValue.of(14, new Instant(50)));
       
           assertEquals(0, tester.getElementsDroppedDueToLateness());
      -    assertEquals(1, tester.getElementsDroppedDueToClosedWindow());
       
      +    tester.advanceInputWatermark(new Instant(110));
      +
      +    // assigned to [-30, 70), [0, 100), [30, 130)
      +    // but the first two are expired
           tester.injectElements(TimestampedValue.of(16, new Instant(40)));
      -        // dropped b/c lateness, assigned to 3 windows
       
      -    assertEquals(3, tester.getElementsDroppedDueToLateness());
      +    assertEquals(2, tester.getElementsDroppedDueToLateness());
      +  }
      +
      +  /**
      +   * Tests that when data is assigned to multiple windows but some of those windows have
      +   * had their triggers finish, then the data is dropped and counted accurately.
      +   */
      +  @Test
      +  public void testDropDataMultipleWindowsFinishedTrigger() throws Exception {
      +    ReduceFnTester tester = ReduceFnTester.combining(
      +        WindowingStrategy.of(
      +            SlidingWindows.of(Duration.millis(100)).every(Duration.millis(30)))
      +        .withTrigger(AfterWatermark.pastEndOfWindow())
      +        .withAllowedLateness(Duration.millis(1000)),
      +        new Sum.SumIntegerFn().asKeyedFn(), VarIntCoder.of());
      +
      +    tester.injectElements(
      +        // assigned to [-60, 40), [-30, 70), [0, 100)
      +        TimestampedValue.of(10, new Instant(23)),
      +        // assigned to [-30, 70), [0, 100), [30, 130)
      +        TimestampedValue.of(12, new Instant(40)));
      +
      +    assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
      +
      +    tester.advanceInputWatermark(new Instant(70));
      +    tester.injectElements(
      +        // assigned to [-30, 70), [0, 100), [30, 130)
      +        // but [-30, 70) is closed by the trigger
      +        TimestampedValue.of(14, new Instant(60)));
      +
           assertEquals(1, tester.getElementsDroppedDueToClosedWindow());
      +
      +    tester.advanceInputWatermark(new Instant(130));
      +    // assigned to [-30, 70), [0, 100), [30, 130)
      +    // but they are all closed
      +    tester.injectElements(TimestampedValue.of(16, new Instant(40)));
      +
      +    assertEquals(4, tester.getElementsDroppedDueToClosedWindow());
         }
       
         @Test
      
      From 2398d0b52b49efb0b2d9c374afb214c37176ed56 Mon Sep 17 00:00:00 2001
      From: klk 
      Date: Mon, 14 Dec 2015 20:27:05 -0800
      Subject: [PATCH 1231/1541] Do not require .withAllowedLateness for
       GlobalWindows
      
      ----Release Notes----
       - When using GlobalWindows it is no longer required to provide
         the .withAllowedLateness parameter - no data is ever dropped.
      
      []
      -------------
      Created by MOE: https://github.com/google/moe
      MOE_MIGRATED_REVID=110221966
      ---
       .../cloud/dataflow/sdk/transforms/windowing/Window.java    | 7 +++++--
       1 file changed, 5 insertions(+), 2 deletions(-)
      
      diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java
      index 531392b551b92..dfdc25903d2ef 100644
      --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java
      +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java
      @@ -559,9 +559,12 @@ public void validate(PCollection input) {
             // Make sure that the windowing strategy is complete & valid.
             if (outputStrategy.isTriggerSpecified()
                 && !(outputStrategy.getTrigger().getSpec() instanceof DefaultTrigger)) {
      -        if (!outputStrategy.isAllowedLatenessSpecified()) {
      +
      +        if (!(outputStrategy.getWindowFn() instanceof GlobalWindows)
      +            && !outputStrategy.isAllowedLatenessSpecified()) {
                 throw new IllegalArgumentException(
      -              "Calling .triggering() to specify a trigger requires that the allowed lateness be"
      +              "Except when using GlobalWindows,"
      +              + " calling .triggering() to specify a trigger requires that the allowed lateness be"
                     + " specified using .withAllowedLateness() to set the upper bound on how late data"
                     + " can arrive before being dropped. See Javadoc for more details.");
               }
      
      From d459ecff6b80979eaa4ee9a5bb62bb1a61c25e83 Mon Sep 17 00:00:00 2001
      From: klk 
      Date: Mon, 14 Dec 2015 20:39:22 -0800
      Subject: [PATCH 1232/1541] Remove Trigger.OnElementContext.element()
      
      This method is not useful for well-behaved triggers, and is
      unused in the SDK.
      
      ----Release Notes----
      []
      -------------
      Created by MOE: https://github.com/google/moe
      MOE_MIGRATED_REVID=110222447
      ---
       .../dataflow/sdk/transforms/windowing/Trigger.java    |  3 ---
       .../dataflow/sdk/util/TriggerContextFactory.java      | 11 ++---------
       2 files changed, 2 insertions(+), 12 deletions(-)
      
      diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Trigger.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Trigger.java
      index e6fc3cd54f6ea..f73401f79315f 100644
      --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Trigger.java
      +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Trigger.java
      @@ -266,9 +266,6 @@ public abstract class TriggerContext {
          * Details about an invocation of {@link Trigger#onElement}.
          */
         public abstract class OnElementContext extends TriggerContext {
      -    /** The element being handled by this call to {@link Trigger#onElement}. */
      -    public abstract Object element();
      -
           /** The event timestamp of the element currently being processed. */
           public abstract Instant eventTimestamp();
       
      diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerContextFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerContextFactory.java
      index 7627c5c3cf505..64c6de39b36eb 100644
      --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerContextFactory.java
      +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerContextFactory.java
      @@ -66,7 +66,7 @@ public Trigger.OnElementContext create(
             ExecutableTrigger rootTrigger, BitSet finishedSet) {
           return new OnElementContextImpl(
               context.window(), context.timers(), rootTrigger, finishedSet,
      -        context.value(), context.timestamp());
      +        context.timestamp());
         }
       
         public Trigger.OnTimerContext create(
      @@ -310,7 +310,6 @@ private class OnElementContextImpl extends Trigger.OnElementContext {
           private final StateContextImpl state;
           private final Timers timers;
           private final TriggerInfoImpl triggerInfo;
      -    private final Object element;
           private final Instant eventTimestamp;
       
           private OnElementContextImpl(
      @@ -318,20 +317,14 @@ private OnElementContextImpl(
               Timers timers,
               ExecutableTrigger trigger,
               BitSet finishedSet,
      -        Object element,
               Instant eventTimestamp) {
             trigger.getSpec().super();
             this.state = triggerState(window, trigger);
             this.timers = new TriggerTimers(window, timers);
             this.triggerInfo = new TriggerInfoImpl(trigger, finishedSet, this);
      -      this.element = element;
             this.eventTimestamp = eventTimestamp;
           }
       
      -    @Override
      -    public Object element() {
      -      return element;
      -    }
       
           @Override
           public Instant eventTimestamp() {
      @@ -341,7 +334,7 @@ public Instant eventTimestamp() {
           @Override
           public Trigger.OnElementContext forTrigger(ExecutableTrigger trigger) {
             return new OnElementContextImpl(
      -          state.window(), timers, trigger, triggerInfo.finishedSet, element, eventTimestamp);
      +          state.window(), timers, trigger, triggerInfo.finishedSet, eventTimestamp);
           }
       
           @Override
      
      From 57de437a77961400dd52bd8c6dbd4f5b4b72d931 Mon Sep 17 00:00:00 2001
      From: bchambers 
      Date: Wed, 16 Dec 2015 09:26:39 -0800
      Subject: [PATCH 1233/1541] Change the assembly of state keys to be more
       efficient
      
      Rather than using multiple calls to construct strings and
      compose them into bigger strings using String.format(),
      this appends the parts into a StringBuilder.
      
      ----Release Notes----
      
      []
      -------------
      Created by MOE: https://github.com/google/moe
      MOE_MIGRATED_REVID=110367598
      ---
       .../worker/WindmillStateInternals.java        | 30 +++++++++++++------
       .../sdk/util/state/StateNamespace.java        |  7 +++++
       .../sdk/util/state/StateNamespaceForTest.java |  6 ++++
       .../sdk/util/state/StateNamespaces.java       | 18 +++++++++++
       .../dataflow/sdk/util/state/StateTag.java     |  4 +++
       .../dataflow/sdk/util/state/StateTags.java    |  9 ++++++
       6 files changed, 65 insertions(+), 9 deletions(-)
      
      diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java
      index 63a0256e072da..91dc2236c5622 100644
      --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java
      +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java
      @@ -107,7 +107,8 @@ public  ValueState bindValue(StateTag> address, Coder cod
       
         @VisibleForTesting
         static final ThreadLocal> COMPACT_NOW =
      -      new ThreadLocal() {
      +      new ThreadLocal>() {
      +        @Override
               public Supplier initialValue() {
                 return new Supplier() {
                   /* The rate at which, on average, this will return true. */
      @@ -121,6 +122,7 @@ private long nextSample() {
                     return (long) Math.floor(Math.log(random.nextDouble()) / Math.log(1 - RATE));
                   }
       
      +            @Override
                   public Boolean get() {
                     counter--;
                     if (counter < 0) {
      @@ -182,18 +184,28 @@ public void persist(final Windmill.WorkItemCommitRequest.Builder commitBuilder)
           }
         }
       
      -  private ByteString encodeKey(StateNamespace namespace, StateTag address) {
      -    if (useStateFamilies) {
      -      // We don't use prefix here, since it's being set as the stateFamily.
      -      return ByteString.copyFromUtf8(
      -          String.format("%s+%s", namespace.stringKey(), address.getId()));
      -    } else {
      +  @VisibleForTesting ByteString encodeKey(StateNamespace namespace, StateTag address) {
      +    try {
      +      // Use a StringBuilder rather than concatenation and String.format. We build these keys
      +      // a lot, and this leads to better performance results. See associated benchmarks.
      +      StringBuilder output = new StringBuilder();
      +
      +      // We only need the prefix if we aren't using state families
      +      if (!useStateFamilies) {
      +        output.append(prefix);
      +      }
      +
             // stringKey starts and ends with a slash. We don't need to seperate it from prefix, because
             // the prefix is guaranteed to be unique and non-overlapping. We separate it from the
             // StateTag ID by a '+' (which is guaranteed not to be in the stringKey) because the
             // ID comes from the user.
      -      return ByteString.copyFromUtf8(String.format(
      -          "%s%s+%s", prefix, namespace.stringKey(), address.getId()));
      +      namespace.appendTo(output);
      +      output.append('+');
      +      address.appendTo(output);
      +      return ByteString.copyFromUtf8(output.toString());
      +    } catch (IOException e) {
      +      throw new RuntimeException(
      +          "Unable to encode state key for " + namespace + ", " + address, e);
           }
         }
       
      diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespace.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespace.java
      index 423d30e2f4caf..4a0364f1db88e 100644
      --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespace.java
      +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespace.java
      @@ -15,6 +15,8 @@
        */
       package com.google.cloud.dataflow.sdk.util.state;
       
      +import java.io.IOException;
      +
       /**
        * A namespace used for scoping state stored with {@link StateInternals}.
        *
      @@ -35,4 +37,9 @@ public interface StateNamespace {
          * the two.
          */
         String stringKey();
      +
      +  /**
      +   * Append the string representation of this key to the {@link Appendable}.
      +   */
      +  void appendTo(Appendable sb) throws IOException;
       }
      diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaceForTest.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaceForTest.java
      index 000244c7aeadb..c11668fd62aef 100644
      --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaceForTest.java
      +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaceForTest.java
      @@ -15,6 +15,7 @@
        */
       package com.google.cloud.dataflow.sdk.util.state;
       
      +import java.io.IOException;
       import java.util.Objects;
       
       /**
      @@ -49,4 +50,9 @@ public boolean equals(Object obj) {
         public int hashCode() {
           return key.hashCode();
         }
      +
      +  @Override
      +  public void appendTo(Appendable sb) throws IOException {
      +    sb.append(key);
      +  }
       }
      diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaces.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaces.java
      index 938cf12c87d63..22115847a3f65 100644
      --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaces.java
      +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaces.java
      @@ -21,6 +21,7 @@
       import com.google.cloud.dataflow.sdk.util.CoderUtils;
       import com.google.common.base.Splitter;
       
      +import java.io.IOException;
       import java.util.List;
       import java.util.Objects;
       
      @@ -76,6 +77,11 @@ public int hashCode() {
           public String toString() {
             return "Global";
           }
      +
      +    @Override
      +    public void appendTo(Appendable sb) throws IOException {
      +      sb.append(GLOBAL_STRING);
      +    }
         }
       
         /**
      @@ -106,6 +112,11 @@ public String stringKey() {
             }
           }
       
      +    @Override
      +    public void appendTo(Appendable sb) throws IOException {
      +      sb.append('/').append(CoderUtils.encodeToBase64(windowCoder, window)).append('/');
      +    }
      +
           @Override
           public boolean equals(Object obj) {
             if (obj == this) {
      @@ -171,6 +182,13 @@ public String stringKey() {
             }
           }
       
      +    @Override
      +    public void appendTo(Appendable sb) throws IOException {
      +      sb.append('/').append(CoderUtils.encodeToBase64(windowCoder, window));
      +      sb.append('/').append(Integer.toString(triggerIndex, TRIGGER_RADIX).toUpperCase());
      +      sb.append('/');
      +    }
      +
           @Override
           public boolean equals(Object obj) {
             if (obj == this) {
      diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTag.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTag.java
      index 2419bb8ed6007..cba405d81e641 100644
      --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTag.java
      +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTag.java
      @@ -23,6 +23,7 @@
       import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
       import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
       
      +import java.io.IOException;
       import java.io.Serializable;
       
       /**
      @@ -63,6 +64,9 @@  WatermarkStateInternal bindWatermark(
               OutputTimeFn outputTimeFn);
         }
       
      +  /** Append the UTF-8 encoding of this tag to the given {@link Appendable}. */
      +  void appendTo(Appendable sb) throws IOException;
      +
         /**
          * Returns the identifier for this state cell.
          */
      diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java
      index 4c6af1ae32d18..f6f0c84e7dc41 100644
      --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java
      +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java
      @@ -25,6 +25,7 @@
       import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
       import com.google.common.base.MoreObjects;
       
      +import java.io.IOException;
       import java.io.Serializable;
       import java.util.Objects;
       
      @@ -153,6 +154,10 @@ public String getIdString() {
             return kind.prefix + rawId;
           }
       
      +    public void appendTo(Appendable sb) throws IOException {
      +      sb.append(kind.prefix).append(rawId);
      +    }
      +
           @Override
           public String toString() {
             return MoreObjects.toStringHelper(getClass())
      @@ -206,6 +211,10 @@ public String toString() {
           }
       
           protected abstract StateTag asKind(StateKind kind);
      +
      +    public void appendTo(Appendable sb) throws IOException {
      +      id.appendTo(sb);
      +    }
         }
       
         /**
      
      From 1f072e803be715dc54f143cfb2f8cf536f9d9131 Mon Sep 17 00:00:00 2001
      From: klk 
      Date: Thu, 17 Dec 2015 16:12:03 -0800
      Subject: [PATCH 1234/1541] Improve javadoc for Pipeline, PipelineResult
      
      ----Release Notes----
      
      []
      -------------
      Created by MOE: https://github.com/google/moe
      MOE_MIGRATED_REVID=110499179
      ---
       .../google/cloud/dataflow/sdk/Pipeline.java   | 113 ++++++++++--------
       .../cloud/dataflow/sdk/PipelineResult.java    |  25 ++--
       2 files changed, 75 insertions(+), 63 deletions(-)
      
      diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java
      index f2e7d72804c7b..b166673e6e9c7 100644
      --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java
      +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/Pipeline.java
      @@ -17,15 +17,18 @@
       package com.google.cloud.dataflow.sdk;
       
       import com.google.cloud.dataflow.sdk.coders.CoderRegistry;
      +import com.google.cloud.dataflow.sdk.io.Read;
       import com.google.cloud.dataflow.sdk.options.PipelineOptions;
       import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
       import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
       import com.google.cloud.dataflow.sdk.runners.TransformHierarchy;
       import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
       import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
      +import com.google.cloud.dataflow.sdk.transforms.Create;
       import com.google.cloud.dataflow.sdk.transforms.PTransform;
       import com.google.cloud.dataflow.sdk.util.UserCodeException;
       import com.google.cloud.dataflow.sdk.values.PBegin;
      +import com.google.cloud.dataflow.sdk.values.PCollection;
       import com.google.cloud.dataflow.sdk.values.PInput;
       import com.google.cloud.dataflow.sdk.values.POutput;
       import com.google.cloud.dataflow.sdk.values.PValue;
      @@ -44,27 +47,26 @@
       import java.util.Set;
       
       /**
      - * A {@code Pipeline} manages a DAG of {@link PTransform}s, and the
      - * {@link com.google.cloud.dataflow.sdk.values.PCollection}s
      - * that the {@link PTransform}s consume and produce.
      + * A {@link Pipeline} manages a directed acyclic graph of {@link PTransform PTransforms}, and the
      + * {@link PCollection PCollections} that the {@link PTransform}s consume and produce.
        *
      - * 

      After a {@code Pipeline} has been constructed, it can be executed, - * using a default or an explicit {@link PipelineRunner}. + *

      A {@link Pipeline} is initialized with a {@link PipelineRunner} that will later + * execute the {@link Pipeline}. * - *

      Multiple {@code Pipeline}s can be constructed and executed independently - * and concurrently. + *

      {@link Pipeline Pipelines} are independent, so they can be constructed and executed + * concurrently. * - *

      Each {@code Pipeline} is self-contained and isolated from any other - * {@code Pipeline}. The {@link PValue PValues} that are inputs and outputs of each of a - * {@code Pipeline}'s {@link PTransform PTransforms} are also owned by that {@code Pipeline}. - * A {@code PValue} owned by one {@code Pipeline} can be read only by {@code PTransform}s - * also owned by that {@code Pipeline}. + *

      Each {@link Pipeline} is self-contained and isolated from any other + * {@link Pipeline}. The {@link PValue PValues} that are inputs and outputs of each of a + * {@link Pipeline Pipeline's} {@link PTransform PTransforms} are also owned by that + * {@link Pipeline}. A {@link PValue} owned by one {@link Pipeline} can be read only by + * {@link PTransform PTransforms} also owned by that {@link Pipeline}. * - *

      Here's a typical example of use: + *

      Here is a typical example of use: *

       {@code
        * // Start by defining the options for the pipeline.
        * PipelineOptions options = PipelineOptionsFactory.create();
      - * // Then create the pipeline.
      + * // Then create the pipeline. The runner is determined by the options.
        * Pipeline p = Pipeline.create(options);
        *
        * // A root PTransform, like TextIO.Read or Create, gets added
      @@ -106,13 +108,14 @@ public class Pipeline {
         private static final Logger LOG = LoggerFactory.getLogger(Pipeline.class);
       
         /**
      -   * Thrown during pipeline execution, whenever user code within a pipeline throws an exception.
      +   * Thrown during execution of a {@link Pipeline}, whenever user code within that
      +   * {@link Pipeline} throws an exception.
          *
      -   * 

      The exception thrown during pipeline execution may be retrieved via {@link #getCause}. + *

      The original exception thrown by user code may be retrieved via {@link #getCause}. */ public static class PipelineExecutionException extends RuntimeException { /** - * Wraps {@code cause} into a {@code PipelineExecutionException}. + * Wraps {@code cause} into a {@link PipelineExecutionException}. */ public PipelineExecutionException(Throwable cause) { super(cause); @@ -135,16 +138,18 @@ public static Pipeline create(PipelineOptions options) { /** * Returns a {@link PBegin} owned by this Pipeline. This is useful - * as the input of a root PTransform such as {@code TextIO.Read} or - * {@link com.google.cloud.dataflow.sdk.transforms.Create}. + * as the input of a root PTransform such as {@link Read} or + * {@link Create}. */ public PBegin begin() { return PBegin.in(this); } /** - * Like {@link #apply(String, PTransform)} but defaulting to the name - * of the {@code PTransform}. + * Like {@link #apply(String, PTransform)} but the transform node in the {@link Pipeline} + * graph will be named according to {@link PTransform#getName}. + * + * @see #apply(String, PTransform) */ public OutputT apply( PTransform root) { @@ -152,11 +157,12 @@ public OutputT apply( } /** - * Starts using this pipeline with a root {@code PTransform} such as - * {@code TextIO.READ} or {@link com.google.cloud.dataflow.sdk.transforms.Create}. - * This specific call to {@code apply} is identified by the provided {@code name}. + * Adds a root {@link PTransform}, such as {@link Read} or {@link Create}, + * to this {@link Pipeline}. + * + *

      The node in the {@link Pipeline} graph will use the provided {@code name}. * This name is used in various places, including the monitoring UI, logging, - * and to stably identify this application node in the job graph. + * and to stably identify this node in the {@link Pipeline} graph upon update. * *

      Alias for {@code begin().apply(name, root)}. */ @@ -166,7 +172,7 @@ public OutputT apply( } /** - * Runs the Pipeline. + * Runs the {@link Pipeline} using its {@link PipelineRunner}. */ public PipelineResult run() { LOG.debug("Running {} via {}", this, runner); @@ -186,7 +192,7 @@ public PipelineResult run() { // Below here are operations that aren't normally called by users. /** - * Returns the {@link CoderRegistry} that this Pipeline uses. + * Returns the {@link CoderRegistry} that this {@link Pipeline} uses. */ public CoderRegistry getCoderRegistry() { if (coderRegistry == null) { @@ -197,7 +203,7 @@ public CoderRegistry getCoderRegistry() { } /** - * Sets the {@link CoderRegistry} that this Pipeline uses. + * Sets the {@link CoderRegistry} that this {@link Pipeline} uses. */ public void setCoderRegistry(CoderRegistry coderRegistry) { this.coderRegistry = coderRegistry; @@ -206,17 +212,17 @@ public void setCoderRegistry(CoderRegistry coderRegistry) { /** * A {@link PipelineVisitor} can be passed into * {@link Pipeline#traverseTopologically} to be called for each of the - * transforms and values in the Pipeline. + * transforms and values in the {@link Pipeline}. */ public interface PipelineVisitor { /** * Called for each composite transform after all topological predecessors have been visited - * but before any of the component transforms. + * but before any of its component transforms. */ public void enterCompositeTransform(TransformTreeNode node); /** - * Called for each composite transform after all of its component transforms and their ouputs + * Called for each composite transform after all of its component transforms and their outputs * have been visited. */ public void leaveCompositeTransform(TransformTreeNode node); @@ -235,14 +241,15 @@ public interface PipelineVisitor { } /** - * Invokes the PipelineVisitor's + * Invokes the {@link PipelineVisitor PipelineVisitor's} * {@link PipelineVisitor#visitTransform} and * {@link PipelineVisitor#visitValue} operations on each of this - * Pipeline's PTransforms and PValues, in forward + * {@link Pipeline Pipeline's} transform and value nodes, in forward * topological order. * - *

      Traversal of the pipeline causes PTransform and PValue instances to - * be marked as finished, at which point they may no longer be modified. + *

      Traversal of the {@link Pipeline} causes {@link PTransform PTransforms} and + * {@link PValue PValues} owned by the {@link Pipeline} to be marked as finished, + * at which point they may no longer be modified. * *

      Typically invoked by {@link PipelineRunner} subclasses. */ @@ -271,9 +278,11 @@ OutputT applyTransform(InputT input, * Applies the given {@code PTransform} to this input {@code InputT} and returns * its {@code OutputT}. This uses {@code name} to identify this specific application * of the transform. This name is used in various places, including the monitoring UI, - * logging, and to stably identify this application node in the job graph. + * logging, and to stably identify this application node in the {@link Pipeline} graph during + * update. * - *

      Called by {@link PInput} subclasses in their {@code apply} methods. + *

      Each {@link PInput} subclass that provides an {@code apply} method should delegate to + * this method to ensure proper registration with the {@link PipelineRunner}. */ public static OutputT applyTransform(String name, InputT input, @@ -312,7 +321,7 @@ public String toString() { } /** - * Applies a transformation to the given input. + * Applies a {@link PTransform} to the given {@link PInput}. * * @see Pipeline#apply */ @@ -386,16 +395,17 @@ OutputT applyInternal(String name, InputT input, } /** - * Verifies that the output of a PTransform is correctly defined. + * Verifies that the output of a {@link PTransform} is correctly configured in its + * {@link TransformTreeNode} in the {@link Pipeline} graph. * - *

      A non-composite transform must have all - * of its outputs registered as produced by the transform. + *

      A non-composite {@link PTransform} must have all + * of its outputs registered as produced by that {@link PTransform}. * - *

      A composite transform must have all of its outputs - * registered as produced by the contained primitive transforms. + *

      A composite {@link PTransform} must have all of its outputs + * registered as produced by the contained primitive {@link PTransform PTransforms}. * They have each had the above check performed already, when * they were applied, so the only possible failure state is - * that the composite transform has returned a primitive output. + * that the composite {@link PTransform} has returned a primitive output. */ private void verifyOutputState(POutput output, TransformTreeNode node) { if (!node.isCompositeNode()) { @@ -427,24 +437,23 @@ private void verifyOutputState(POutput output, TransformTreeNode node) { } /** - * Returns the configured pipeline runner. + * Returns the configured {@link PipelineRunner}. */ public PipelineRunner getRunner() { return runner; } /** - * Returns the configured pipeline options. + * Returns the configured {@link PipelineOptions}. */ public PipelineOptions getOptions() { return options; } /** - * Returns the fully qualified name of a transform for testing. - * - * @throws IllegalStateException if the transform has not been applied to the pipeline - * or was applied multiple times. + * @deprecated this method is no longer compatible with the design of {@link Pipeline}, + * as {@link PTransform PTransforms} can be applied multiple times, with different names + * each time. */ @Deprecated public String getFullNameForTesting(PTransform transform) { @@ -475,14 +484,14 @@ private String uniquifyInternal(String namePrefix, String origName) { } /** - * Builds a name from a /-delimited prefix and a name. + * Builds a name from a "/"-delimited prefix and a name. */ private String buildName(String namePrefix, String name) { return namePrefix.isEmpty() ? name : namePrefix + "/" + name; } /** - * Adds the given PValue to this Pipeline. + * Adds the given {@link PValue} to this {@link Pipeline}. * *

      For internal use only. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/PipelineResult.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/PipelineResult.java index fece71b8486d0..6b9a36b728e63 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/PipelineResult.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/PipelineResult.java @@ -21,7 +21,7 @@ import com.google.cloud.dataflow.sdk.transforms.Aggregator; /** - * Result of {@link com.google.cloud.dataflow.sdk.Pipeline#run()}. + * Result of {@link Pipeline#run()}. */ public interface PipelineResult { @@ -35,9 +35,10 @@ public interface PipelineResult { /** * Retrieves the current value of the provided {@link Aggregator}. * - * @param aggregator the Aggregator to retrieve values for - * @return the current values of the aggregator, which may be empty if there are no values yet - * @throws AggregatorRetrievalException if the aggregator values could not be retrieved + * @param aggregator the {@link Aggregator} to retrieve values for. + * @return the current values of the {@link Aggregator}, + * which may be empty if there are no values yet. + * @throws AggregatorRetrievalException if the {@link Aggregator} values could not be retrieved. */ AggregatorValues getAggregatorValues(Aggregator aggregator) throws AggregatorRetrievalException; @@ -46,18 +47,25 @@ AggregatorValues getAggregatorValues(Aggregator aggregator) /** Named constants for common values for the job state. */ public enum State { + /** The job state could not be obtained or was not specified. */ UNKNOWN(false, false), + /** The job has been paused, or has not yet started. */ STOPPED(false, false), + /** The job is currently running. */ RUNNING(false, false), + /** The job has successfully completed. */ DONE(true, false), + /** The job has failed. */ FAILED(true, false), + /** The job has been explicitly cancelled. */ CANCELLED(true, false), + /** The job has been updated. */ UPDATED(true, true); @@ -71,22 +79,17 @@ private State(boolean terminal, boolean hasReplacement) { } /** - * Returns if the job state can no longer complete work. - * - * @return if this JobState represents a terminal state. + * @return {@code true} if the job state can no longer complete work. */ public final boolean isTerminal() { return terminal; } /** - * Returns {@code true} if this job state indicates that a replacement job exists. + * @return {@code true} if this job state indicates that a replacement job exists. */ public final boolean hasReplacementJob() { return hasReplacement; } - } - - } From afd9c2697c53ec581a15a66556b46511a9f2206e Mon Sep 17 00:00:00 2001 From: Kris Hildrum Date: Mon, 21 Dec 2015 14:52:58 -0800 Subject: [PATCH 1235/1541] fix typo --- .../java/com/google/cloud/dataflow/sdk/transforms/ParDo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java index 965d3cd77a26a..6a043505c9e32 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java @@ -149,7 +149,7 @@ * final PCollectionView maxWordLengthCutOffView = * maxWordLengthCutOff.apply(View.asSingleton()); * PCollection wordsBelowCutOff = - * words.apply(ParDo.withSideInput(maxWordLengthCutOffView) + * words.apply(ParDo.withSideInputs(maxWordLengthCutOffView) * .of(new DoFn() { * public void processElement(ProcessContext c) { * String word = c.element(); From 1ed2bffbecd3d640698fe33615fe9cd293cd8a90 Mon Sep 17 00:00:00 2001 From: bchambers Date: Fri, 18 Dec 2015 10:52:19 -0800 Subject: [PATCH 1236/1541] Remove jetty-jmx dependency ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110556252 --- sdk/pom.xml | 6 ------ 1 file changed, 6 deletions(-) diff --git a/sdk/pom.xml b/sdk/pom.xml index 376f84a993034..bea0e986d11aa 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -616,12 +616,6 @@ 9.2.10.v20150310 - - org.eclipse.jetty - jetty-jmx - 9.2.10.v20150310 - - javax.servlet javax.servlet-api From 0d0894a35247af19ec1bc6d8d8b2fcc86087ed8d Mon Sep 17 00:00:00 2001 From: robertwb Date: Fri, 18 Dec 2015 11:24:59 -0800 Subject: [PATCH 1237/1541] Small javadoc fixes for DoFn and friends ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110559193 --- .../cloud/dataflow/sdk/transforms/DoFn.java | 42 +++++++++---------- .../sdk/transforms/DoFnWithContext.java | 6 +-- 2 files changed, 21 insertions(+), 27 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java index 1cb1f9d094f81..f299954b11bd0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java @@ -62,15 +62,18 @@ * mechanism for accessing {@link ProcessContext#window()} without the need * to implement {@link RequiresWindowAccess}. * + *

      See also {@link #processElement} for details on implementing the transformation + * from {@code InputT} to {@code OutputT}. + * * @param the type of the (main) input elements * @param the type of the (main) output elements - * - * @see #processElement for details on implementing the transformation - * from {@code InputT} to {@code OutputT}. */ public abstract class DoFn implements Serializable { - /** Information accessible to all methods in this {@code DoFn}. */ + /** + * Information accessible to all methods in this {@code DoFn}. + * Used primarily to output elements. + */ public abstract class Context { /** @@ -89,11 +92,11 @@ public abstract class Context { * by the Dataflow runtime or later steps in the pipeline, or used in * other unspecified ways. * - *

      If invoked from {@link DoFn#processElement}, the output + *

      If invoked from {@link DoFn#processElement processElement}, the output * element will have the same timestamp and be in the same windows - * as the input element passed to {@link DoFn#processElement}). + * as the input element passed to {@link DoFn#processElement processElement}. * - *

      If invoked from {@link #startBundle} or {@link #finishBundle}, + *

      If invoked from {@link #startBundle startBundle} or {@link #finishBundle finishBundle}, * this will attempt to use the * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * of the input {@code PCollection} to determine what windows the element @@ -110,12 +113,12 @@ public abstract class Context { *

      Once passed to {@code outputWithTimestamp} the element should not be * modified in any way. * - *

      If invoked from {@link DoFn#processElement}), the timestamp + *

      If invoked from {@link DoFn#processElement processElement}, the timestamp * must not be older than the input element's timestamp minus - * {@link DoFn#getAllowedTimestampSkew}. The output element will + * {@link DoFn#getAllowedTimestampSkew getAllowedTimestampSkew}. The output element will * be in the same windows as the input element. * - *

      If invoked from {@link #startBundle} or {@link #finishBundle}, + *

      If invoked from {@link #startBundle startBundle} or {@link #finishBundle finishBundle}, * this will attempt to use the * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * of the input {@code PCollection} to determine what windows the element @@ -132,15 +135,15 @@ public abstract class Context { *

      Once passed to {@code sideOutput} the element should not be modified * in any way. * - *

      The caller of {@code ParDo} uses {@link ParDo#withOutputTags} to + *

      The caller of {@code ParDo} uses {@link ParDo#withOutputTags withOutputTags} to * specify the tags of side outputs that it consumes. Non-consumed side * outputs, e.g., outputs for monitoring purposes only, don't necessarily * need to be specified. * *

      The output element will have the same timestamp and be in the same - * windows as the input element passed to {@link DoFn#processElement}). + * windows as the input element passed to {@link DoFn#processElement processElement}. * - *

      If invoked from {@link #startBundle} or {@link #finishBundle}, + *

      If invoked from {@link #startBundle startBundle} or {@link #finishBundle finishBundle}, * this will attempt to use the * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * of the input {@code PCollection} to determine what windows the element @@ -148,8 +151,6 @@ public abstract class Context { * to access any information about the input element. The output element * will have a timestamp of negative infinity. * - * @throws IllegalArgumentException if the number of outputs exceeds - * the limit of 1,000 outputs per DoFn * @see ParDo#withOutputTags */ public abstract void sideOutput(TupleTag tag, T output); @@ -161,12 +162,12 @@ public abstract class Context { *

      Once passed to {@code sideOutputWithTimestamp} the element should not be * modified in any way. * - *

      If invoked from {@link DoFn#processElement}), the timestamp + *

      If invoked from {@link DoFn#processElement processElement}, the timestamp * must not be older than the input element's timestamp minus - * {@link DoFn#getAllowedTimestampSkew}. The output element will + * {@link DoFn#getAllowedTimestampSkew getAllowedTimestampSkew}. The output element will * be in the same windows as the input element. * - *

      If invoked from {@link #startBundle} or {@link #finishBundle}, + *

      If invoked from {@link #startBundle startBundle} or {@link #finishBundle finishBundle}, * this will attempt to use the * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} * of the input {@code PCollection} to determine what windows the element @@ -174,8 +175,6 @@ public abstract class Context { * to access any information about the input element except for the * timestamp. * - * @throws IllegalArgumentException if the number of outputs exceeds - * the limit of 1,000 outputs per DoFn * @see ParDo#withOutputTags */ public abstract void sideOutputWithTimestamp( @@ -344,8 +343,7 @@ public void startBundle(Context c) throws Exception { public abstract void processElement(ProcessContext c) throws Exception; /** - * Finishes processing this batch of elements. This {@code DoFn} - * instance will be thrown away after this operation returns. + * Finishes processing this batch of elements. * *

      By default, does nothing. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContext.java index 36f2c43b86ffd..f2f5ba7743e27 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContext.java @@ -70,7 +70,7 @@ * PCollection lines = ... ; * PCollection words = * lines.apply(ParDo.of(new DoFnWithContext() { - * {@literal @}ProcessElement + * @ProcessElement * public void processElement(ProcessContext c, BoundedWindow window) { * * }})); @@ -159,8 +159,6 @@ public abstract class Context { * to access any information about the input element. The output element * will have a timestamp of negative infinity. * - * @throws IllegalArgumentException if the number of outputs exceeds - * the limit of 1,000 outputs per DoFn * @see ParDo#withOutputTags */ public abstract void sideOutput(TupleTag tag, T output); @@ -185,8 +183,6 @@ public abstract class Context { * to access any information about the input element except for the * timestamp. * - * @throws IllegalArgumentException if the number of outputs exceeds - * the limit of 1,000 outputs per DoFn * @see ParDo#withOutputTags */ public abstract void sideOutputWithTimestamp( From cfd1c672223a8e13e3f62075b8c5a946fd633202 Mon Sep 17 00:00:00 2001 From: klk Date: Fri, 18 Dec 2015 11:31:58 -0800 Subject: [PATCH 1238/1541] Add Proto2Coder as universal fallback ----Release Notes---- - Any classes that are protocol buffers (v2) Message subclasses will have their coders automatically provided. [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110559835 --- .../dataflow/sdk/coders/CoderRegistry.java | 3 +- .../dataflow/sdk/coders/Proto2Coder.java | 34 ++++++++++++++++++- .../sdk/coders/CoderRegistryTest.java | 8 +++++ .../dataflow/sdk/coders/Proto2CoderTest.java | 26 ++++++++++++-- 4 files changed, 66 insertions(+), 5 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java index 082344c113a9d..82c3a4980e140 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java @@ -80,7 +80,8 @@ public class CoderRegistry implements CoderProvider { private static final Logger LOG = LoggerFactory.getLogger(CoderRegistry.class); public CoderRegistry() { - setFallbackCoderProvider(SerializableCoder.PROVIDER); + setFallbackCoderProvider(CoderProviders.firstOf(Proto2Coder.coderProvider(), + SerializableCoder.PROVIDER)); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java index 9305fbeb4ae2a..a012cc36cff22 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/Proto2Coder.java @@ -17,6 +17,7 @@ import com.google.cloud.dataflow.sdk.util.CloudObject; import com.google.cloud.dataflow.sdk.util.Structs; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -87,11 +88,42 @@ private Proto2Coder(Class protoMessageClass, List> extensionHostClas this.extensionHostClasses = extensionHostClasses; } + private static final CoderProvider PROVIDER = new CoderProvider() { + @Override + public Coder getCoder(TypeDescriptor type) throws CannotProvideCoderException { + if (type.isSubtypeOf(new TypeDescriptor() {})) { + @SuppressWarnings("unchecked") + TypeDescriptor messageType = (TypeDescriptor) type; + @SuppressWarnings("unchecked") + Coder coder = (Coder) Proto2Coder.of(messageType); + return coder; + } else { + throw new CannotProvideCoderException( + String.format("Cannot provide Proto2Coder because %s " + + "is not a subclass of protocol buffer Messsage", + type)); + } + } + }; + + public static CoderProvider coderProvider() { + return PROVIDER; + } + /** * Returns a {@code Proto2Coder} for the given Protobuf message class. */ public static Proto2Coder of(Class protoMessageClass) { - return new Proto2Coder<>(protoMessageClass, Collections.>emptyList()); + return new Proto2Coder(protoMessageClass, Collections.>emptyList()); + } + + /** + * Returns a {@code Proto2Coder} for the given Protobuf message class. + */ + public static Proto2Coder of(TypeDescriptor protoMessageType) { + @SuppressWarnings("unchecked") + Class protoMessageClass = (Class) protoMessageType.getRawType(); + return of(protoMessageClass); } /** diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java index 8ad44117721e3..7fd0d22ea5602 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/CoderRegistryTest.java @@ -81,6 +81,14 @@ public void testSerializableFallbackCoderProvider() throws Exception { assertEquals(serializableCoder, SerializableCoder.of(SerializableClass.class)); } + @Test + public void testProto2CoderFallbackCoderProvider() throws Exception { + CoderRegistry registry = getStandardRegistry(); + Coder coder = + registry.getDefaultCoder(Proto2CoderTestMessages.MessageA.class); + assertEquals(coder, Proto2Coder.of(new TypeDescriptor() {})); + } + @Test public void testAvroFallbackCoderProvider() throws Exception { CoderRegistry registry = getStandardRegistry(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/Proto2CoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/Proto2CoderTest.java index f845e598aa48c..f4c355715a470 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/Proto2CoderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/Proto2CoderTest.java @@ -16,11 +16,14 @@ package com.google.cloud.dataflow.sdk.coders; +import static org.junit.Assert.assertEquals; + import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA; import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageB; import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageC; import com.google.cloud.dataflow.sdk.testing.CoderProperties; import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; import com.google.common.collect.ImmutableList; import org.junit.Rule; @@ -35,6 +38,26 @@ @RunWith(JUnit4.class) public class Proto2CoderTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testFactoryMethodAgreement() throws Exception { + assertEquals( + Proto2Coder.of(new TypeDescriptor() {}), + Proto2Coder.of(MessageA.class)); + + assertEquals( + Proto2Coder.of(new TypeDescriptor() {}), + Proto2Coder.coderProvider().getCoder(new TypeDescriptor() {})); + } + + @Test + public void testProviderCannotProvideCoder() throws Exception { + thrown.expect(CannotProvideCoderException.class); + Proto2Coder.coderProvider().getCoder(new TypeDescriptor() {}); + } + @Test public void testCoderEncodeDecodeEqual() throws Exception { MessageA value = MessageA.newBuilder() @@ -111,9 +134,6 @@ public void testEncodingId() throws Exception { CoderProperties.coderHasEncodingId(coder, MessageC.class.getName()); } - @Rule - public ExpectedException thrown = ExpectedException.none(); - @Test public void encodeNullThrowsCoderException() throws Exception { thrown.expect(CoderException.class); From af0367ca583a4ec85740218299a21774a0afa3ee Mon Sep 17 00:00:00 2001 From: bchambers Date: Fri, 18 Dec 2015 11:37:55 -0800 Subject: [PATCH 1239/1541] Add dependency on jetty-servlet ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110560293 --- sdk/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sdk/pom.xml b/sdk/pom.xml index bea0e986d11aa..5f6d714a8e159 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -616,6 +616,12 @@ 9.2.10.v20150310 + + org.eclipse.jetty + jetty-servlet + 9.2.10.v20150310 + + javax.servlet javax.servlet-api From 4057defd085e8f4f9110e44a86dfb297ab3704a2 Mon Sep 17 00:00:00 2001 From: dhalperi Date: Fri, 18 Dec 2015 17:29:38 -0800 Subject: [PATCH 1240/1541] CountingSource: bounded/unbounded source of longs These sources are useful for testing and for data generation. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110585645 --- .../cloud/dataflow/sdk/io/CountingSource.java | 386 ++++++++++++++++++ .../dataflow/sdk/io/CountingSourceTest.java | 216 ++++++++++ 2 files changed, 602 insertions(+) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CountingSource.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/io/CountingSourceTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CountingSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CountingSource.java new file mode 100644 index 0000000000000..2938534168abd --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CountingSource.java @@ -0,0 +1,386 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.DefaultCoder; +import com.google.cloud.dataflow.sdk.coders.VarLongCoder; +import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.collect.ImmutableList; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * A source that produces longs. When used as a {@link BoundedSource}, {@link CountingSource} + * starts at {@code 0} and counts up to a specified maximum. When used as an + * {@link UnboundedSource}, it counts up to {@link Long#MAX_VALUE} and then never produces more + * output. (In practice, this limit should never be reached.) + * + *

      The bounded {@link CountingSource} is implemented based on {@link OffsetBasedSource} and + * {@link OffsetBasedSource.OffsetBasedReader}, so it performs efficient initial splitting and it + * supports dynamic work rebalancing. + * + *

      To produce a bounded {@code PCollection}, use {@link CountingSource#upTo(long)}: + * + *

      {@code
      + * Pipeline p = ...
      + * BoundedSource source = CountingSource.upTo(1000);
      + * PCollection bounded = p.apply(Read.from(source));
      + * }
      + * + *

      To produce an unbounded {@code PCollection}, use {@link CountingSource#unbounded} or + * {@link CountingSource#unboundedWithTimestampFn}: + * + *

      {@code
      + * Pipeline p = ...
      + *
      + * // To create an unbounded source that uses processing time as the element timestamp.
      + * UnboundedSource source = CountingSource.unbounded();
      + * // Or, to create an unbounded source that uses a provided function to set the element timestamp.
      + * UnboundedSource source = CountingSource.unboundedWithTimestampFn(someFn);
      + *
      + * PCollection unbounded = p.apply(Read.from(source));
      + * }
      + */ +public class CountingSource { + /** + * Creates a {@link BoundedSource} that will produce the specified number of elements, + * from {@code 0} to {@code numElements - 1}. + */ + public static BoundedSource upTo(long numElements) { + checkArgument(numElements > 0, "numElements (%s) must be greater than 0", numElements); + return new BoundedCountingSource(0, numElements); + } + + /** + * Creates an {@link UnboundedSource} that will produce numbers starting from {@code 0} up to + * {@link Long#MAX_VALUE}. + * + *

      After {@link Long#MAX_VALUE}, the source never produces more output. (In practice, this + * limit should never be reached.) + * + *

      Elements in the resulting {@link PCollection PCollection<Long>} will have timestamps + * corresponding to processing time at element generation, provided by {@link Instant#now}. + */ + public static UnboundedSource unbounded() { + return unboundedWithTimestampFn(new NowTimestampFn()); + } + + /** + * Creates an {@link UnboundedSource} that will produce numbers starting from {@code 0} up to + * {@link Long#MAX_VALUE}, with element timestamps supplied by the specified function. + * + *

      After {@link Long#MAX_VALUE}, the source never produces more output. (In practice, this + * limit should never be reached.) + * + *

      Note that the timestamps produced by {@code timestampFn} may not decrease. + */ + public static UnboundedSource unboundedWithTimestampFn( + SerializableFunction timestampFn) { + return new UnboundedCountingSource(0, 1, timestampFn); + } + + ///////////////////////////////////////////////////////////////////////////////////////////// + + /** Prevent instantiation. */ + private CountingSource() {} + + + /** + * A function that returns {@link Instant#now} as the timestamp for each generated element. + */ + private static class NowTimestampFn implements SerializableFunction { + @Override + public Instant apply(Long input) { + return Instant.now(); + } + } + + /** + * An implementation of {@link CountingSource} that produces a bounded {@link PCollection}. + * It is implemented on top of {@link OffsetBasedSource} (with associated reader + * {@link BoundedCountingReader}) and performs efficient initial splitting and supports dynamic + * work rebalancing. + */ + private static class BoundedCountingSource extends OffsetBasedSource { + /** + * Creates a {@link BoundedCountingSource} that generates the numbers in the specified + * {@code [start, end)} range. + */ + public BoundedCountingSource(long start, long end) { + super(start, end, 1 /* can be split every 1 offset */); + } + + //////////////////////////////////////////////////////////////////////////////////////////// + + @Override + public long getBytesPerOffset() { + return 8; + } + + @Override + public long getMaxEndOffset(PipelineOptions options) throws Exception { + return getEndOffset(); + } + + @Override + public OffsetBasedSource createSourceForSubrange(long start, long end) { + return new BoundedCountingSource(start, end); + } + + @Override + public boolean producesSortedKeys(PipelineOptions options) throws Exception { + return true; + } + + @Override + public com.google.cloud.dataflow.sdk.io.BoundedSource.BoundedReader createReader( + PipelineOptions options) throws IOException { + return new BoundedCountingReader(this); + } + + @Override + public Coder getDefaultOutputCoder() { + return VarLongCoder.of(); + } + } + + /** + * The reader associated with {@link BoundedCountingSource}. + * + * @see BoundedCountingSource + */ + private static class BoundedCountingReader extends OffsetBasedSource.OffsetBasedReader { + private long current; + + public BoundedCountingReader(OffsetBasedSource source) { + super(source); + } + + @Override + protected long getCurrentOffset() throws NoSuchElementException { + return current; + } + + @Override + public synchronized BoundedCountingSource getCurrentSource() { + return (BoundedCountingSource) super.getCurrentSource(); + } + + @Override + public Long getCurrent() throws NoSuchElementException { + return current; + } + + @Override + protected boolean startImpl() throws IOException { + current = getCurrentSource().getStartOffset(); + return true; + } + + @Override + protected boolean advanceImpl() throws IOException { + current++; + return true; + } + + @Override + public void close() throws IOException {} + } + + /** + * An implementation of {@link CountingSource} that produces an unbounded {@link PCollection}. + */ + private static class UnboundedCountingSource extends UnboundedSource { + /** The first number (>= 0) generated by this {@link UnboundedCountingSource}. */ + private final long start; + /** The interval between numbers generated by this {@link UnboundedCountingSource}. */ + private final long stride; + /** The function used to produce timestamps for the generated elements. */ + private final SerializableFunction timestampFn; + + /** + * Creates an {@link UnboundedSource} that will produce numbers starting from {@code 0} up to + * {@link Long#MAX_VALUE}, with element timestamps supplied by the specified function. + * + *

      After {@link Long#MAX_VALUE}, the source never produces more output. (In practice, this + * limit should never be reached.) + * + *

      Note that the timestamps produced by {@code timestampFn} may not decrease. + */ + public UnboundedCountingSource( + long start, long stride, SerializableFunction timestampFn) { + this.start = start; + this.stride = stride; + this.timestampFn = timestampFn; + } + + /** + * Splits an unbounded source {@code desiredNumSplits} ways by giving each split every + * {@code desiredNumSplits}th element that this {@link UnboundedCountingSource} + * produces. + * + *

      E.g., if a source produces all even numbers {@code [0, 2, 4, 6, 8, ...)} and we want to + * split into 3 new sources, then the new sources will produce numbers that are 6 apart and + * are offset at the start by the original stride: {@code [0, 6, 12, ...)}, + * {@code [2, 8, 14, ...)}, and {@code [4, 10, 16, ...)}. + */ + @Override + public List> generateInitialSplits( + int desiredNumSplits, PipelineOptions options) throws Exception { + // Using Javadoc example, stride 2 with 3 splits becomes stride 6. + long newStride = stride * desiredNumSplits; + + ImmutableList.Builder splits = ImmutableList.builder(); + for (int i = 0; i < desiredNumSplits; ++i) { + // Starts offset by the original stride. Using Javadoc example, this generates starts of + // 0, 2, and 4. + splits.add(new UnboundedCountingSource(start + i * stride, newStride, timestampFn)); + } + return splits.build(); + } + + @Override + public UnboundedReader createReader( + PipelineOptions options, CounterMark checkpointMark) { + return new UnboundedCountingReader(this, checkpointMark); + } + + @Override + public Coder getCheckpointMarkCoder() { + return AvroCoder.of(CountingSource.CounterMark.class); + } + + @Override + public void validate() {} + + @Override + public Coder getDefaultOutputCoder() { + return VarLongCoder.of(); + } + } + + /** + * The reader associated with {@link UnboundedCountingSource}. + * + * @see UnboundedCountingSource + */ + private static class UnboundedCountingReader extends UnboundedReader { + private UnboundedCountingSource source; + private long current; + private Instant currentTimestamp; + + public UnboundedCountingReader(UnboundedCountingSource source, CounterMark mark) { + this.source = source; + if (mark == null) { + // Because we have not emitted an element yet, and start() calls advance, we need to + // "un-advance" so that start() produces the correct output. + this.current = source.start - source.stride; + } else { + this.current = mark.getLastEmitted(); + } + } + + @Override + public boolean start() throws IOException { + return advance(); + } + + @Override + public boolean advance() throws IOException { + // Overflow-safe check that (current + source.stride) <= LONG.MAX_VALUE. Else, stop producing. + if (Long.MAX_VALUE - source.stride < current) { + return false; + } + current += source.stride; + currentTimestamp = source.timestampFn.apply(current); + return true; + } + + @Override + public Instant getWatermark() { + return source.timestampFn.apply(current); + } + + @Override + public CounterMark getCheckpointMark() { + return new CounterMark(current); + } + + @Override + public UnboundedSource getCurrentSource() { + return source; + } + + @Override + public Long getCurrent() throws NoSuchElementException { + return current; + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return currentTimestamp; + } + + @Override + public void close() throws IOException {} + } + + /** + * The checkpoint for an unbounded {@link CountingSource} is simply the last value produced. The + * associated source object encapsulates the information needed to produce the next value. + */ + @DefaultCoder(AvroCoder.class) + public static class CounterMark implements UnboundedSource.CheckpointMark { + /** The last value emitted. */ + private final long lastEmitted; + + /** + * Creates a checkpoint mark reflecting the last emitted value. + */ + public CounterMark(long lastEmitted) { + this.lastEmitted = lastEmitted; + } + + /** + * Returns the last value emitted by the reader. + */ + public long getLastEmitted() { + return lastEmitted; + } + + ///////////////////////////////////////////////////////////////////////////////////// + + @SuppressWarnings("unused") // For AvroCoder + private CounterMark() { + this.lastEmitted = 0L; + } + + @Override + public void finalizeCheckpoint() throws IOException {} + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/CountingSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/CountingSourceTest.java new file mode 100644 index 0000000000000..178287660066c --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/CountingSourceTest.java @@ -0,0 +1,216 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.CountingSource.CounterMark; +import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.RunnableOnService; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Flatten; +import com.google.cloud.dataflow.sdk.transforms.Max; +import com.google.cloud.dataflow.sdk.transforms.Min; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; + +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.List; + +/** + * Tests of {@link CountingSource}. + */ +@RunWith(JUnit4.class) +public class CountingSourceTest { + + public static void addCountingAsserts(PCollection input, long numElements) { + // Count == numElements + DataflowAssert + .thatSingleton(input.apply("Count", Count.globally())) + .isEqualTo(numElements); + // Unique count == numElements + DataflowAssert + .thatSingleton(input.apply(RemoveDuplicates.create()) + .apply("UniqueCount", Count.globally())) + .isEqualTo(numElements); + // Min == 0 + DataflowAssert + .thatSingleton(input.apply("Min", Min.globally())) + .isEqualTo(0L); + // Max == numElements-1 + DataflowAssert + .thatSingleton(input.apply("Max", Max.globally())) + .isEqualTo(numElements - 1); + } + + @Test + @Category(RunnableOnService.class) + public void testBoundedSource() { + Pipeline p = TestPipeline.create(); + long numElements = 1000; + PCollection input = p.apply(Read.from(CountingSource.upTo(numElements))); + + addCountingAsserts(input, numElements); + p.run(); + } + + @Test + @Category(RunnableOnService.class) + public void testBoundedSourceSplits() throws Exception { + Pipeline p = TestPipeline.create(); + long numElements = 1000; + long numSplits = 10; + long splitSizeBytes = numElements * 8 / numSplits; // 8 bytes per long element. + + BoundedSource initial = CountingSource.upTo(numElements); + List> splits = + initial.splitIntoBundles(splitSizeBytes, p.getOptions()); + assertEquals("Expected exact splitting", numSplits, splits.size()); + + // Assemble all the splits into one flattened PCollection, also verify their sizes. + PCollectionList pcollections = PCollectionList.empty(p); + for (int i = 0; i < splits.size(); ++i) { + BoundedSource split = splits.get(i); + pcollections = pcollections.and(p.apply("split" + i, Read.from(split))); + assertEquals("Expected even splitting", + splitSizeBytes, split.getEstimatedSizeBytes(p.getOptions())); + } + PCollection input = pcollections.apply(Flatten.pCollections()); + + addCountingAsserts(input, numElements); + p.run(); + } + + @Test + @Category(RunnableOnService.class) + public void testUnboundedSource() { + Pipeline p = TestPipeline.create(); + long numElements = 1000; + + PCollection input = p + .apply(Read.from(CountingSource.unbounded()).withMaxNumRecords(numElements)); + + addCountingAsserts(input, numElements); + p.run(); + } + + private static class ElementValueDiff extends DoFn { + @Override + public void processElement(ProcessContext c) throws Exception { + c.output(c.element() - c.timestamp().getMillis()); + } + } + + @Test + @Category(RunnableOnService.class) + public void testUnboundedSourceTimestamps() { + Pipeline p = TestPipeline.create(); + long numElements = 1000; + + PCollection input = p.apply( + Read.from(CountingSource.unboundedWithTimestampFn(new ValueAsTimestampFn())) + .withMaxNumRecords(numElements)); + addCountingAsserts(input, numElements); + + PCollection diffs = input + .apply("TimestampDiff", ParDo.of(new ElementValueDiff())) + .apply("RemoveDuplicateTimestamps", RemoveDuplicates.create()); + // This assert also confirms that diffs only has one unique value. + DataflowAssert.thatSingleton(diffs).equals(0L); + + p.run(); + } + + @Test + @Category(RunnableOnService.class) + public void testUnboundedSourceSplits() throws Exception { + Pipeline p = TestPipeline.create(); + long numElements = 1000; + int numSplits = 10; + + UnboundedSource initial = CountingSource.unbounded(); + List> splits = + initial.generateInitialSplits(numSplits, p.getOptions()); + assertEquals("Expected exact splitting", numSplits, splits.size()); + + long elementsPerSplit = numElements / numSplits; + assertEquals("Expected even splits", numElements, elementsPerSplit * numSplits); + PCollectionList pcollections = PCollectionList.empty(p); + for (int i = 0; i < splits.size(); ++i) { + pcollections = pcollections.and( + p.apply("split" + i, Read.from(splits.get(i)).withMaxNumRecords(elementsPerSplit))); + } + PCollection input = pcollections.apply(Flatten.pCollections()); + + addCountingAsserts(input, numElements); + p.run(); + } + + /** + * A timestamp function that uses the given value as the timestamp. Because the input values will + * not wrap, this function is non-decreasing and meets the timestamp function criteria laid out + * in {@link CountingSource#unboundedWithTimestampFn(SerializableFunction)}. + */ + private static class ValueAsTimestampFn implements SerializableFunction { + @Override + public Instant apply(Long input) { + return new Instant(input); + } + } + + @Test + public void testUnboundedSourceCheckpointMark() throws Exception { + UnboundedSource source = + CountingSource.unboundedWithTimestampFn(new ValueAsTimestampFn()); + UnboundedReader reader = source.createReader(null, null); + final long numToSkip = 3; + assertTrue(reader.start()); + + // Advance the source numToSkip elements and manually save state. + for (long l = 0; l < numToSkip; ++l) { + reader.advance(); + } + + // Confirm that we get the expected element in sequence before checkpointing. + assertEquals(numToSkip, (long) reader.getCurrent()); + assertEquals(numToSkip, reader.getCurrentTimestamp().getMillis()); + + // Checkpoint and restart, and confirm that the source continues correctly. + CounterMark mark = CoderUtils.clone( + source.getCheckpointMarkCoder(), (CounterMark) reader.getCheckpointMark()); + reader = source.createReader(null, mark); + assertTrue(reader.start()); + + // Confirm that we get the next element in sequence. + assertEquals(numToSkip + 1, (long) reader.getCurrent()); + assertEquals(numToSkip + 1, reader.getCurrentTimestamp().getMillis()); + } +} From 325471571f0b3cee74b3a80b6ec583b34f752f00 Mon Sep 17 00:00:00 2001 From: lcwik Date: Fri, 18 Dec 2015 20:43:31 -0800 Subject: [PATCH 1241/1541] Have Dataflow internally depend on Guava 19.0 ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110591070 --- pom.xml | 2 +- sdk/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 734eaf8775291..bd43b277cea8f 100644 --- a/pom.xml +++ b/pom.xml @@ -72,7 +72,7 @@ v1b3-rev13-1.21.0 v1beta2-rev1-3.0.2 1.21.0 - 18.0 + 19.0 1.3 2.4.2 2.4 diff --git a/sdk/pom.xml b/sdk/pom.xml index 5f6d714a8e159..002b2d8e4f727 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -245,7 +245,7 @@ ${basedir}/../javadoc/datastore-docs - http://docs.guava-libraries.googlecode.com/git-history/release18/javadoc/ + http://docs.guava-libraries.googlecode.com/git-history/release19/javadoc/ ${basedir}/../javadoc/guava-docs From e20747f09654e7a97b67034dbc015551e178e648 Mon Sep 17 00:00:00 2001 From: lcwik Date: Sat, 19 Dec 2015 14:20:15 -0800 Subject: [PATCH 1242/1541] Add a scalable bloom filter implementation ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110616150 --- .../sdk/util/ScalableBloomFilter.java | 331 ++++++++++++++++++ .../sdk/util/ScalableBloomFilterTest.java | 169 +++++++++ 2 files changed, 500 insertions(+) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ScalableBloomFilter.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ScalableBloomFilterTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ScalableBloomFilter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ScalableBloomFilter.java new file mode 100644 index 0000000000000..b509dcfa28b94 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ScalableBloomFilter.java @@ -0,0 +1,331 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; +import com.google.common.hash.BloomFilter; +import com.google.common.hash.Funnel; +import com.google.common.hash.PrimitiveSink; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * A Bloom filter implementation with an expected false positive probability of {@code 0.000001} + * which grows dynamically with the number of insertions. For less than {@code 2^20} insertions + * which would modify a Bloom filter, we brute force all the Bloom filter combinations in powers of + * {@code 2} to only produce a scalable Bloom filter with one slice. + * + *

      Otherwise, we use an implementation of + * Scalable Bloom Filters + * by Paulo Sergio Almeida, Carlos Baquero, Nuno Preguica, David Hutchison. Our implementation + * has an effective positive probability of {@code 0.000001}, given that we use a ratio of + * {@code 0.9} and a scaling factor of {@code 2}. + */ +public class ScalableBloomFilter implements Serializable { + /** + * A {@link Coder} for scalable Bloom filters. The encoded format is: + *

        + *
      • var int encoding of number of Bloom filter slices + *
      • N Bloom filter slices + *
      + * + *

      The encoded form of each Bloom filter slice is: + *

        + *
      • 1 signed byte for the strategy + *
      • 1 unsigned byte for the number of hash functions + *
      • 1 big endian int, the number of longs in our bitset + *
      • N big endian longs of our bitset + *
      + */ + public static class ScalableBloomFilterCoder extends AtomicCoder { + private static final ScalableBloomFilterCoder INSTANCE = new ScalableBloomFilterCoder(); + + @JsonCreator + public static ScalableBloomFilterCoder of() { + return INSTANCE; + } + + @Override + public void encode(ScalableBloomFilter value, OutputStream outStream, Coder.Context context) + throws CoderException, IOException { + VarInt.encode(value.bloomFilterSlices.size(), outStream); + for (BloomFilter bloomFilter : value.bloomFilterSlices) { + bloomFilter.writeTo(outStream); + } + } + + @Override + public ScalableBloomFilter decode(InputStream inStream, Coder.Context context) + throws CoderException, IOException { + int numberOfBloomFilters = VarInt.decodeInt(inStream); + List> bloomFilters = new ArrayList<>(numberOfBloomFilters); + for (int i = 0; i < numberOfBloomFilters; ++i) { + bloomFilters.add(BloomFilter.readFrom(inStream, ByteBufferFunnel.INSTANCE)); + } + return new ScalableBloomFilter(bloomFilters); + } + + @Override + public boolean consistentWithEquals() { + return true; + } + } + + private final List> bloomFilterSlices; + private ScalableBloomFilter(List> bloomFilters) { + this.bloomFilterSlices = bloomFilters; + } + + /** + * Returns false if the Bloom filter definitely does not contain the byte + * representation of an element contained in {@code buf} from {@code [offset, offset + length)}. + */ + public boolean mightContain(byte[] buf, int offset, int length) { + ByteBuffer byteBuffer = ByteBuffer.wrap(buf, offset, length); + return mightContain(byteBuffer); + } + + /** + * Returns false if the Bloom filter definitely does not contain the byte + * representation of an element contained in {@code byteBuffer} from {@code [position, limit)}. + */ + public boolean mightContain(ByteBuffer byteBuffer) { + for (int i = bloomFilterSlices.size() - 1; i >= 0; i--) { + if (bloomFilterSlices.get(i).mightContain(byteBuffer)) { + return true; + } + } + return false; + } + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + if (!(other instanceof ScalableBloomFilter)) { + return false; + } + ScalableBloomFilter scalableBloomFilter = (ScalableBloomFilter) other; + if (bloomFilterSlices.size() != scalableBloomFilter.bloomFilterSlices.size()) { + return false; + } + for (int i = 0; i < bloomFilterSlices.size(); ++i) { + if (!bloomFilterSlices.get(i).equals(scalableBloomFilter.bloomFilterSlices.get(i))) { + return false; + } + } + return true; + } + + @Override + public int hashCode() { + return bloomFilterSlices.hashCode(); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(ScalableBloomFilter.class) + .add("bloomFilterSlices", bloomFilterSlices) + .toString(); + } + + @VisibleForTesting + int numberOfBloomFilterSlices() { + return bloomFilterSlices.size(); + } + + /** + * Returns a scalable Bloom filter builder allowing one to construct a Bloom filter + * with an expected false positive probability of {@code 0.000001} irrespective + * of the number of elements inserted. + */ + public static Builder builder() { + return builder(Builder.MAX_INSERTIONS_FOR_ADD_TO_ALL_MODE_LOG_2); + } + + @VisibleForTesting + static Builder builder(int maxInsertionsForAddToAllModeLog2) { + return new Builder(maxInsertionsForAddToAllModeLog2); + } + + /** + * A scalable Bloom filter builder which during the build process will attempt to + * create a Bloom filter no larger than twice the required size for small Bloom filters. + * For large Bloom filters, we create a list of Bloom filters which are successively twice as + * large as the previous which we insert elements into. + * + *

      This scalable Bloom filter builder uses 8mb of memory per instance to start when + * fewer than {@code 2^20} elements have been inserted. Afterwards, it increases in space usage + * by a factor of {@code 2.2} for every doubling in the number of unique insertions. + */ + public static class Builder { + private static final long MAX_ELEMENTS = 1L << 62; + private static final int MAX_INSERTIONS_FOR_ADD_TO_ALL_MODE_LOG_2 = 20; + private static final double DEFAULT_FALSE_POSITIVE_PROBABILITY = 0.000001; + private static final double RATIO = 0.9; + + private enum Mode { + ADD_TO_ALL, ADD_TO_LAST + } + + private final List> bloomFilters; + private Mode mode; + private long numberOfInsertions; + + private Builder(int maxInsertionsForAddToAllModeLog2) { + checkArgument(maxInsertionsForAddToAllModeLog2 < Long.SIZE - 1, + "%s does not support an initial size with more than 2^63 elements.", + ScalableBloomFilter.class.getSimpleName()); + this.bloomFilters = new ArrayList<>(); + this.mode = Mode.ADD_TO_ALL; + // 1, 2, 4, 8, 16, 32, ... + for (int i = 0; i <= maxInsertionsForAddToAllModeLog2; ++i) { + bloomFilters.add(BloomFilter.create( + ByteBufferFunnel.INSTANCE, + 1 << i, + DEFAULT_FALSE_POSITIVE_PROBABILITY)); + } + } + + /** + * Returns true if the Bloom filter was modified by inserting the byte + * representation of an element contained in {@code buf} from {@code [offset, offset + length)}. + */ + public boolean put(final byte[] buf, final int off, final int len) { + ByteBuffer buffer = ByteBuffer.wrap(buf, off, len); + return put(buffer); + } + + /** + * Returns true if the Bloom filter was modified by inserting the byte + * representation of an element contained in {@code byteBuffer} from {@code [position, limit)}. + */ + public boolean put(final ByteBuffer byteBuffer) { + // Check to see if we gain any information by adding this element. + switch (mode) { + case ADD_TO_ALL: + if (bloomFilters.get(bloomFilters.size() - 1).mightContain(byteBuffer)) { + // We do not gain any information by adding this element + return false; + } + break; + case ADD_TO_LAST: + for (int i = bloomFilters.size() - 1; i >= 0; i--) { + if (bloomFilters.get(i).mightContain(byteBuffer)) { + // One of the Bloom filters already considers that this element exists so skip + // adding it. + return false; + } + } + break; + default: + throw new IllegalStateException("Unknown builder mode: " + mode); + } + + // We now need to add the element to the appropriate Bloom filter(s) depending on the mode. + switch (mode) { + case ADD_TO_ALL: + int bloomFilterToStartWith = + Long.SIZE - Long.numberOfLeadingZeros(numberOfInsertions); + // If we were to attempt to add to a non-existent Bloom filter, we need to + // swap to the other mode. + if (bloomFilterToStartWith == bloomFilters.size()) { + BloomFilter last = bloomFilters.get(bloomFilters.size() - 1); + bloomFilters.clear(); + bloomFilters.add(last); + mode = Mode.ADD_TO_LAST; + addToLast(byteBuffer); + } else { + for (int i = bloomFilterToStartWith; i < bloomFilters.size(); ++i) { + bloomFilters.get(i).put(byteBuffer); + } + } + break; + case ADD_TO_LAST: + addToLast(byteBuffer); + break; + default: + throw new IllegalStateException("Unknown builder mode: " + mode); + } + numberOfInsertions += 1; + return true; + } + + /** + * Returns a scalable Bloom filter with the elements that were added. + */ + public ScalableBloomFilter build() { + switch (mode) { + case ADD_TO_ALL: + int bloomFilterToUse = Long.SIZE - Long.numberOfLeadingZeros(numberOfInsertions); + if (Long.bitCount(numberOfInsertions) == 1) { + bloomFilterToUse -= 1; + } + return new ScalableBloomFilter(Arrays.asList(bloomFilters.get(bloomFilterToUse))); + case ADD_TO_LAST: + return new ScalableBloomFilter(bloomFilters); + default: + throw new IllegalStateException("Unknown builder mode: " + mode); + } + } + + private void addToLast(ByteBuffer byteBuffer) { + // If we are a power of 2, we have hit the number of expected insertions + // for the last Bloom filter and we have to add a new one. + if (Long.bitCount(numberOfInsertions) == 1) { + checkArgument(numberOfInsertions <= MAX_ELEMENTS, + "%s does not support Bloom filter slices with more than 2^63 elements.", + ScalableBloomFilter.class); + bloomFilters.add(BloomFilter.create( + ByteBufferFunnel.INSTANCE, + numberOfInsertions, + DEFAULT_FALSE_POSITIVE_PROBABILITY * Math.pow(RATIO, bloomFilters.size()))); + } + BloomFilter last = bloomFilters.get(bloomFilters.size() - 1); + last.put(byteBuffer); + } + } + + /** + * Writes {@link ByteBuffer}s to {@link PrimitiveSink}s and meant to be used + * with Guava's {@link BloomFilter} API. This {@link Funnel} does not modify the + * underlying byte buffer and assumes that {@code ByteBuffer#array} returns the backing data. + */ + private static class ByteBufferFunnel implements Funnel { + private static final ByteBufferFunnel INSTANCE = new ByteBufferFunnel(); + @Override + public void funnel(ByteBuffer from, PrimitiveSink into) { + into.putBytes(from.array(), from.position(), from.remaining()); + } + } +} + diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ScalableBloomFilterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ScalableBloomFilterTest.java new file mode 100644 index 0000000000000..ad72e2914726c --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ScalableBloomFilterTest.java @@ -0,0 +1,169 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.dataflow.sdk.testing.CoderProperties; +import com.google.cloud.dataflow.sdk.util.ScalableBloomFilter.Builder; +import com.google.cloud.dataflow.sdk.util.ScalableBloomFilter.ScalableBloomFilterCoder; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.nio.ByteBuffer; + +/** + * Tests for {@link ScalableBloomFilter}. + */ +@RunWith(JUnit4.class) +public class ScalableBloomFilterTest { + private static final ByteBuffer BUFFER = ByteBuffer.wrap(new byte[]{ 0x01, 0x02 }); + + @Test + public void testBuilderModeAddAll() throws Exception { + Builder builder = ScalableBloomFilter.builder(); + assertTrue("Expected Bloom filter to have been modified.", builder.put(BUFFER)); + + // Re-adding should skip and not record the insertion. + assertFalse("Expected Bloom filter to not have been modified.", builder.put(BUFFER)); + + // Verify insertion + int maxValue = insertAndVerifyContents(builder, 31); + + // Verify that we only have one bloom filter instead of many since the number of insertions + // is small. + ScalableBloomFilter bloomFilter = builder.build(); + assertEquals(1, bloomFilter.numberOfBloomFilterSlices()); + verifyCoder(builder.build(), maxValue); + } + + @Test + public void testBuilderModeAddAllModeAtThreshold() throws Exception { + // Use a builder where the insertion threshold to swap to add to last mode is 2^4 elements. + Builder builder = ScalableBloomFilter.builder(4); + + // Verify insertion + int maxValue = insertAndVerifyContents(builder, 16); + + ScalableBloomFilter bloomFilter = builder.build(); + // Verify at the threshold we have only built a single Bloom filter slice. + assertEquals(1, bloomFilter.numberOfBloomFilterSlices()); + + verifyCoder(bloomFilter, maxValue); + } + + @Test + public void testBuilderModeAddAllModeAtThresholdPlusOne() throws Exception { + // Use a builder where the insertion threshold to swap to add to last mode is 2^4 elements. + Builder builder = ScalableBloomFilter.builder(4); + + // Verify insertion + int maxValue = insertAndVerifyContents(builder, 17); + + ScalableBloomFilter bloomFilter = builder.build(); + // Verify that at one over the threshold, we created two Bloom filter slices. + assertEquals(2, bloomFilter.numberOfBloomFilterSlices()); + + verifyCoder(bloomFilter, maxValue); + } + + @Test + public void testBuilderModeAddLastMode() throws Exception { + // Use a builder where the insertion threshold to swap to add to last mode is 2^4 elements. + Builder builder = ScalableBloomFilter.builder(4); + + // Verify insertion + int maxValue = insertAndVerifyContents(builder, (int) Math.pow(2, 12) - 16); + + ScalableBloomFilter bloomFilter = builder.build(); + // Verify that we swapped to the scalable mode. + // This is 9 because we inserted 16 elements swapping us into the add to all mode. + // Then at every power of 2 (e.g. 32, 64, 128, ..) we add another filter. + // Thus we have a filter for every power of 2 from 2^4 to 2^12 giving us 9 filters. + assertEquals(9, bloomFilter.numberOfBloomFilterSlices()); + + verifyCoder(bloomFilter, maxValue); + } + + @Test + public void testScalableBloomFilterCoder() throws Exception { + Builder builderA = ScalableBloomFilter.builder(); + builderA.put(BUFFER); + ScalableBloomFilter filterA = builderA.build(); + Builder builderB = ScalableBloomFilter.builder(); + builderB.put(BUFFER); + ScalableBloomFilter filterB = builderB.build(); + + CoderProperties.coderDecodeEncodeEqual(ScalableBloomFilterCoder.of(), filterA); + CoderProperties.coderDeterministic(ScalableBloomFilterCoder.of(), filterA, filterB); + CoderProperties.coderConsistentWithEquals(ScalableBloomFilterCoder.of(), filterA, filterB); + CoderProperties.coderSerializable(ScalableBloomFilterCoder.of()); + CoderProperties.structuralValueConsistentWithEquals( + ScalableBloomFilterCoder.of(), filterA, filterB); + } + + /** + * Inserts elements {@code 0, 1, ...} until the internal bloom filters have + * been modified {@code maxNumberOfInsertions} times. Returns the largest value inserted. + */ + private int insertAndVerifyContents(Builder builder, int maxNumberOfInsertions) { + ByteBuffer byteBuffer = ByteBuffer.allocate(4); + int value = -1; + while (maxNumberOfInsertions > 0) { + value += 1; + byteBuffer.clear(); + byteBuffer.putInt(value); + byteBuffer.rewind(); + if (builder.put(byteBuffer)) { + maxNumberOfInsertions -= 1; + } + } + + verifyContents(builder.build(), value); + return value; + } + + /** + * Verifies that the bloom filter contains all the values from {@code [0, 1, ..., maxValue]}. + */ + private void verifyContents(ScalableBloomFilter bloomFilter, int maxValue) { + ByteBuffer byteBuffer = ByteBuffer.allocate(4); + // Verify that all the values exist + for (int i = 0; i <= maxValue; ++i) { + byteBuffer.clear(); + byteBuffer.putInt(i); + byteBuffer.rewind(); + assertTrue(bloomFilter.mightContain(byteBuffer)); + } + } + + /** + * Verifies that the coder correctly encodes and decodes and that all the values + * {@code [0, 1, 2, ..., maxValue]} are contained within the decoded bloom filter. + */ + private void verifyCoder(ScalableBloomFilter bloomFilter, int maxValue) throws Exception { + byte[] encodedValue = + CoderUtils.encodeToByteArray(ScalableBloomFilterCoder.of(), bloomFilter); + ScalableBloomFilter decoded = + CoderUtils.decodeFromByteArray(ScalableBloomFilterCoder.of(), encodedValue); + verifyContents(decoded, maxValue); + } +} + From f15d459d0757d63c2e3274a45d92e1bb8ffdef91 Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 21 Dec 2015 09:29:50 -0800 Subject: [PATCH 1243/1541] Touch up ParDo javadoc ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110694633 --- .../cloud/dataflow/sdk/transforms/ParDo.java | 484 +++++++++--------- 1 file changed, 249 insertions(+), 235 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java index 6a043505c9e32..245dae01a7123 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java @@ -52,40 +52,52 @@ import javax.annotation.Nullable; /** - * {@code ParDo} is the core element-wise transform in Google Cloud - * Dataflow, invoking a user-specified function (from {@code I} to - * {@code Output}) on each of the elements of the input - * {@code PCollection} to produce zero or more output elements, all - * of which are collected into the output {@code PCollection}. + * {@link ParDo} is the core element-wise transform in Google Cloud + * Dataflow, invoking a user-specified function on each of the elements of the input + * {@link PCollection} to produce zero or more output elements, all + * of which are collected into the output {@link PCollection}. * *

      Elements are processed independently, and possibly in parallel across * distributed cloud resources. * - *

      The {@code ParDo} processing style is similar to what happens inside + *

      The {@link ParDo} processing style is similar to what happens inside * the "Mapper" or "Reducer" class of a MapReduce-style algorithm. * - *

      {@code DoFn}s

      + *

      {@link DoFn DoFns}

      * *

      The function to use to process each element is specified by a - * {@link DoFn}. + * {@link DoFn DoFn<InputT, OutputT>}, primarily via its + * {@link DoFn#processElement processElement} method. The {@link DoFn} may also + * override the default implementations of {@link DoFn#startBundle startBundle} + * and {@link DoFn#finishBundle finishBundle}. * - *

      Conceptually, when a {@code ParDo} transform is executed, the - * elements of the input {@code PCollection} are first divided up - * into some number of "batches". These are farmed off to distributed - * worker machines (or run locally, if using the - * {@link com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner}). - * For each batch of input elements, a fresh instance of the argument - * {@code DoFn} is created on a worker, then the {@code DoFn}'s - * optional {@link DoFn#startBundle} method is called to initialize it, - * then the {@code DoFn}'s required {@link DoFn#processElement} method - * is called on each of the input elements in the batch, then the - * {@code DoFn}'s optional {@link DoFn#finishBundle} method is called - * to complete its work, and finally the {@code DoFn} instance is - * thrown away. Each of the calls to any of the {@code DoFn}'s - * methods can produce zero or more output elements, which are - * collected together into a batch of output elements. All of the - * batches of output elements from all of the {@code DoFn} instances - * are "flattened" together into the output {@code PCollection}. + *

      Conceptually, when a {@link ParDo} transform is executed, the + * elements of the input {@link PCollection} are first divided up + * into some number of "bundles". These are farmed off to distributed + * worker machines (or run locally, if using the {@link DirectPipelineRunner}). + * For each bundle of input elements processing proceeds as follows: + * + *

        + *
      1. A fresh instance of the argument {@link DoFn} is created on a worker. This may + * be through deserialization or other means. If the {@link DoFn} subclass + * does not override {@link DoFn#startBundle startBundle} or + * {@link DoFn#finishBundle finishBundle} then this may be optimized since + * it cannot observe the start and end of a bundle.
      2. + *
      3. The {@link DoFn DoFn's} {@link DoFn#startBundle} method is called to + * initialize it. If this method is not overridden, the call may be optimized + * away.
      4. + *
      5. The {@link DoFn DoFn's} {@link DoFn#processElement} method + * is called on each of the input elements in the bundle.
      6. + *
      7. The {@link DoFn DoFn's} {@link DoFn#finishBundle} method is called + * to complete its work. After {@link DoFn#finishBundle} is called, the + * framework will never again invoke any of these three processing methods. + * If this method is not overridden, this call may be optimized away.
      8. + *
      + * + * Each of the calls to any of the {@link DoFn DoFn's} processing + * methods can produce zero or more output elements. All of the + * of output elements from all of the {@link DoFn} instances + * are included in the output {@link PCollection}. * *

      For example: * @@ -109,18 +121,20 @@ * }

      * *

      Each output element has the same timestamp and is in the same windows - * as its corresponding input element, and the output {@code PCollection} - * has the same - * {@link com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn} - * associated with it as the input. + * as its corresponding input element, and the output {@code PCollection} + * has the same {@link WindowFn} associated with it as the input. + * + *

      Naming {@link ParDo ParDo} transforms

      * - *

      Naming {@code ParDo}s

      + *

      The name of a transform is used to provide a name for any node in the + * {@link Pipeline} graph resulting from application of the transform. + * It is best practice to provide a name at the time of application, + * via {@link PCollection#apply(String, PTransform)}. Otherwise, + * a unique name - which may not be stable across pipeline revision - + * will be generated, based on the transform name. * - *

      A {@code ParDo} transform can be given a name using - * {@link #named}. While the system will automatically provide a name - * if none is specified explicitly, it is still a good practice to - * provide an explicit name, since that will probably make monitoring - * output more readable. For example: + *

      If a {@link ParDo} is applied exactly once inlined, then + * it can be given a name via {@link #named}. For example: * *

       {@code
        * PCollection words =
      @@ -133,14 +147,14 @@
        *
        * 

      Side Inputs

      * - *

      While a {@code ParDo} iterates over a single "main input" - * {@code PCollection}, it can take additional "side input" - * {@code PCollectionView}s. These side input - * {@code PCollectionView}s express styles of accessing - * {@code PCollection}s computed by earlier pipeline operations, - * passed in to the {@code ParDo} transform using + *

      While a {@link ParDo} processes elements from a single "main input" + * {@link PCollection}, it can take additional "side input" + * {@link PCollectionView PCollectionViews}. These side input + * {@link PCollectionView PCollectionViews} express styles of accessing + * {@link PCollection PCollections} computed by earlier pipeline operations, + * passed in to the {@link ParDo} transform using * {@link #withSideInputs}, and their contents accessible to each of - * the {@code DoFn} operations via {@link DoFn.ProcessContext#sideInput sideInput}. + * the {@link DoFn} operations via {@link DoFn.ProcessContext#sideInput sideInput}. * For example: * *

       {@code
      @@ -162,18 +176,18 @@
        *
        * 

      Side Outputs

      * - *

      Optionally, a {@code ParDo} transform can produce multiple - * output {@code PCollection}s, both a "main output" + *

      Optionally, a {@link ParDo} transform can produce multiple + * output {@link PCollection PCollections}, both a "main output" * {@code PCollection} plus any number of "side output" - * {@code PCollection}s, each keyed by a distinct {@link TupleTag}, - * and bundled in a {@link PCollectionTuple}. The {@code TupleTag}s - * to be used for the output {@code PCollectionTuple} is specified by - * invoking {@link #withOutputTags}. Unconsumed side outputs does not - * necessarily need to be explicity specified, even if the {@code DoFn} - * generates them. Within the {@code DoFn}, an element is added to the - * main output {@code PCollection} as normal, using + * {@link PCollection PCollections}, each keyed by a distinct {@link TupleTag}, + * and bundled in a {@link PCollectionTuple}. The {@link TupleTag TupleTags} + * to be used for the output {@link PCollectionTuple} are specified by + * invoking {@link #withOutputTags}. Unconsumed side outputs do not + * necessarily need to be explicitly specified, even if the {@link DoFn} + * generates them. Within the {@link DoFn}, an element is added to the + * main output {@link PCollection} as normal, using * {@link DoFn.Context#output}, while an element is added to a side output - * {@code PCollection} using {@link DoFn.Context#sideOutput}. For example: + * {@link PCollection} using {@link DoFn.Context#sideOutput}. For example: * *

       {@code
        * PCollection words = ...;
      @@ -229,46 +243,46 @@
        *
        * 

      Properties May Be Specified In Any Order

      * - *

      Several properties can be specified for a {@code ParDo} - * {@code PTransform}, including name, side inputs, side output tags, - * and {@code DoFn} to invoke. Only the {@code DoFn} is required; the + *

      Several properties can be specified for a {@link ParDo} + * {@link PTransform}, including name, side inputs, side output tags, + * and {@link DoFn} to invoke. Only the {@link DoFn} is required; the * name is encouraged but not required, and side inputs and side - * output tags are only specified when they're needed. These + * output tags are only specified when they're needed. These * properties can be specified in any order, as long as they're - * specified before the {@code ParDo} {@code PTransform} is applied. + * specified before the {@link ParDo} {@link PTransform} is applied. * *

      The approach used to allow these properties to be specified in * any order, with some properties omitted, is to have each of the * property "setter" methods defined as static factory methods on - * {@code ParDo} itself, which return an instance of either - * {@link ParDo.Unbound ParDo.Unbound} or - * {@link ParDo.Bound ParDo.Bound} nested classes, each of which offer + * {@link ParDo} itself, which return an instance of either + * {@link ParDo.Unbound} or + * {@link ParDo.Bound} nested classes, each of which offer * property setter instance methods to enable setting additional - * properties. {@code ParDo.Bound} is used for {@code ParDo} - * transforms whose {@code DoFn} is specified and whose input and - * output static types have been bound. {@code ParDo.Unbound} is used - * for {@code ParDo} transforms that have not yet had their - * {@code DoFn} specified. Only {@code ParDo.Bound} instances can be + * properties. {@link ParDo.Bound} is used for {@link ParDo} + * transforms whose {@link DoFn} is specified and whose input and + * output static types have been bound. {@link ParDo.Unbound ParDo.Unbound} is used + * for {@link ParDo} transforms that have not yet had their + * {@link DoFn} specified. Only {@link ParDo.Bound} instances can be * applied. * *

      Another benefit of this approach is that it reduces the number - * of type parameters that need to be specified manually. In - * particular, the input and output types of the {@code ParDo} - * {@code PTransform} are inferred automatically from the type - * parameters of the {@code DoFn} argument passed to {@link ParDo#of}. + * of type parameters that need to be specified manually. In + * particular, the input and output types of the {@link ParDo} + * {@link PTransform} are inferred automatically from the type + * parameters of the {@link DoFn} argument passed to {@link ParDo#of}. * *

      Output Coders

      * - *

      By default, the {@code Coder} of the - * elements of the main output {@code PCollection} is inferred from the - * concrete type of the {@code DoFn}'s output type {@code Output}. + *

      By default, the {@link Coder Coder<OutputT>} for the + * elements of the main output {@link PCollection PCollection<OutputT>} is + * inferred from the concrete type of the {@link DoFn DoFn<InputT, OutputT>}. * - *

      By default, the {@code Coder} of the elements of a side output - * {@code PCollection} is inferred from the concrete type of the - * corresponding {@code TupleTag}'s type {@code X}. To be - * successful, the {@code TupleTag} should be created as an instance + *

      By default, the {@link Coder Coder<SideOutputT>} for the elements of + * a side output {@link PCollection PCollection<SideOutputT>} is inferred + * from the concrete type of the corresponding {@link TupleTag TupleTag<SideOutputT>}. + * To be successful, the {@link TupleTag} should be created as an instance * of a trivial anonymous subclass, with {@code {}} suffixed to the - * constructor call. Such uses block Java's generic type parameter + * constructor call. Such uses block Java's generic type parameter * inference, so the {@code } argument must be provided explicitly. * For example: *

       {@code
      @@ -281,96 +295,96 @@
        * This style of {@code TupleTag} instantiation is used in the example of
        * multiple side outputs, above.
        *
      - * 

      Serializability of {@code DoFn}s

      + *

      Serializability of {@link DoFn DoFns}

      * - *

      A {@code DoFn} passed to a {@code ParDo} transform must be - * {@code Serializable}. This allows the {@code DoFn} instance + *

      A {@link DoFn} passed to a {@link ParDo} transform must be + * {@link Serializable}. This allows the {@link DoFn} instance * created in this "main program" to be sent (in serialized form) to - * remote worker machines and reconstituted for each batch of elements - * of the input {@code PCollection} being processed. A {@code DoFn} + * remote worker machines and reconstituted for each bundles of elements + * of the input {@link PCollection} being processed. A {@link DoFn} * can have instance variable state, and non-transient instance * variable state will be serialized in the main program and then - * deserialized on remote worker machines for each batch of elements + * deserialized on remote worker machines for each bundle of elements * to process. * - *

      To aid in ensuring that {@code DoFn}s are properly - * {@code Serializable}, even local execution using the + *

      To aid in ensuring that {@link DoFn DoFns} are properly + * {@link Serializable}, even local execution using the * {@link DirectPipelineRunner} will serialize and then deserialize - * {@code DoFn}s before executing them on a batch. + * {@link DoFn DoFns} before executing them on a bundle. * - *

      {@code DoFn}s expressed as anonymous inner classes can be + *

      {@link DoFn DoFns} expressed as anonymous inner classes can be * convenient, but due to a quirk in Java's rules for serializability, * non-static inner or nested classes (including anonymous inner * classes) automatically capture their enclosing class's instance in - * their serialized state. This can lead to including much more than - * intended in the serialized state of a {@code DoFn}, or even things - * that aren't {@code Serializable}. + * their serialized state. This can lead to including much more than + * intended in the serialized state of a {@link DoFn}, or even things + * that aren't {@link Serializable}. * *

      There are two ways to avoid unintended serialized state in a - * {@code DoFn}: + * {@link DoFn}: * *

        * - *
      • Define the {@code DoFn} as a named, static class. + *
      • Define the {@link DoFn} as a named, static class. * - *
      • Define the {@code DoFn} as an anonymous inner class inside of + *
      • Define the {@link DoFn} as an anonymous inner class inside of * a static method. * *
      * - *

      Both these approaches ensure that there is no implicit enclosing - * class instance serialized along with the {@code DoFn} instance. + *

      Both of these approaches ensure that there is no implicit enclosing + * instance serialized along with the {@link DoFn} instance. * *

      Prior to Java 8, any local variables of the enclosing * method referenced from within an anonymous inner class need to be - * marked as {@code final}. If defining the {@code DoFn} as a named + * marked as {@code final}. If defining the {@link DoFn} as a named * static class, such variables would be passed as explicit * constructor arguments and stored in explicit instance variables. * *

      There are three main ways to initialize the state of a - * {@code DoFn} instance processing a batch: + * {@link DoFn} instance processing a bundle: * *

        * - *
      • Define instance variable state (including implicit instance + *
      • Define instance variable state (including implicit instance * variables holding final variables captured by an anonymous inner - * class), initialized by the {@code DoFn}'s constructor (which is - * implicit for an anonymous inner class). This state will be + * class), initialized by the {@link DoFn}'s constructor (which is + * implicit for an anonymous inner class). This state will be * automatically serialized and then deserialized in the {@code DoFn} - * instance created for each batch. This method is good for state + * instance created for each bundle. This method is good for state * known when the original {@code DoFn} is created in the main * program, if it's not overly large. * - *
      • Compute the state as a singleton {@code PCollection} and pass it - * in as a side input to the {@code DoFn}. This is good if the state + *
      • Compute the state as a singleton {@link PCollection} and pass it + * in as a side input to the {@link DoFn}. This is good if the state * needs to be computed by the pipeline, or if the state is very large * and so is best read from file(s) rather than sent as part of the * {@code DoFn}'s serialized state. * - *
      • Initialize the state in each {@code DoFn} instance, in - * {@link DoFn#startBundle}. This is good if the initialization + *
      • Initialize the state in each {@link DoFn} instance, in + * {@link DoFn#startBundle}. This is good if the initialization * doesn't depend on any information known only by the main program or * computed by earlier pipeline operations, but is the same for all - * instances of this {@code DoFn} for all program executions, say + * instances of this {@link DoFn} for all program executions, say * setting up empty caches or initializing constant data. * *
      * *

      No Global Shared State

      * - *

      {@code ParDo} operations are intended to be able to run in - * parallel across multiple worker machines. This precludes easy - * sharing and updating mutable state across those machines. There is + *

      {@link ParDo} operations are intended to be able to run in + * parallel across multiple worker machines. This precludes easy + * sharing and updating mutable state across those machines. There is * no support in the Google Cloud Dataflow system for communicating * and synchronizing updates to shared state across worker machines, * so programs should not access any mutable static variable state in - * their {@code DoFn}, without understanding that the Java processes + * their {@link DoFn}, without understanding that the Java processes * for the main program and workers will each have its own independent * copy of such state, and there won't be any automatic copying of - * that state across Java processes. All information should be - * communicated to {@code DoFn} instances via main and side inputs and + * that state across Java processes. All information should be + * communicated to {@link DoFn} instances via main and side inputs and * serialized state, and all output should be communicated from a - * {@code DoFn} instance via main and side outputs, in the absence of + * {@link DoFn} instance via main and side outputs, in the absence of * external communication mechanisms written by user code. * *

      Fault Tolerance

      @@ -378,53 +392,53 @@ *

      In a distributed system, things can fail: machines can crash, * machines can be unable to communicate across the network, etc. * While individual failures are rare, the larger the job, the greater - * the chance that something, somewhere, will fail. The Google Cloud + * the chance that something, somewhere, will fail. The Google Cloud * Dataflow service strives to mask such failures automatically, - * principally by retrying failed {@code DoFn} batches. This means - * that a {@code DoFn} instance might process a batch partially, then + * principally by retrying failed {@link DoFn} bundle. This means + * that a {@code DoFn} instance might process a bundle partially, then * crash for some reason, then be rerun (often on a different worker - * machine) on that same batch and on the same elements as before. - * Sometimes two or more {@code DoFn} instances will be running on the - * same batch simultaneously, with the system taking the results of - * the first instance to complete successfully. Consequently, the - * code in a {@code DoFn} needs to be written such that these + * machine) on that same bundle and on the same elements as before. + * Sometimes two or more {@link DoFn} instances will be running on the + * same bundle simultaneously, with the system taking the results of + * the first instance to complete successfully. Consequently, the + * code in a {@link DoFn} needs to be written such that these * duplicate (sequential or concurrent) executions do not cause - * problems. If the outputs of a {@code DoFn} are a pure function of - * its inputs, then this requirement is satisfied. However, if a - * {@code DoFn}'s execution has external side-effects, say performing - * updates to external HTTP services, then the {@code DoFn}'s code + * problems. If the outputs of a {@link DoFn} are a pure function of + * its inputs, then this requirement is satisfied. However, if a + * {@link DoFn DoFn's} execution has external side-effects, such as performing + * updates to external HTTP services, then the {@link DoFn DoFn's} code * needs to take care to ensure that those updates are idempotent and - * that concurrent updates are acceptable. This property can be + * that concurrent updates are acceptable. This property can be * difficult to achieve, so it is advisable to strive to keep - * {@code DoFn}s as pure functions as much as possible. + * {@link DoFn DoFns} as pure functions as much as possible. * *

      Optimization

      * *

      The Google Cloud Dataflow service automatically optimizes a - * pipeline before it is executed. A key optimization, fusion, - * relates to ParDo operations. If one ParDo operation produces a - * PCollection that is then consumed as the main input of another - * ParDo operation, the two ParDo operations will be fused + * pipeline before it is executed. A key optimization, fusion, + * relates to {@link ParDo} operations. If one {@link ParDo} operation produces a + * {@link PCollection} that is then consumed as the main input of another + * {@link ParDo} operation, the two {@link ParDo} operations will be fused * together into a single ParDo operation and run in a single pass; - * this is "producer-consumer fusion". Similarly, if - * two or more ParDo operations have the same PCollection main input, - * they will be fused into a single ParDo that makes just one pass - * over the input PCollection; this is "sibling fusion". + * this is "producer-consumer fusion". Similarly, if + * two or more ParDo operations have the same {@link PCollection} main input, + * they will be fused into a single {@link ParDo} that makes just one pass + * over the input {@link PCollection}; this is "sibling fusion". * *

      If after fusion there are no more unfused references to a - * PCollection (e.g., one between a producer ParDo and a consumer - * ParDo), the PCollection itself is "fused away" and won't ever be + * {@link PCollection} (e.g., one between a producer ParDo and a consumer + * {@link ParDo}), the {@link PCollection} itself is "fused away" and won't ever be * written to disk, saving all the I/O and space expense of * constructing it. * *

      The Google Cloud Dataflow service applies fusion as much as - * possible, greatly reducing the cost of executing pipelines. As a - * result, it is essentially "free" to write ParDo operations in a - * very modular, composable style, each ParDo operation doing one - * clear task, and stringing together sequences of ParDo operations to - * get the desired overall effect. Such programs can be easier to + * possible, greatly reducing the cost of executing pipelines. As a + * result, it is essentially "free" to write {@link ParDo} operations in a + * very modular, composable style, each {@link ParDo} operation doing one + * clear task, and stringing together sequences of {@link ParDo} operations to + * get the desired overall effect. Such programs can be easier to * understand, easier to unit-test, easier to extend and evolve, and - * easier to reuse in new programs. The predefined library of + * easier to reuse in new programs. The predefined library of * PTransforms that come with Google Cloud Dataflow makes heavy use of * this modular, composable style, trusting to the Google Cloud * Dataflow service's optimizer to "flatten out" all the compositions @@ -436,44 +450,44 @@ public class ParDo { /** - * Creates a {@code ParDo} {@code PTransform} with the given name. + * Creates a {@link ParDo} {@link PTransform} with the given name. * - *

      See the discussion of Naming above for more explanation. + *

      See the discussion of naming above for more explanation. * - *

      The resulting {@code PTransform} is incomplete, and its - * input/output types are not yet bound. Use + *

      The resulting {@link PTransform} is incomplete, and its + * input/output types are not yet bound. Use * {@link ParDo.Unbound#of} to specify the {@link DoFn} to * invoke, which will also bind the input/output types of this - * {@code PTransform}. + * {@link PTransform}. */ public static Unbound named(String name) { return new Unbound().named(name); } /** - * Creates a {@code ParDo} {@code PTransform} with the given + * Creates a {@link ParDo} {@link PTransform} with the given * side inputs. * - *

      Side inputs are {@link PCollectionView}s, whose contents are + *

      Side inputs are {@link PCollectionView PCollectionViews}, whose contents are * computed during pipeline execution and then made accessible to - * {@code DoFn} code via {@link DoFn.ProcessContext#sideInput sideInput}. Each - * invocation of the {@code DoFn} receives the same values for these + * {@link DoFn} code via {@link DoFn.ProcessContext#sideInput sideInput}. Each + * invocation of the {@link DoFn} receives the same values for these * side inputs. * *

      See the discussion of Side Inputs above for more explanation. * - *

      The resulting {@code PTransform} is incomplete, and its - * input/output types are not yet bound. Use + *

      The resulting {@link PTransform} is incomplete, and its + * input/output types are not yet bound. Use * {@link ParDo.Unbound#of} to specify the {@link DoFn} to * invoke, which will also bind the input/output types of this - * {@code PTransform}. + * {@link PTransform}. */ public static Unbound withSideInputs(PCollectionView... sideInputs) { return new Unbound().withSideInputs(sideInputs); } /** - * Creates a {@code ParDo} with the given side inputs. + * Creates a {@link ParDo} with the given side inputs. * *

      Side inputs are {@link PCollectionView}s, whose contents are * computed during pipeline execution and then made accessible to @@ -481,11 +495,11 @@ public static Unbound withSideInputs(PCollectionView... sideInputs) { * *

      See the discussion of Side Inputs above for more explanation. * - *

      The resulting {@code PTransform} is incomplete, and its - * input/output types are not yet bound. Use + *

      The resulting {@link PTransform} is incomplete, and its + * input/output types are not yet bound. Use * {@link ParDo.Unbound#of} to specify the {@link DoFn} to * invoke, which will also bind the input/output types of this - * {@code PTransform}. + * {@link PTransform}. */ public static Unbound withSideInputs( Iterable> sideInputs) { @@ -493,18 +507,18 @@ public static Unbound withSideInputs( } /** - * Creates a multi-output {@code ParDo} {@code PTransform} whose + * Creates a multi-output {@link ParDo} {@link PTransform} whose * output {@link PCollection}s will be referenced using the given main * output and side output tags. * - *

      {@link TupleTag}s are used to name (with its static element + *

      {@link TupleTag TupleTags} are used to name (with its static element * type {@code T}) each main and side output {@code PCollection}. - * This {@code PTransform}'s {@link DoFn} emits elements to the main - * output {@code PCollection} as normal, using - * {@link DoFn.Context#output}. The {@code DoFn} emits elements to + * This {@link PTransform PTransform's} {@link DoFn} emits elements to the main + * output {@link PCollection} as normal, using + * {@link DoFn.Context#output}. The {@link DoFn} emits elements to * a side output {@code PCollection} using * {@link DoFn.Context#sideOutput}, passing that side output's tag - * as an argument. The result of invoking this {@code PTransform} + * as an argument. The result of invoking this {@link PTransform} * will be a {@link PCollectionTuple}, and any of the the main and * side output {@code PCollection}s can be retrieved from it via * {@link PCollectionTuple#get}, passing the output's tag as an @@ -512,10 +526,10 @@ public static Unbound withSideInputs( * *

      See the discussion of Side Outputs above for more explanation. * - *

      The resulting {@code PTransform} is incomplete, and its input - * type is not yet bound. Use {@link ParDo.UnboundMulti#of} + *

      The resulting {@link PTransform} is incomplete, and its input + * type is not yet bound. Use {@link ParDo.UnboundMulti#of} * to specify the {@link DoFn} to invoke, which will also bind the - * input type of this {@code PTransform}. + * input type of this {@link PTransform}. */ public static UnboundMulti withOutputTags( TupleTag mainOutputTag, @@ -524,13 +538,13 @@ public static UnboundMulti withOutputTags( } /** - * Creates a {@code ParDo} {@code PTransform} that will invoke the + * Creates a {@link ParDo} {@link PTransform} that will invoke the * given {@link DoFn} function. * - *

      The resulting {@code PTransform}'s types have been bound, with the + *

      The resulting {@link PTransform PTransform's} types have been bound, with the * input being a {@code PCollection} and the output a * {@code PCollection}, inferred from the types of the argument - * {@code DoFn}. It is ready to be applied, or further + * {@code DoFn}. It is ready to be applied, or further * properties can be set on it first. */ public static Bound of(DoFn fn) { @@ -543,13 +557,13 @@ public static Bound of(DoFn } /** - * Creates a {@code ParDo} {@code PTransform} that will invoke the + * Creates a {@link ParDo} {@link PTransform} that will invoke the * given {@link DoFnWithContext} function. * - *

      The resulting {@code PTransform}'s types have been bound, with the + *

      The resulting {@link PTransform PTransform's} types have been bound, with the * input being a {@code PCollection} and the output a * {@code PCollection}, inferred from the types of the argument - * {@code DoFn}. It is ready to be applied, or further + * {@code DoFn}. It is ready to be applied, or further * properties can be set on it first. * *

      {@link DoFnWithContext} is an experimental alternative to @@ -561,11 +575,11 @@ public static Bound of(DoFnWithContextBefore being applied, {@link ParDo.Unbound#of} must be * invoked to specify the {@link DoFn} to invoke, which will also - * bind the input/output types of this {@code PTransform}. + * bind the input/output types of this {@link PTransform}. */ public static class Unbound { private final String name; @@ -581,18 +595,18 @@ public static class Unbound { } /** - * Returns a new {@code ParDo} transform that's like this - * transform but with the specified name. Does not modify this - * transform. The resulting transform is still incomplete. + * Returns a new {@link ParDo} transform that's like this + * transform but with the specified name. Does not modify this + * transform. The resulting transform is still incomplete. * - *

      See the discussion of Naming above for more explanation. + *

      See the discussion of naming above for more explanation. */ public Unbound named(String name) { return new Unbound(name, sideInputs); } /** - * Returns a new {@code ParDo} transform that's like this + * Returns a new {@link ParDo} transform that's like this * transform but with the specified side inputs. * Does not modify this transform. The resulting transform is * still incomplete. @@ -605,9 +619,9 @@ public Unbound withSideInputs(PCollectionView... sideInputs) { } /** - * Returns a new {@code ParDo} transform that's like this - * transform but with the specified side inputs. Does not modify - * this transform. The resulting transform is still incomplete. + * Returns a new {@link ParDo} transform that is like this + * transform but with the specified side inputs. Does not modify + * this transform. The resulting transform is still incomplete. * *

      See the discussion of Side Inputs above and on * {@link ParDo#withSideInputs} for more explanation. @@ -618,9 +632,9 @@ public Unbound withSideInputs( } /** - * Returns a new multi-output {@code ParDo} transform that's like + * Returns a new multi-output {@link ParDo} transform that's like * this transform but with the specified main and side output - * tags. Does not modify this transform. The resulting transform + * tags. Does not modify this transform. The resulting transform * is still incomplete. * *

      See the discussion of Side Outputs above and on @@ -633,10 +647,10 @@ public UnboundMulti withOutputTags(TupleTag mainOutp } /** - * Returns a new {@code ParDo} {@code PTransform} that's like this + * Returns a new {@link ParDo} {@link PTransform} that's like this * transform but that will invoke the given {@link DoFn} - * function, and that has its input and output types bound. Does - * not modify this transform. The resulting {@code PTransform} is + * function, and that has its input and output types bound. Does + * not modify this transform. The resulting {@link PTransform} is * sufficiently specified to be applied, but more properties can * still be specified. */ @@ -645,10 +659,10 @@ public Bound of(DoFn fn) { } /** - * Returns a new {@code ParDo} {@code PTransform} that's like this + * Returns a new {@link ParDo} {@link PTransform} that's like this * transform but which will invoke the given {@link DoFnWithContext} - * function, and which has its input and output types bound. Does - * not modify this transform. The resulting {@code PTransform} is + * function, and which has its input and output types bound. Does + * not modify this transform. The resulting {@link PTransform} is * sufficiently specified to be applied, but more properties can * still be specified. */ @@ -658,7 +672,7 @@ public Bound of(DoFnWithContext}, + * A {@link PTransform} that, when applied to a {@code PCollection}, * invokes a user-specified {@code DoFn} on all its elements, * with all its outputs collected into an output * {@code PCollection}. @@ -666,8 +680,8 @@ public Bound of(DoFnWithContextA multi-output form of this transform can be created with * {@link ParDo.Bound#withOutputTags}. * - * @param the type of the (main) input {@code PCollection} elements - * @param the type of the (main) output {@code PCollection} elements + * @param the type of the (main) input {@link PCollection} elements + * @param the type of the (main) output {@link PCollection} elements */ public static class Bound extends PTransform, PCollection> { @@ -684,9 +698,9 @@ public static class Bound } /** - * Returns a new {@code ParDo} {@code PTransform} that's like this - * {@code PTransform} but with the specified name. Does not - * modify this {@code PTransform}. + * Returns a new {@link ParDo} {@link PTransform} that's like this + * {@link PTransform} but with the specified name. Does not + * modify this {@link PTransform}. * *

      See the discussion of Naming above for more explanation. */ @@ -695,9 +709,9 @@ public Bound named(String name) { } /** - * Returns a new {@code ParDo} {@code PTransform} that's like this - * {@code PTransform} but with the specified side inputs. Does not - * modify this {@code PTransform}. + * Returns a new {@link ParDo} {@link PTransform} that's like this + * {@link PTransform} but with the specified side inputs. Does not + * modify this {@link PTransform}. * *

      See the discussion of Side Inputs above and on * {@link ParDo#withSideInputs} for more explanation. @@ -707,9 +721,9 @@ public Bound withSideInputs(PCollectionView... sideInputs) { } /** - * Returns a new {@code ParDo} {@code PTransform} that's like this - * {@code PTransform} but with the specified side inputs. Does not - * modify this {@code PTransform}. + * Returns a new {@link ParDo} {@link PTransform} that's like this + * {@link PTransform} but with the specified side inputs. Does not + * modify this {@link PTransform}. * *

      See the discussion of Side Inputs above and on * {@link ParDo#withSideInputs} for more explanation. @@ -720,9 +734,9 @@ public Bound withSideInputs( } /** - * Returns a new multi-output {@code ParDo} {@code PTransform} - * that's like this {@code PTransform} but with the specified main - * and side output tags. Does not modify this {@code PTransform}. + * Returns a new multi-output {@link ParDo} {@link PTransform} + * that's like this {@link PTransform} but with the specified main + * and side output tags. Does not modify this {@link PTransform}. * *

      See the discussion of Side Outputs above and on * {@link ParDo#withOutputTags} for more explanation. @@ -772,12 +786,12 @@ public List> getSideInputs() { } /** - * An incomplete multi-output {@code ParDo} transform, with unbound + * An incomplete multi-output {@link ParDo} transform, with unbound * input type. * *

      Before being applied, {@link ParDo.UnboundMulti#of} must be * invoked to specify the {@link DoFn} to invoke, which will also - * bind the input type of this {@code PTransform}. + * bind the input type of this {@link PTransform}. * * @param the type of the main output {@code PCollection} elements */ @@ -798,9 +812,9 @@ public static class UnboundMulti { } /** - * Returns a new multi-output {@code ParDo} transform that's like - * this transform but with the specified name. Does not modify - * this transform. The resulting transform is still incomplete. + * Returns a new multi-output {@link ParDo} transform that's like + * this transform but with the specified name. Does not modify + * this transform. The resulting transform is still incomplete. * *

      See the discussion of Naming above for more explanation. */ @@ -810,9 +824,9 @@ public UnboundMulti named(String name) { } /** - * Returns a new multi-output {@code ParDo} transform that's like - * this transform but with the specified side inputs. Does not - * modify this transform. The resulting transform is still + * Returns a new multi-output {@link ParDo} transform that's like + * this transform but with the specified side inputs. Does not + * modify this transform. The resulting transform is still * incomplete. * *

      See the discussion of Side Inputs above and on @@ -824,9 +838,9 @@ public UnboundMulti withSideInputs( } /** - * Returns a new multi-output {@code ParDo} transform that's like - * this transform but with the specified side inputs. Does not - * modify this transform. The resulting transform is still + * Returns a new multi-output {@link ParDo} transform that's like + * this transform but with the specified side inputs. Does not + * modify this transform. The resulting transform is still * incomplete. * *

      See the discussion of Side Inputs above and on @@ -840,11 +854,11 @@ public UnboundMulti withSideInputs( } /** - * Returns a new multi-output {@code ParDo} {@code PTransform} + * Returns a new multi-output {@link ParDo} {@link PTransform} * that's like this transform but that will invoke the given * {@link DoFn} function, and that has its input type bound. - * Does not modify this transform. The resulting - * {@code PTransform} is sufficiently specified to be applied, but + * Does not modify this transform. The resulting + * {@link PTransform} is sufficiently specified to be applied, but * more properties can still be specified. */ public BoundMulti of(DoFn fn) { @@ -853,11 +867,11 @@ public BoundMulti of(DoFn fn) { } /** - * Returns a new multi-output {@code ParDo} {@code PTransform} + * Returns a new multi-output {@link ParDo} {@link PTransform} * that's like this transform but which will invoke the given * {@link DoFnWithContext} function, and which has its input type bound. - * Does not modify this transform. The resulting - * {@code PTransform} is sufficiently specified to be applied, but + * Does not modify this transform. The resulting + * {@link PTransform} is sufficiently specified to be applied, but * more properties can still be specified. */ public BoundMulti of(DoFnWithContext fn) { @@ -866,10 +880,10 @@ public BoundMulti of(DoFnWithContext } /** - * A {@code PTransform} that, when applied to a + * A {@link PTransform} that, when applied to a * {@code PCollection}, invokes a user-specified * {@code DoFn} on all its elements, which can emit elements - * to any of the {@code PTransform}'s main and side output + * to any of the {@link PTransform}'s main and side output * {@code PCollection}s, which are bundled into a result * {@code PCollectionTuple}. * @@ -897,9 +911,9 @@ public static class BoundMulti } /** - * Returns a new multi-output {@code ParDo} {@code PTransform} - * that's like this {@code PTransform} but with the specified - * name. Does not modify this {@code PTransform}. + * Returns a new multi-output {@link ParDo} {@link PTransform} + * that's like this {@link PTransform} but with the specified + * name. Does not modify this {@link PTransform}. * *

      See the discussion of Naming above for more explanation. */ @@ -909,9 +923,9 @@ public BoundMulti named(String name) { } /** - * Returns a new multi-output {@code ParDo} {@code PTransform} - * that's like this {@code PTransform} but with the specified side - * inputs. Does not modify this {@code PTransform}. + * Returns a new multi-output {@link ParDo} {@link PTransform} + * that's like this {@link PTransform} but with the specified side + * inputs. Does not modify this {@link PTransform}. * *

      See the discussion of Side Inputs above and on * {@link ParDo#withSideInputs} for more explanation. @@ -922,9 +936,9 @@ public BoundMulti withSideInputs( } /** - * Returns a new multi-output {@code ParDo} {@code PTransform} - * that's like this {@code PTransform} but with the specified side - * inputs. Does not modify this {@code PTransform}. + * Returns a new multi-output {@link ParDo} {@link PTransform} + * that's like this {@link PTransform} but with the specified side + * inputs. Does not modify this {@link PTransform}. * *

      See the discussion of Side Inputs above and on * {@link ParDo#withSideInputs} for more explanation. From 23d96f3cd3a185c8d9429903c3f0be0a86daf064 Mon Sep 17 00:00:00 2001 From: millsd Date: Mon, 21 Dec 2015 15:01:58 -0800 Subject: [PATCH 1244/1541] Avoid integer overflow when computing maximum cache weight ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110719783 --- .../cloud/dataflow/sdk/runners/worker/DataflowWorker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java index e3536a92a5273..65c3f41728f33 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java @@ -115,7 +115,7 @@ public class DataflowWorker { */ private static final int OVERHEAD_WEIGHT = 8; - private static final int MEGABYTES = 1024 * 1024; + private static final long MEGABYTES = 1024 * 1024; public static final int DEFAULT_STATUS_PORT = 18081; From 61bd0816f743abddf78bc4d6bf9a8cfdcb6ac12d Mon Sep 17 00:00:00 2001 From: millsd Date: Mon, 21 Dec 2015 15:27:13 -0800 Subject: [PATCH 1245/1541] Enable state caching for per-key windmill state ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110721947 --- .../worker/CachingSideInputReader.java | 33 +- .../worker/DataflowSideInputReader.java | 14 +- .../sdk/runners/worker/DataflowWorker.java | 10 +- .../worker/StreamingDataflowWorker.java | 14 +- .../worker/StreamingModeExecutionContext.java | 27 +- .../{SizedWeigher.java => Weighers.java} | 34 +- .../runners/worker/WindmillStateCache.java | 292 ++++++++ .../worker/WindmillStateInternals.java | 630 +++++++++++------- .../runners/worker/WindmillStateReader.java | 42 +- .../sdk/transforms/ApproximateQuantiles.java | 28 +- .../cloud/dataflow/sdk/util/Weighted.java | 27 + ...ader.java => WeightedSideInputReader.java} | 18 +- .../util/{Sized.java => WeightedValue.java} | 19 +- .../util/state/InMemoryStateInternals.java | 2 +- .../sdk/util/state/MergingStateInternals.java | 2 +- .../sdk/util/state/StateNamespace.java | 9 + .../sdk/util/state/StateNamespaceForTest.java | 5 + .../sdk/util/state/StateNamespaces.java | 21 + .../dataflow/sdk/util/state/StateTag.java | 4 +- .../dataflow/sdk/util/state/StateTags.java | 9 +- .../cloud/dataflow/sdk/DataflowMatchers.java | 65 ++ .../runners/dataflow/CustomSourcesTest.java | 5 +- .../worker/CachingSideInputReaderTest.java | 33 +- .../worker/DataflowSideInputReaderTest.java | 16 +- .../StreamingModeExecutionContextTest.java | 13 +- ...ava => WeightedDirectSideInputReader.java} | 33 +- .../worker/WindmillStateCacheTest.java | 210 ++++++ .../worker/WindmillStateInternalsTest.java | 265 +++++--- .../dataflow/sdk/util/ApiSurfaceTest.java | 2 +- 29 files changed, 1376 insertions(+), 506 deletions(-) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/{SizedWeigher.java => Weighers.java} (61%) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateCache.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Weighted.java rename sdk/src/main/java/com/google/cloud/dataflow/sdk/util/{SizedSideInputReader.java => WeightedSideInputReader.java} (73%) rename sdk/src/main/java/com/google/cloud/dataflow/sdk/util/{Sized.java => WeightedValue.java} (67%) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/DataflowMatchers.java rename sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/{SizedDirectSideInputReader.java => WeightedDirectSideInputReader.java} (60%) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateCacheTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CachingSideInputReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CachingSideInputReader.java index b1dd17efbdd33..afd77f0666cee 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CachingSideInputReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CachingSideInputReader.java @@ -19,8 +19,8 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.util.PCollectionViewWindow; import com.google.cloud.dataflow.sdk.util.SideInputReader; -import com.google.cloud.dataflow.sdk.util.Sized; -import com.google.cloud.dataflow.sdk.util.SizedSideInputReader; +import com.google.cloud.dataflow.sdk.util.WeightedSideInputReader; +import com.google.cloud.dataflow.sdk.util.WeightedValue; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.common.cache.Cache; @@ -37,19 +37,19 @@ * with a {@code Cache} created by anything other than the SDK. */ final class CachingSideInputReader - extends SizedSideInputReader.Defaults - implements SizedSideInputReader { - private final SizedSideInputReader subReader; - private final Cache, Sized> cache; + extends WeightedSideInputReader.Defaults + implements WeightedSideInputReader { + private final WeightedSideInputReader subReader; + private final Cache, WeightedValue> cache; - private CachingSideInputReader( - SizedSideInputReader subReader, Cache, Sized> cache) { + private CachingSideInputReader(WeightedSideInputReader subReader, + Cache, WeightedValue> cache) { this.subReader = subReader; this.cache = cache; } - public static CachingSideInputReader of( - SizedSideInputReader subReader, Cache, Sized> cache) { + public static CachingSideInputReader of(WeightedSideInputReader subReader, + Cache, WeightedValue> cache) { return new CachingSideInputReader(subReader, cache); } @@ -64,24 +64,25 @@ public boolean isEmpty() { } @Override - public Sized getSized( + public WeightedValue getWeighted( final PCollectionView view, final BoundedWindow window) { PCollectionViewWindow cacheKey = PCollectionViewWindow.of(view, window); try { @SuppressWarnings("unchecked") // safely uncasting the thing from the callback - Sized sideInputContents = (Sized) cache.get(cacheKey, - new Callable>() { + WeightedValue sideInputContents = (WeightedValue) cache.get(cacheKey, + new Callable>() { @Override - public Sized call() { + public WeightedValue call() { @SuppressWarnings("unchecked") // safe covariant cast - Sized value = (Sized) subReader.getSized(view, window); + WeightedValue value = + (WeightedValue) subReader.getWeighted(view, window); return value; } }); return sideInputContents; } catch (ExecutionException checkedException) { - // The call to subReader.getSized() is not permitted to throw any checked exceptions, + // The call to subReader.getWeighted() is not permitted to throw any checked exceptions, // so the Callable created above should not throw any either. throw new RuntimeException("Unexpected checked exception.", checkedException.getCause()); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReader.java index 545f32bacad43..ed1d702791ab1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReader.java @@ -23,8 +23,8 @@ import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.util.PTuple; import com.google.cloud.dataflow.sdk.util.SideInputReader; -import com.google.cloud.dataflow.sdk.util.Sized; -import com.google.cloud.dataflow.sdk.util.SizedSideInputReader; +import com.google.cloud.dataflow.sdk.util.WeightedSideInputReader; +import com.google.cloud.dataflow.sdk.util.WeightedValue; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; @@ -45,8 +45,8 @@ * amount of data for each access. */ public class DataflowSideInputReader - extends SizedSideInputReader.Defaults - implements SizedSideInputReader { + extends WeightedSideInputReader.Defaults + implements WeightedSideInputReader { /** An observer for each side input to count its size as it is being read. */ private final Map, ByteSizeObserver> observers; @@ -110,7 +110,7 @@ public boolean isEmpty() { * the value for the appropriate window. */ @Override - public Sized getSized(PCollectionView view, final BoundedWindow window) { + public WeightedValue getWeighted(PCollectionView view, final BoundedWindow window) { final TupleTag>> tag = view.getTagInternal(); if (!sideInputValues.has(tag)) { throw new IllegalArgumentException("calling getSideInput() with unknown view"); @@ -125,7 +125,7 @@ public Sized getSized(PCollectionView view, final BoundedWindow window value = view.fromIterableInternal(sideInputValues.get(tag)); long bytesRead = observer.getBytes(); observer.reset(); - return Sized.of(value, overhead + bytesRead); + return WeightedValue.of(value, overhead + bytesRead); } else { final long[] sum = new long[]{ 0L }; value = view.fromIterableInternal( @@ -142,7 +142,7 @@ public boolean apply(WindowedValue element) { return containsWindow; } })); - return Sized.of(value, overhead + sum[0]); + return WeightedValue.of(value, overhead + sum[0]); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java index 65c3f41728f33..5ade506440ab9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorker.java @@ -39,8 +39,8 @@ import com.google.cloud.dataflow.sdk.util.CloudMetricUtils; import com.google.cloud.dataflow.sdk.util.PCollectionViewWindow; import com.google.cloud.dataflow.sdk.util.SideInputReader; -import com.google.cloud.dataflow.sdk.util.Sized; import com.google.cloud.dataflow.sdk.util.UserCodeException; +import com.google.cloud.dataflow.sdk.util.WeightedValue; import com.google.cloud.dataflow.sdk.util.common.Counter; import com.google.cloud.dataflow.sdk.util.common.CounterSet; import com.google.cloud.dataflow.sdk.util.common.Metric; @@ -97,7 +97,7 @@ public class DataflowWorker { /** * A side input cache shared between all execution contexts. */ - private final Cache, Sized> sideInputCache; + private final Cache, WeightedValue> sideInputCache; /** * Status server returning health of worker. @@ -124,7 +124,7 @@ public DataflowWorker(WorkUnitClient workUnitClient, DataflowWorkerHarnessOption this.options = options; this.sideInputCache = CacheBuilder.newBuilder() .maximumWeight(options.getWorkerCacheMb() * MEGABYTES) // weights are in bytes - .weigher(SizedWeigher., Object>withBaseWeight(OVERHEAD_WEIGHT)) + .weigher(Weighers.fixedWeightKeys(OVERHEAD_WEIGHT)) .softValues() .build(); } @@ -421,11 +421,11 @@ public abstract WorkItemServiceState reportWorkItemStatus(WorkItemStatus workIte */ private static class DataflowWorkerExecutionContext extends BatchModeExecutionContext { - private final Cache, Sized> cache; + private final Cache, WeightedValue> cache; private final PipelineOptions options; public DataflowWorkerExecutionContext( - Cache, Sized> cache, PipelineOptions options) { + Cache, WeightedValue> cache, PipelineOptions options) { super(options); this.cache = cache; this.options = options; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java index 81bab4e2402c9..121cf16de9f00 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingDataflowWorker.java @@ -221,6 +221,8 @@ public ReaderCacheEntry(UnboundedSource.UnboundedReader reader, long token) { private ConcurrentMap stateNameMap; private ConcurrentMap systemNameToComputationIdMap; + private WindmillStateCache stateCache = new WindmillStateCache(); + private ThreadFactory threadFactory; private BoundedQueueExecutor workUnitExecutor; private ExecutorService commitExecutor; @@ -479,8 +481,8 @@ private void process(final String computation, final MapTask mapTask, WorkerAndContext workerAndContext = mapTaskExecutors.get(computation).poll(); if (workerAndContext == null) { CounterSet counters = new CounterSet(); - context = new StreamingModeExecutionContext( - mapTask.getSystemName(), readerCache.get(computation), stateNameMap); + context = new StreamingModeExecutionContext(mapTask.getSystemName(), + readerCache.get(computation), stateNameMap, stateCache.forComputation(computation)); StateSampler sampler = new StateSampler(mapTask.getStageName() + "-", counters.getAddCounterMutator()); // In streaming mode, state samplers are long lived. So here a unique id is generated as @@ -943,6 +945,8 @@ public void handle( printThreads(responseWriter); } else if (target.equals("/heapz")) { dumpHeap(responseWriter); + } else if (target.equals("/cachez")) { + stateCache.printDetailedHtml(responseWriter); } else { printHeader(responseWriter); printResources(responseWriter); @@ -978,6 +982,11 @@ private void printMetrics(PrintWriter response) { response.println(""); } response.println(""); + + stateCache.printSummaryHtml(response); + + metricTrackingWindmillServer.printHtml(response); + response.println("Active Keys:
        "); for (Map.Entry computationEntry : activeWorkMap.entrySet()) { @@ -988,7 +997,6 @@ private void printMetrics(PrintWriter response) { response.println(""); } response.println("
      "); - metricTrackingWindmillServer.printHtml(response); } private void printResources(PrintWriter response) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java index 116d3c6a7dc5d..38e005e0f24bd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContext.java @@ -65,6 +65,7 @@ public class StreamingModeExecutionContext // Per-key cache of active Reader objects in use by this process. private final ConcurrentMap readerCache; private final ConcurrentMap stateNameMap; + private final WindmillStateCache.ForComputation stateCache; private Windmill.WorkItem work; @Nullable private Instant inputDataWatermark; @@ -74,14 +75,14 @@ public class StreamingModeExecutionContext private Windmill.WorkItemCommitRequest.Builder outputBuilder; private UnboundedSource.UnboundedReader activeReader; - public StreamingModeExecutionContext( - String stageName, + public StreamingModeExecutionContext(String stageName, ConcurrentMap readerCache, - ConcurrentMap stateNameMap) { + ConcurrentMap stateNameMap, WindmillStateCache.ForComputation stateCache) { this.stageName = stageName; this.sideInputCache = new HashMap<>(); this.readerCache = readerCache; this.stateNameMap = stateNameMap; + this.stateCache = stateCache; } public void start( @@ -380,9 +381,10 @@ private Windmill.Timer.Type timerType(TimeDomain domain) { } class StepContext extends BaseExecutionContext.StepContext { + private static final String DEFAULT_STATE_FAMILY = ""; + private WindmillStateInternals stateInternals; private WindmillTimerInternals timerInternals; - private final String prefix; private final String stateFamily; private final Supplier scopedReadStateSupplier; @@ -390,14 +392,9 @@ public StepContext( final String stepName, String transformName, final StateSampler stateSampler) { super(StreamingModeExecutionContext.this, stepName, transformName); - if (stateNameMap.isEmpty()) { - this.prefix = transformName; - this.stateFamily = ""; - } else { - String mappedName = stateNameMap.get(transformName); - this.prefix = mappedName == null ? "" : mappedName; - this.stateFamily = prefix; - } + String mappedName = stateNameMap.get(transformName); + this.stateFamily = mappedName == null ? DEFAULT_STATE_FAMILY : mappedName; + this.scopedReadStateSupplier = new Supplier() { private int readState = -1; // Uninitialized value. @@ -420,9 +417,9 @@ public StateSampler.ScopedState get() { public void start( WindmillStateReader stateReader, @Nullable Instant inputDataWatermark, @Nullable Instant outputDataWatermark) { - boolean useStateFamilies = !stateNameMap.isEmpty(); - this.stateInternals = new WindmillStateInternals( - prefix, useStateFamilies, stateReader, scopedReadStateSupplier); + this.stateInternals = new WindmillStateInternals(stateFamily, stateReader, + stateCache.forKey(getSerializedKey(), stateFamily, getWork().getCacheToken()), + scopedReadStateSupplier); this.timerInternals = new WindmillTimerInternals(stateFamily, inputDataWatermark, outputDataWatermark); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SizedWeigher.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/Weighers.java similarity index 61% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SizedWeigher.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/Weighers.java index 9b4b246beac22..5bafca766e2c8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/SizedWeigher.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/Weighers.java @@ -16,8 +16,7 @@ package com.google.cloud.dataflow.sdk.runners.worker; -import com.google.cloud.dataflow.sdk.util.Sized; -import com.google.common.base.Preconditions; +import com.google.cloud.dataflow.sdk.util.Weighted; import com.google.common.cache.Weigher; /** @@ -28,23 +27,22 @@ *

      Package-private here so that the dependency on Guava does not leak into the public API * surface. */ -class SizedWeigher implements Weigher>{ - - public static SizedWeigher withBaseWeight(int baseWeight) { - return new SizedWeigher<>(baseWeight); - } - - private final int baseWeight; - - private SizedWeigher(int baseWeight) { - Preconditions.checkArgument( - baseWeight > 0, - "base weight for SizedWeigher must be positive"); - this.baseWeight = baseWeight; +class Weighers { + public static Weigher fixedWeightKeys(final int keyWeight) { + return new Weigher() { + @Override + public int weigh(Object key, Weighted value) { + return (int) Math.min(keyWeight + value.getWeight(), Integer.MAX_VALUE); + } + }; } - @Override - public int weigh(K key, Sized value) { - return baseWeight + (int) Math.min(value.getSize(), Integer.MAX_VALUE); + public static Weigher weightedKeysAndValues() { + return new Weigher() { + @Override + public int weigh(Weighted key, Weighted value) { + return (int) Math.min(key.getWeight() + value.getWeight(), Integer.MAX_VALUE); + } + }; } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateCache.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateCache.java new file mode 100644 index 0000000000000..5eaefa2445be5 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateCache.java @@ -0,0 +1,292 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.util.Weighted; +import com.google.cloud.dataflow.sdk.util.state.State; +import com.google.cloud.dataflow.sdk.util.state.StateNamespace; +import com.google.cloud.dataflow.sdk.util.state.StateTag; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.RemovalCause; +import com.google.common.cache.RemovalListener; +import com.google.common.cache.RemovalNotification; +import com.google.common.cache.Weigher; +import com.google.protobuf.ByteString; + +import java.io.PrintWriter; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +/** + * Process-wide cache of per-key state. + */ +public class WindmillStateCache { + + private Cache stateCache; + private int weight = 0; + + private static class CacheWeigher implements Weigher { + @Override + public int weigh(Weighted key, Weighted value) { + return (int) Math.max(key.getWeight() + value.getWeight(), Integer.MAX_VALUE); + } + } + + public WindmillStateCache() { + final Weigher weigher = Weighers.weightedKeysAndValues(); + + stateCache = + CacheBuilder.newBuilder() + .maximumWeight(100000000 /* 100 MB */) + .recordStats() + .weigher(weigher) + .removalListener(new RemovalListener() { + @Override + public void onRemoval(RemovalNotification removal) { + if (removal.getCause() != RemovalCause.REPLACED) { + weight -= weigher.weigh(removal.getKey(), removal.getValue()); + } + } + }) + .build(); + } + + public long getWeight() { + return weight; + } + + /** + * Per-computation view of the state cache. + */ + public class ForComputation { + private final String computation; + private ForComputation(String computation) { + this.computation = computation; + } + + /** + * Returns a per-computation, per-key view of the state cache. + */ + public ForKey forKey(ByteString key, String stateFamily, long cacheToken) { + return new ForKey(computation, key, stateFamily, cacheToken); + } + } + + /** + * Per-computation, per-key view of the state cache. + */ + public class ForKey { + private final String computation; + private final ByteString key; + private final String stateFamily; + private final long cacheToken; + + private ForKey(String computation, ByteString key, String stateFamily, long cacheToken) { + this.computation = computation; + this.key = key; + this.stateFamily = stateFamily; + this.cacheToken = cacheToken; + } + + public T get(StateNamespace namespace, StateTag address) { + return WindmillStateCache.this.get( + computation, key, stateFamily, cacheToken, namespace, address); + } + + public void put( + StateNamespace namespace, StateTag address, T value, long weight) { + WindmillStateCache.this.put( + computation, key, stateFamily, cacheToken, namespace, address, value, weight); + } + } + + /** + * Returns a per-computation view of the state cache. + */ + public ForComputation forComputation(String computation) { + return new ForComputation(computation); + } + + private T get(String computation, ByteString processingKey, String stateFamily, + long token, StateNamespace namespace, StateTag address) { + StateId id = new StateId(computation, processingKey, stateFamily, namespace); + StateCacheEntry entry = stateCache.getIfPresent(id); + if (entry == null) { + return null; + } + if (entry.getToken() != token) { + stateCache.invalidate(id); + return null; + } + return entry.get(namespace, address); + } + + private void put(String computation, ByteString processingKey, + String stateFamily, long token, StateNamespace namespace, StateTag address, T value, + long weight) { + StateId id = new StateId(computation, processingKey, stateFamily, namespace); + StateCacheEntry entry = stateCache.getIfPresent(id); + if (entry == null || entry.getToken() != token) { + entry = new StateCacheEntry(token); + this.weight += id.getWeight(); + } + this.weight += entry.put(namespace, address, value, weight); + // Always add back to the cache to update the weight. + stateCache.put(id, entry); + } + + /** + * Struct identifying a cache entry that contains all data for a key and namespace. + */ + private static class StateId implements Weighted { + public final String computation; + public final ByteString processingKey; + public final String stateFamily; + public final Object namespaceKey; + + public StateId(String computation, ByteString processingKey, String stateFamily, + StateNamespace namespace) { + this.computation = computation; + this.processingKey = processingKey; + this.stateFamily = stateFamily; + this.namespaceKey = namespace.getCacheKey(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof StateId) { + StateId otherId = (StateId) other; + return computation.equals(otherId.computation) + && processingKey.equals(otherId.processingKey) + && stateFamily.equals(otherId.stateFamily) + && namespaceKey.equals(otherId.namespaceKey); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hash(computation, processingKey, namespaceKey); + } + + @Override + public long getWeight() { + return processingKey.size(); + } + } + + /** + * Entry in the state cache that stores a map of values and a token representing the + * validity of the values. + */ + private static class StateCacheEntry implements Weighted { + private final long token; + private final Map, WeightedValue> values; + private long weight; + + public StateCacheEntry(long token) { + this.values = new HashMap<>(); + this.token = token; + this.weight = 0; + } + + @SuppressWarnings("unchecked") + public T get(StateNamespace namespace, StateTag tag) { + WeightedValue weightedValue = + (WeightedValue) values.get(new NamespacedTag(namespace, tag)); + return weightedValue == null ? null : weightedValue.value; + } + + public long put( + StateNamespace namespace, StateTag tag, T value, long weight) { + WeightedValue weightedValue = + (WeightedValue) values.get(new NamespacedTag(namespace, tag)); + long weightDelta = 0; + if (weightedValue == null) { + weightedValue = new WeightedValue(); + } else { + weightDelta -= weightedValue.weight; + } + weightedValue.value = value; + weightedValue.weight = weight; + weightDelta += weight; + this.weight += weightDelta; + values.put(new NamespacedTag(namespace, tag), weightedValue); + return weightDelta; + } + + @Override + public long getWeight() { + return weight; + } + + public long getToken() { + return token; + } + + private static class NamespacedTag { + private final StateNamespace namespace; + private final StateTag tag; + NamespacedTag(StateNamespace namespace, StateTag tag) { + this.namespace = namespace; + this.tag = tag; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof NamespacedTag)) { + return false; + } + NamespacedTag that = (NamespacedTag) other; + return namespace.equals(that.namespace) && tag.equals(that.tag); + } + + @Override + public int hashCode() { + return Objects.hash(namespace, tag); + } + } + + private static class WeightedValue { + public long weight = 0; + public T value = null; + } + } + + /** + * Print summary statistics of the cache to the given {@link PrintWriter}. + */ + public void printSummaryHtml(PrintWriter response) { + response.println("Cache Stats:

"); + response.println( + ""); + response.println(""); + response.println(""); + response.println(""); + response.println(""); + response.println("
Hit RatioEvictionsSizeWeight
" + stateCache.stats().hitRate() + "" + stateCache.stats().evictionCount() + "" + stateCache.size() + "" + getWeight() + "

"); + } + + /** + * Print detailed information about the cache to the given {@link PrintWriter}. + */ + public void printDetailedHtml(PrintWriter response) { + response.println("

Cache Information

"); + printSummaryHtml(response); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java index 91dc2236c5622..e6a6cd6dadf35 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternals.java @@ -21,6 +21,7 @@ import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; +import com.google.cloud.dataflow.sdk.util.Weighted; import com.google.cloud.dataflow.sdk.util.common.worker.StateSampler; import com.google.cloud.dataflow.sdk.util.state.BagState; import com.google.cloud.dataflow.sdk.util.state.CombiningValueStateInternal; @@ -32,128 +33,118 @@ import com.google.cloud.dataflow.sdk.util.state.StateTable; import com.google.cloud.dataflow.sdk.util.state.StateTag; import com.google.cloud.dataflow.sdk.util.state.StateTag.StateBinder; +import com.google.cloud.dataflow.sdk.util.state.StateTags; import com.google.cloud.dataflow.sdk.util.state.ValueState; import com.google.cloud.dataflow.sdk.util.state.WatermarkStateInternal; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; +import com.google.common.base.Optional; import com.google.common.base.Supplier; +import com.google.common.base.Throwables; import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; import com.google.common.util.concurrent.Futures; import com.google.protobuf.ByteString; import org.joda.time.Instant; import java.io.IOException; +import java.io.OutputStreamWriter; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Random; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import javax.annotation.concurrent.NotThreadSafe; /** * Implementation of {@link StateInternals} using Windmill to manage the underlying data. */ class WindmillStateInternals extends MergingStateInternals { + private static class CachingStateTable extends StateTable { + private final String stateFamily; + private final WindmillStateReader reader; + private final WindmillStateCache.ForKey cache; + private final Supplier scopedReadStateSupplier; - private final StateTable inMemoryState = - new StateTable() { - @Override - protected StateBinder binderForNamespace(final StateNamespace namespace) { - return new StateBinder() { - @Override - public BagState bindBag(StateTag> address, Coder elemCoder) { - return new WindmillBag<>(encodeKey(namespace, address), stateFamily, elemCoder, - reader, scopedReadStateSupplier); - } - - @Override - public WatermarkStateInternal bindWatermark( - StateTag address, - OutputTimeFn outputTimeFn) { - return new WindmillWatermarkState( - encodeKey(namespace, address), - stateFamily, - reader, - scopedReadStateSupplier, - outputTimeFn); - } - - @Override - public - CombiningValueStateInternal bindCombiningValue( - StateTag> address, - Coder accumCoder, - CombineFn combineFn) { - return new WindmillCombiningValue<>(encodeKey(namespace, address), stateFamily, - accumCoder, combineFn, reader, scopedReadStateSupplier); - } + public CachingStateTable(String stateFamily, + WindmillStateReader reader, WindmillStateCache.ForKey cache, + Supplier scopedReadStateSupplier) { + this.stateFamily = stateFamily; + this.reader = reader; + this.cache = cache; + this.scopedReadStateSupplier = scopedReadStateSupplier; + } - @Override - public ValueState bindValue(StateTag> address, Coder coder) { - return new WindmillValue<>(encodeKey(namespace, address), stateFamily, coder, reader, - scopedReadStateSupplier); - } - }; + @Override + protected StateBinder binderForNamespace(final StateNamespace namespace) { + // Look up state objects in the cache or create new ones if not found. The state will + // be added to the cache in persist(). + return new StateBinder() { + @Override + public BagState bindBag(StateTag> address, Coder elemCoder) { + WindmillBag result = (WindmillBag) cache.get(namespace, address); + if (result == null) { + result = new WindmillBag(namespace, address, stateFamily, elemCoder); + } + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; } - }; - private final String prefix; - private final String stateFamily; - private final WindmillStateReader reader; - private final boolean useStateFamilies; - private final Supplier scopedReadStateSupplier; - - @VisibleForTesting - static final ThreadLocal> COMPACT_NOW = - new ThreadLocal>() { @Override - public Supplier initialValue() { - return new Supplier() { - /* The rate at which, on average, this will return true. */ - static final double RATE = 0.002; - Random random = new Random(); - long counter = nextSample(); + public WatermarkStateInternal bindWatermark( + StateTag address, OutputTimeFn outputTimeFn) { + WindmillWatermarkState result = (WindmillWatermarkState) cache.get(namespace, address); + if (result == null) { + result = new WindmillWatermarkState(namespace, address, stateFamily, outputTimeFn); + } + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } - private long nextSample() { - // Use geometric distribution to find next true value. - // This lets us avoid invoking random.nextDouble() on every call. - return (long) Math.floor(Math.log(random.nextDouble()) / Math.log(1 - RATE)); - } + @Override + public CombiningValueStateInternal + bindCombiningValue(StateTag> address, + Coder accumCoder, CombineFn combineFn) { + WindmillCombiningValue result = new WindmillCombiningValue<>( + namespace, address, stateFamily, accumCoder, combineFn, cache); + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } - @Override - public Boolean get() { - counter--; - if (counter < 0) { - counter = nextSample(); - return true; - } else { - return false; - } - } - }; + @Override + public ValueState bindValue(StateTag> address, Coder coder) { + WindmillValue result = (WindmillValue) cache.get(namespace, address); + if (result == null) { + result = new WindmillValue(namespace, address, stateFamily, coder); + } + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; } }; - - public WindmillStateInternals(String prefix, boolean useStateFamilies, - WindmillStateReader reader, Supplier scopedReadStateSupplier) { - this.prefix = prefix; - if (useStateFamilies) { - this.stateFamily = prefix; - } else { - this.stateFamily = ""; } - this.reader = reader; - this.useStateFamilies = useStateFamilies; + }; + + private WindmillStateCache.ForKey cache; + Supplier scopedReadStateSupplier; + private StateTable workItemState; + + public WindmillStateInternals(String stateFamily, WindmillStateReader reader, + WindmillStateCache.ForKey cache, Supplier scopedReadStateSupplier) { + this.cache = cache; this.scopedReadStateSupplier = scopedReadStateSupplier; + this.workItemState = new CachingStateTable(stateFamily, reader, cache, scopedReadStateSupplier); } public void persist(final Windmill.WorkItemCommitRequest.Builder commitBuilder) { List> commitsToMerge = new ArrayList<>(); // Call persist on each first, which may schedule some futures for reading. - for (State location : inMemoryState.values()) { + for (State location : workItemState.values()) { if (!(location instanceof WindmillState)) { throw new IllegalStateException(String.format( "%s wasn't created by %s -- unable to persist it", @@ -162,20 +153,16 @@ public void persist(final Windmill.WorkItemCommitRequest.Builder commitBuilder) } try { - commitsToMerge.add(((WindmillState) location).persist()); + commitsToMerge.add(((WindmillState) location).persist(cache)); } catch (IOException e) { throw new RuntimeException("Unable to persist state", e); } } - // Kick off the fetches that prevent blind-writes. We do this before returning - // to ensure that the reads have happened before the persist actually happens. - reader.startBatchAndBlock(); - // Clear out the map of already retrieved state instances. - inMemoryState.clear(); + workItemState.clear(); - try { + try (StateSampler.ScopedState scope = scopedReadStateSupplier.get()) { for (Future commitFuture : commitsToMerge) { commitBuilder.mergeFrom(commitFuture.get()); } @@ -184,103 +171,123 @@ public void persist(final Windmill.WorkItemCommitRequest.Builder commitBuilder) } } - @VisibleForTesting ByteString encodeKey(StateNamespace namespace, StateTag address) { + /** + * Encodes the given namespace and address as {@code <namespace>+<address>}. + */ + @VisibleForTesting + static ByteString encodeKey(StateNamespace namespace, StateTag address) { try { - // Use a StringBuilder rather than concatenation and String.format. We build these keys + // Use ByteString.Output rather than concatenation and String.format. We build these keys // a lot, and this leads to better performance results. See associated benchmarks. - StringBuilder output = new StringBuilder(); - - // We only need the prefix if we aren't using state families - if (!useStateFamilies) { - output.append(prefix); - } + ByteString.Output stream = ByteString.newOutput(); + OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8); - // stringKey starts and ends with a slash. We don't need to seperate it from prefix, because - // the prefix is guaranteed to be unique and non-overlapping. We separate it from the + // stringKey starts and ends with a slash. We separate it from the // StateTag ID by a '+' (which is guaranteed not to be in the stringKey) because the // ID comes from the user. - namespace.appendTo(output); - output.append('+'); - address.appendTo(output); - return ByteString.copyFromUtf8(output.toString()); + namespace.appendTo(writer); + writer.write('+'); + address.appendTo(writer); + writer.flush(); + return stream.toByteString(); } catch (IOException e) { - throw new RuntimeException( - "Unable to encode state key for " + namespace + ", " + address, e); + throw Throwables.propagate(e); } } /** - * Anything that can provide a {@link WorkItemCommitRequest} to persist its state; it may need - * to read some state in order to build the commit request. + * Abstract base class for all Windmill state. + * + *

Note that these are not thread safe; each state object is associated with a key + * and thus only accessed by a single thread at once. */ - private interface WindmillState { + @NotThreadSafe + private abstract static class WindmillState { + protected Supplier scopedReadStateSupplier; + protected WindmillStateReader reader; + /** * Return an asynchronously computed {@link WorkItemCommitRequest}. The request should * be of a form that can be merged with others (only add to repeated fields). */ - Future persist() + abstract Future persist(WindmillStateCache.ForKey cache) throws IOException; + + void initializeForWorkItem( + WindmillStateReader reader, Supplier scopedReadStateSupplier) { + this.reader = reader; + this.scopedReadStateSupplier = scopedReadStateSupplier; + } + + StateSampler.ScopedState scopedReadState() { + return scopedReadStateSupplier.get(); + } } /** * Base class for implementations of {@link WindmillState} where the {@link #persist} call does * not require any asynchronous reading. */ - private abstract static class SimpleWindmillState implements WindmillState { + private abstract static class SimpleWindmillState extends WindmillState { @Override - public final Future persist() throws IOException{ - return Futures.immediateFuture(persistDirectly()); + public final Future persist(WindmillStateCache.ForKey cache) + throws IOException { + return Futures.immediateFuture(persistDirectly(cache)); } /** * Returns a {@link WorkItemCommitRequest} that can be used to persist this state to * Windmill. */ - protected abstract WorkItemCommitRequest persistDirectly() throws IOException; + protected abstract WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKey cache) + throws IOException; } @Override public T state(StateNamespace namespace, StateTag address) { - return inMemoryState.get(namespace, address); + return workItemState.get(namespace, address); } - private static class WindmillValue extends SimpleWindmillState - implements ValueState, WindmillState { - + private static class WindmillValue extends SimpleWindmillState implements ValueState { + private final StateNamespace namespace; + private final StateTag> address; private final ByteString stateKey; private final String stateFamily; private final Coder coder; - private final WindmillStateReader reader; - private final Supplier readStateSupplier; /** Whether we've modified the value since creation of this state. */ private boolean modified = false; - private T modifiedValue; - - private WindmillValue(ByteString stateKey, String stateFamily, Coder coder, - WindmillStateReader reader, Supplier readStateSupplier) { - this.stateKey = stateKey; + /** Whether the in memory value is the true value. */ + private boolean valueIsKnown = false; + private T value; + + private WindmillValue(StateNamespace namespace, StateTag> address, + String stateFamily, Coder coder) { + this.namespace = namespace; + this.address = address; + this.stateKey = encodeKey(namespace, address); this.stateFamily = stateFamily; this.coder = coder; - this.reader = reader; - this.readStateSupplier = readStateSupplier; } @Override public void clear() { modified = true; - modifiedValue = null; + valueIsKnown = true; + value = null; } @Override public StateContents get() { - final Future future = modified ? null : reader.valueFuture(stateKey, stateFamily, coder); + final Future future = valueIsKnown ? Futures.immediateFuture(value) + : reader.valueFuture(stateKey, stateFamily, coder); return new StateContents() { @Override public T read() { - try (StateSampler.ScopedState scope = readStateSupplier.get()) { - return modified ? modifiedValue : future.get(); + try (StateSampler.ScopedState scope = scopedReadState()) { + valueIsKnown = true; + return future.get(); } catch (InterruptedException | ExecutionException e) { throw new RuntimeException("Unable to read value from state", e); } @@ -291,63 +298,96 @@ public T read() { @Override public void set(T value) { modified = true; - modifiedValue = value; + valueIsKnown = true; + this.value = value; } @Override - protected WorkItemCommitRequest persistDirectly() throws IOException { + protected WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKey cache) + throws IOException { if (!modified) { // No in-memory changes. return WorkItemCommitRequest.newBuilder().buildPartial(); } - // We can't write without doing a read, so we need to kick off a read if we get here. - // Call reader.valueFuture directly, since our read() method will avoid actually reading from - // Windmill since the value is already inMemory. - reader.valueFuture(stateKey, stateFamily, coder); - ByteString.Output stream = ByteString.newOutput(); - if (modifiedValue != null) { - coder.encode(modifiedValue, stream, Coder.Context.OUTER); + if (value != null) { + coder.encode(value, stream, Coder.Context.OUTER); } + ByteString encoded = stream.toByteString(); WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); + // Update the entry of the cache with the new value and change in encoded size. + cache.put(namespace, address, this, encoded.size()); + + modified = false; + commitBuilder .addValueUpdatesBuilder() .setTag(stateKey) .setStateFamily(stateFamily) .getValueBuilder() - .setData(stream.toByteString()) + .setData(encoded) .setTimestamp(Long.MAX_VALUE); + return commitBuilder.buildPartial(); } } - private static class WindmillBag extends SimpleWindmillState - implements BagState, WindmillState { + private static class WindmillBag extends SimpleWindmillState implements BagState { + private final StateNamespace namespace; + private final StateTag> address; private final ByteString stateKey; private final String stateFamily; private final Coder elemCoder; - private final WindmillStateReader reader; - private final Supplier readStateSupplier; - - private boolean cleared = false; - private final List localAdditions = new ArrayList<>(); - private WindmillBag(ByteString stateKey, String stateFamily, Coder elemCoder, - WindmillStateReader reader, Supplier readStateSupplier) { - this.stateKey = stateKey; + private boolean cleared; + // Cache of all values in this bag. Null if the persisted state is unknown. + private ConcatIterables cachedValues = null; + private List localAdditions = new ArrayList<>(); + private long encodedSize = 0; + + private WindmillBag(StateNamespace namespace, StateTag> address, String stateFamily, + Coder elemCoder) { + this.namespace = namespace; + this.address = address; + this.stateKey = encodeKey(namespace, address); this.stateFamily = stateFamily; this.elemCoder = elemCoder; - this.reader = reader; - this.readStateSupplier = readStateSupplier; } @Override public void clear() { cleared = true; + cachedValues = new ConcatIterables(); localAdditions.clear(); + encodedSize = 0; + } + + private Iterable fetchData(Future> persistedData) { + try (StateSampler.ScopedState scope = scopedReadState()) { + if (cachedValues != null) { + return cachedValues; + } + Iterable data = persistedData.get(); + if (data instanceof Weighted) { + // We have a known bounded amount of data; cache it. + cachedValues = new ConcatIterables(); + cachedValues.extendWith(data); + encodedSize = ((Weighted) data).getWeight(); + return cachedValues; + } else { + // This is an iterable that may not fit in memory at once; don't cache it. + return data; + } + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Unable to read state", e); + } + } + + public boolean valuesAreCached() { + return cachedValues != null; } @Override @@ -355,21 +395,14 @@ public StateContents> get() { // If we clear after calling get() but before calling read(), technically we didn't need the // underlying windmill read. But, we need to register the desire now if we aren't going to // clear (in order to get it added to the prefetch). - final Future> persistedData = cleared - ? Futures.>immediateFuture(Collections.emptyList()) + final Future> persistedData = (cachedValues != null) + ? null : reader.listFuture(stateKey, stateFamily, elemCoder); return new StateContents>() { @Override public Iterable read() { - try (StateSampler.ScopedState scope = readStateSupplier.get()) { - // We need to check cleared again, because it may have become clear in between creating - // the future and calling read. - Iterable input = cleared ? Collections.emptyList() : persistedData.get(); - return Iterables.concat(input, localAdditions); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException("Unable to read state", e); - } + return Iterables.concat(fetchData(persistedData), localAdditions); } }; } @@ -379,21 +412,14 @@ public StateContents isEmpty() { // If we clear after calling isEmpty() but before calling read(), technically we didn't need // the underlying windmill read. But, we need to register the desire now if we aren't going to // clear (in order to get it added to the prefetch). - final Future> persistedData = cleared - ? Futures.>immediateFuture(Collections.emptyList()) + final Future> persistedData = (cachedValues != null) + ? null : reader.listFuture(stateKey, stateFamily, elemCoder); return new StateContents() { @Override public Boolean read() { - try (StateSampler.ScopedState scope = readStateSupplier.get()) { - // We need to check cleared again, because it may have become clear in between creating - // the future and calling read. - Iterable input = cleared ? Collections.emptyList() : persistedData.get(); - return Iterables.isEmpty(input) && Iterables.isEmpty(localAdditions); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException("Unable to read state", e); - } + return Iterables.isEmpty(fetchData(persistedData)) && localAdditions.isEmpty(); } }; } @@ -404,14 +430,11 @@ public void add(T input) { } @Override - public WorkItemCommitRequest persistDirectly() throws IOException { + public WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKey cache) + throws IOException { WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); if (cleared) { - // If we do a delete, we need to have done a read to prevent Windmill complaining about - // blind deletes. We use the underlying reader, because we normally skip the actual read - // if we've already cleared the state. - reader.listFuture(stateKey, stateFamily, elemCoder); commitBuilder.addListUpdatesBuilder() .setTag(stateKey) .setStateFamily(stateFamily) @@ -430,43 +453,88 @@ public WorkItemCommitRequest persistDirectly() throws IOException { // Encode the value elemCoder.encode(value, stream, Coder.Context.OUTER); + ByteString encoded = stream.toByteString(); + if (cachedValues != null) { + encodedSize += encoded.size() - 1; + } listUpdatesBuilder.addValuesBuilder() - .setData(stream.toByteString()) + .setData(encoded) .setTimestamp(Long.MAX_VALUE); } } + + if (cachedValues != null) { + cachedValues.extendWith(localAdditions); + // Don't reuse the localAdditions object; we don't want future changes to it to modify the + // value of cachedValues. + localAdditions = new ArrayList(); + cache.put(namespace, address, this, encodedSize); + } else { + localAdditions.clear(); + } + cleared = false; + return commitBuilder.buildPartial(); } } - private static class WindmillWatermarkState implements WatermarkStateInternal, WindmillState { + private static class ConcatIterables implements Iterable { + List> iterables; + + public ConcatIterables() { + this.iterables = new ArrayList<>(); + } + + public void extendWith(Iterable iterable) { + iterables.add(iterable); + } + + @Override + public Iterator iterator() { + return Iterators.concat( + Iterables.transform( + iterables, + new Function, Iterator>() { + @Override + public Iterator apply(Iterable iterable) { + return iterable.iterator(); + } + }) + .iterator()); + } + } + + private static class WindmillWatermarkState + extends WindmillState implements WatermarkStateInternal { + // The encoded size of an Instant. + private static final int ENCODED_SIZE = 8; private final OutputTimeFn outputTimeFn; + private final StateNamespace namespace; + private final StateTag address; private final ByteString stateKey; private final String stateFamily; - private final WindmillStateReader reader; - private final Supplier readStateSupplier; private boolean cleared = false; + // The hold value, Optional.absent() if no hold, or null if unknown. + private Optional cachedValue = null; private Instant localAdditions = null; - private WindmillWatermarkState( - ByteString stateKey, - String stateFamily, - WindmillStateReader reader, - Supplier readStateSupplier, + private WindmillWatermarkState(StateNamespace namespace, + StateTag address, String stateFamily, OutputTimeFn outputTimeFn) { - this.stateKey = stateKey; + this.namespace = namespace; + this.address = address; + this.stateKey = encodeKey(namespace, address); this.stateFamily = stateFamily; - this.reader = reader; - this.readStateSupplier = readStateSupplier; this.outputTimeFn = outputTimeFn; } @Override public void clear() { cleared = true; + cachedValue = Optional.absent(); localAdditions = null; } @@ -484,46 +552,38 @@ public StateContents get() { // If we clear after calling get() but before calling read(), technically we didn't need the // underlying windmill read. But, we need to register the desire now if we aren't going to // clear (in order to get it added to the prefetch). - final Future persistedData = cleared - ? Futures.immediateFuture(null) + final Future persistedData = (cachedValue != null) + ? Futures.immediateFuture(cachedValue.orNull()) : reader.watermarkFuture(stateKey, stateFamily); return new StateContents() { @Override public Instant read() { - Instant value = localAdditions; - if (!cleared) { - try (StateSampler.ScopedState scope = readStateSupplier.get()) { - Instant persisted = persistedData.get(); - value = (value == null) ? persisted : outputTimeFn.combine(value, persisted); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException("Unable to read state", e); + try (StateSampler.ScopedState scope = scopedReadState()) { + Instant persistedHold = persistedData.get(); + if (persistedHold == null || persistedHold.equals(BoundedWindow.TIMESTAMP_MAX_VALUE)) { + cachedValue = Optional.absent(); + } else { + cachedValue = Optional.of(persistedHold); } + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Unable to read state", e); + } + + if (localAdditions == null) { + return cachedValue.orNull(); + } else if (!cachedValue.isPresent()) { + return localAdditions; + } else { + return outputTimeFn.combine(localAdditions, cachedValue.get()); } - return value; } }; } @Override public StateContents isEmpty() { - // If we clear after calling get() but before calling read(), technically we didn't need the - // underlying windmill read. But, we need to register the desire now if we aren't going to - // clear (in order to get it added to the prefetch). - final Future persistedData = cleared - ? Futures.immediateFuture(null) - : reader.watermarkFuture(stateKey, stateFamily); - - return new StateContents() { - @Override - public Boolean read() { - try (StateSampler.ScopedState scope = readStateSupplier.get()) { - return localAdditions == null && (cleared || persistedData.get() == null); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException("Unable to read state", e); - } - } - }; + throw new UnsupportedOperationException(); } @Override @@ -533,7 +593,9 @@ public void add(Instant outputTime) { } @Override - public Future persist() { + public Future persist(final WindmillStateCache.ForKey cache) { + Future result; + if (!cleared && localAdditions == null) { // Nothing to do return Futures.immediateFuture(WorkItemCommitRequest.newBuilder().buildPartial()); @@ -544,7 +606,8 @@ public Future persist() { .setTag(stateKey) .setStateFamily(stateFamily) .setReset(true); - return Futures.immediateFuture(commitBuilder.buildPartial()); + + result = Futures.immediateFuture(commitBuilder.buildPartial()); } else if (cleared && localAdditions != null) { // Since we cleared before adding, we can do a blind overwrite of persisted state WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); @@ -553,13 +616,30 @@ public Future persist() { .setStateFamily(stateFamily) .setReset(true) .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); - return Futures.immediateFuture(commitBuilder.buildPartial()); - } else if (!cleared && localAdditions != null){ + + cachedValue = Optional.of(localAdditions); + + result = Futures.immediateFuture(commitBuilder.buildPartial()); + } else if (!cleared && localAdditions != null) { // Otherwise, we need to combine the local additions with the already persisted data - return combineWithPersisted(); + result = combineWithPersisted(); } else { throw new IllegalStateException("Unreachable condition"); } + + return Futures.lazyTransform( + result, new Function() { + @Override + public WorkItemCommitRequest apply(WorkItemCommitRequest result) { + cleared = false; + localAdditions = null; + if (cachedValue != null) { + cache.put( + namespace, address, WindmillWatermarkState.this, ENCODED_SIZE); + } + return result; + } + }); } /** @@ -589,35 +669,42 @@ private Future combineWithPersisted() { .setStateFamily(stateFamily) .addTimestamps( WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); - return Futures.immediateFuture(commitBuilder.buildPartial()); + + if (cachedValue != null) { + cachedValue = Optional.of(cachedValue.isPresent() + ? outputTimeFn.combine(cachedValue.get(), localAdditions) + : localAdditions); + } + + return Futures.immediateFuture(commitBuilder.buildPartial()); } else { // The non-fast path does a read-modify-write - return Futures.lazyTransform(reader.watermarkFuture(stateKey, stateFamily), + return Futures.lazyTransform((cachedValue != null) + ? Futures.immediateFuture(cachedValue.orNull()) + : reader.watermarkFuture(stateKey, stateFamily), new Function() { - - @Override - public WorkItemCommitRequest apply(Instant priorHold) { - - Instant combinedHold = (priorHold == null) ? localAdditions - : outputTimeFn.combine(priorHold, localAdditions); - - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - commitBuilder.addWatermarkHoldsBuilder() - .setTag(stateKey) - .setStateFamily(stateFamily) - .setReset(true) - .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(combinedHold)); - - return commitBuilder.buildPartial(); - } - }); + @Override + public WorkItemCommitRequest apply(Instant priorHold) { + cachedValue = Optional.of((priorHold != null) + ? outputTimeFn.combine(priorHold, localAdditions) + : localAdditions); + + WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); + commitBuilder.addWatermarkHoldsBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .setReset(true) + .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(cachedValue.get())); + + return commitBuilder.buildPartial(); + } + }); } } } private static class WindmillCombiningValue - implements CombiningValueStateInternal, WindmillState { - + extends WindmillState implements CombiningValueStateInternal { private final WindmillBag bag; private final CombineFn combineFn; @@ -628,15 +715,28 @@ private static class WindmillCombiningValue private AccumT localAdditionsAccum; private boolean hasLocalAdditions = false; - private WindmillCombiningValue(ByteString stateKey, String stateFamily, - Coder accumCoder, - CombineFn combineFn, - WindmillStateReader reader, Supplier readStateSupplier) { - this.bag = new WindmillBag<>(stateKey, stateFamily, accumCoder, reader, readStateSupplier); + private WindmillCombiningValue(StateNamespace namespace, + StateTag> address, String stateFamily, + Coder accumCoder, CombineFn combineFn, + WindmillStateCache.ForKey cache) { + StateTag> internalBagAddress = StateTags.bag(address.getId(), accumCoder); + WindmillBag cachedBag = + (WindmillBag) cache.get(namespace, internalBagAddress); + this.bag = + (cachedBag != null) + ? cachedBag + : new WindmillBag<>(namespace, internalBagAddress, stateFamily, accumCoder); this.combineFn = combineFn; this.localAdditionsAccum = combineFn.createAccumulator(); } + @Override + void initializeForWorkItem( + WindmillStateReader reader, Supplier scopedReadStateSupplier) { + super.initializeForWorkItem(reader, scopedReadStateSupplier); + this.bag.initializeForWorkItem(reader, scopedReadStateSupplier); + } + @Override public StateContents get() { final StateContents accum = getAccum(); @@ -662,10 +762,10 @@ public void clear() { } @Override - public Future persist() throws IOException { + public Future persist(WindmillStateCache.ForKey cache) + throws IOException { if (hasLocalAdditions) { - // TODO: Take into account whether it's in the cache. - if (COMPACT_NOW.get().get()) { + if (COMPACT_NOW.get().get() || bag.valuesAreCached()) { // Implicitly clears the bag and combines local and persisted accumulators. localAdditionsAccum = getAccum().read(); } @@ -673,7 +773,8 @@ public Future persist() throws IOException { localAdditionsAccum = combineFn.createAccumulator(); hasLocalAdditions = false; } - return bag.persist(); + + return bag.persist(cache); } @Override @@ -715,4 +816,33 @@ public void addAccum(AccumT accum) { localAdditionsAccum = combineFn.mergeAccumulators(Arrays.asList(localAdditionsAccum, accum)); } } + + @VisibleForTesting + static final ThreadLocal> COMPACT_NOW = + new ThreadLocal>() { + public Supplier initialValue() { + return new Supplier() { + /* The rate at which, on average, this will return true. */ + static final double RATE = 0.002; + Random random = new Random(); + long counter = nextSample(); + + private long nextSample() { + // Use geometric distribution to find next true value. + // This lets us avoid invoking random.nextDouble() on every call. + return (long) Math.floor(Math.log(random.nextDouble()) / Math.log(1 - RATE)); + } + + public Boolean get() { + counter--; + if (counter < 0) { + counter = nextSample(); + return true; + } else { + return false; + } + } + }; + } + }; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateReader.java index fb0421a703757..af7a64e4784d9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateReader.java @@ -19,11 +19,13 @@ import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill; import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill.TagList; import com.google.cloud.dataflow.sdk.runners.worker.windmill.Windmill.TagValue; +import com.google.cloud.dataflow.sdk.util.Weighted; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.collect.AbstractIterator; +import com.google.common.collect.ForwardingList; import com.google.common.util.concurrent.ForwardingFuture; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.SettableFuture; @@ -474,28 +476,60 @@ private void consumeResponse(Windmill.GetDataRequest request, } } + @VisibleForTesting + static class WeightedList extends ForwardingList implements Weighted { + private List delegate; + long weight; + + WeightedList(List delegate) { + this.delegate = delegate; + this.weight = 0; + } + + @Override + protected List delegate() { + return delegate; + } + + @Override + public boolean add(T elem) { + throw new UnsupportedOperationException("Must use AddWeighted()"); + } + + @Override + public long getWeight() { + return weight; + } + + public void addWeighted(T elem, long weight) { + delegate.add(elem); + this.weight += weight; + } + } + /** * The deserialized values in {@code tagList} as a read-only array list. */ private List tagListPageValues(TagList tagList, Coder elemCoder) { if (tagList.getValuesCount() == 0) { - return Collections.emptyList(); + return new WeightedList(Collections.emptyList()); } - List valueList = new ArrayList<>(tagList.getValuesCount()); + WeightedList valueList = new WeightedList<>(new ArrayList(tagList.getValuesCount())); for (Windmill.Value value : tagList.getValuesList()) { if (value.hasData() && !value.getData().isEmpty()) { // Drop the first byte of the data; it's the zero byte we prepended to avoid writing // empty data. InputStream inputStream = value.getData().substring(1).newInput(); try { - valueList.add(elemCoder.decode(inputStream, Coder.Context.OUTER)); + valueList.addWeighted( + elemCoder.decode(inputStream, Coder.Context.OUTER), value.getData().size() - 1); } catch (IOException e) { throw new IllegalStateException("Unable to decode tag list using " + elemCoder, e); } } } - return Collections.unmodifiableList(valueList); + return valueList; } private void consumeTagList(TagList tagList, StateTag stateTag) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java index f1b607c182a47..57dd51009b8f1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java @@ -24,7 +24,7 @@ import com.google.cloud.dataflow.sdk.coders.ListCoder; import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn; import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator; -import com.google.cloud.dataflow.sdk.util.Sized; +import com.google.cloud.dataflow.sdk.util.WeightedValue; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -550,30 +550,30 @@ private long offset(long newWeight) { */ private List interpolate(Iterable> buffers, int count, double step, double offset) { - List>> iterators = Lists.newArrayList(); + List>> iterators = Lists.newArrayList(); for (QuantileBuffer buffer : buffers) { iterators.add(buffer.sizedIterator()); } // Each of the buffers is already sorted by element. - Iterator> sorted = Iterators.mergeSorted( + Iterator> sorted = Iterators.mergeSorted( iterators, - new Comparator>() { + new Comparator>() { @Override - public int compare(Sized a, Sized b) { + public int compare(WeightedValue a, WeightedValue b) { return compareFn.compare(a.getValue(), b.getValue()); } }); List newElements = Lists.newArrayListWithCapacity(count); - Sized sizedElement = sorted.next(); - double current = sizedElement.getSize(); + WeightedValue weightedElement = sorted.next(); + double current = weightedElement.getWeight(); for (int j = 0; j < count; j++) { double target = j * step + offset; while (current <= target && sorted.hasNext()) { - sizedElement = sorted.next(); - current += sizedElement.getSize(); + weightedElement = sorted.next(); + current += weightedElement.getWeight(); } - newElements.add(sizedElement.getValue()); + newElements.add(weightedElement.getValue()); } return newElements; } @@ -638,15 +638,15 @@ public String toString() { + weight + ", elements=" + elements + "]"; } - public Iterator> sizedIterator() { - return new UnmodifiableIterator>() { + public Iterator> sizedIterator() { + return new UnmodifiableIterator>() { Iterator iter = elements.iterator(); @Override public boolean hasNext() { return iter.hasNext(); } - @Override public Sized next() { - return Sized.of(iter.next(), weight); + @Override public WeightedValue next() { + return WeightedValue.of(iter.next(), weight); } }; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Weighted.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Weighted.java new file mode 100644 index 0000000000000..c31ad7f861c45 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Weighted.java @@ -0,0 +1,27 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +/** + * Interface representing an object that has a weight, in unspecified units. + */ +public interface Weighted { + /** + * Returns the weight of the object. + */ + long getWeight(); +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SizedSideInputReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WeightedSideInputReader.java similarity index 73% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SizedSideInputReader.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WeightedSideInputReader.java index 9cc2140d6970b..0323f2cafdab9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SizedSideInputReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WeightedSideInputReader.java @@ -22,27 +22,27 @@ /** * Extension to {@link SideInputReader} that can approximate the size of the side input. */ -public interface SizedSideInputReader extends SideInputReader { +public interface WeightedSideInputReader extends SideInputReader { /** * Returns the value of the requested {@link PCollectionView} for the given {@link BoundedWindow} * along with a rough estimate of the number of bytes of memory it consumes. * *

It is valid for a side input value to be {@code null}. In this case, the return - * value of this method must still be non-{@code null}. It should be a {@link Sized} - * object where {@link Sized#getValue()} returns {@code null} and {@link Sized#getSize()} may - * still return any non-negative value. + * value of this method must still be non-{@code null}. It should be a {@link Weighted} + * object where {@link WeightedValue#getValue()} returns {@code null} and + * {@link WeightedValue#getWeight()} may still return any non-negative value. */ - Sized getSized(PCollectionView view, BoundedWindow window); + WeightedValue getWeighted(PCollectionView view, BoundedWindow window); /** - * Abstract class providing default implementations for methods of {@link SizedSideInputReader}. + * Abstract class providing default implementations for methods of + * {@link WeightedSideInputReader}. */ - abstract static class Defaults implements SizedSideInputReader { + abstract static class Defaults implements WeightedSideInputReader { @Override public T get(PCollectionView view, BoundedWindow window) { - return getSized(view, window).getValue(); + return getWeighted(view, window).getValue(); } } } - diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Sized.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WeightedValue.java similarity index 67% rename from sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Sized.java rename to sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WeightedValue.java index f019ecca7a5d6..4a6e84079faa1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Sized.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WeightedValue.java @@ -17,30 +17,29 @@ package com.google.cloud.dataflow.sdk.util; /** - * A {@code T} with an accompanying size estimate. Units are unspecified. + * A {@code T} with an accompanying weight. Units are unspecified. * * @param the underlying type of object */ -public final class Sized { +public final class WeightedValue implements Weighted { private final T value; - private final long size; + private final long weight; - private Sized(T value, long size) { + private WeightedValue(T value, long weight) { this.value = value; - this.size = size; + this.weight = weight; } - public static Sized of(T value, long size) { - return new Sized<>(value, size); + public static WeightedValue of(T value, long weight) { + return new WeightedValue<>(value, weight); } - public long getSize() { - return size; + public long getWeight() { + return weight; } public T getValue() { return value; } } - diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternals.java index 1af60ad209461..9a28d040c4d91 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternals.java @@ -63,7 +63,7 @@ public BagState bindBag(final StateTag> address, Coder ele } @Override - public WatermarkStateInternal bindWatermark( + public WatermarkStateInternal bindWatermark( StateTag address, OutputTimeFn outputTimeFn) { return new WatermarkStateInternalImplementation(outputTimeFn); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergingStateInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergingStateInternals.java index c6a7ae17e3c56..11a73ad7d2ea9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergingStateInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergingStateInternals.java @@ -80,7 +80,7 @@ CombiningValueStateInternal bindCombiningValue( } @Override - public WatermarkStateInternal bindWatermark( + public WatermarkStateInternal bindWatermark( StateTag address, OutputTimeFn outputTimeFn) { List sources = new ArrayList<>(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespace.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespace.java index 4a0364f1db88e..f972e312f9eec 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespace.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespace.java @@ -42,4 +42,13 @@ public interface StateNamespace { * Append the string representation of this key to the {@link Appendable}. */ void appendTo(Appendable sb) throws IOException; + + /** + * Return an {@code Object} to use as a key in a cache. + * + *

Different namespaces may use the same key in order to be treated as a unit in the cache. + * The {@code Object}'s {@code hashCode} and {@code equals} methods will be used to determine + * equality. + */ + Object getCacheKey(); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaceForTest.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaceForTest.java index c11668fd62aef..09b86d67e9bfd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaceForTest.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaceForTest.java @@ -33,6 +33,11 @@ public String stringKey() { return key; } + @Override + public Object getCacheKey() { + return key; + } + @Override public boolean equals(Object obj) { if (this == obj) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaces.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaces.java index 22115847a3f65..8fee9959b944e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaces.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateNamespaces.java @@ -63,6 +63,11 @@ public String stringKey() { return GLOBAL_STRING; } + @Override + public Object getCacheKey() { + return GLOBAL_STRING; + } + @Override public boolean equals(Object obj) { return obj == this || obj instanceof GlobalNamespace; @@ -117,6 +122,14 @@ public void appendTo(Appendable sb) throws IOException { sb.append('/').append(CoderUtils.encodeToBase64(windowCoder, window)).append('/'); } + /** + * State in the same window will all be evicted together. + */ + @Override + public Object getCacheKey() { + return window; + } + @Override public boolean equals(Object obj) { if (obj == this) { @@ -189,6 +202,14 @@ public void appendTo(Appendable sb) throws IOException { sb.append('/'); } + /** + * State in the same window will all be evicted together. + */ + @Override + public Object getCacheKey() { + return window; + } + @Override public boolean equals(Object obj) { if (obj == this) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTag.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTag.java index cba405d81e641..be114f893fc0d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTag.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTag.java @@ -59,7 +59,7 @@ public interface StateBinder { *

This accepts the {@link OutputTimeFn} that dictates how watermark hold timestamps * added to the returned {@link WatermarkStateInternal} are to be combined. */ - WatermarkStateInternal bindWatermark( + WatermarkStateInternal bindWatermark( StateTag address, OutputTimeFn outputTimeFn); } @@ -68,7 +68,7 @@ WatermarkStateInternal bindWatermark( void appendTo(Appendable sb) throws IOException; /** - * Returns the identifier for this state cell. + * Returns the user-provided name of this state cell. */ String getId(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java index f6f0c84e7dc41..7347efcf658f0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/StateTags.java @@ -158,6 +158,10 @@ public void appendTo(Appendable sb) throws IOException { sb.append(kind.prefix).append(rawId); } + public String getRawId() { + return rawId; + } + @Override public String toString() { return MoreObjects.toStringHelper(getClass()) @@ -195,12 +199,9 @@ protected StateTagBase(StructuredId id) { this.id = id; } - /** - * Returns the identifier for this state cell. - */ @Override public String getId() { - return id.getIdString(); + return id.getRawId(); } @Override diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/DataflowMatchers.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/DataflowMatchers.java new file mode 100644 index 0000000000000..ad21072dc4c90 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/DataflowMatchers.java @@ -0,0 +1,65 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk; + +import com.google.protobuf.ByteString; + +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; + +import java.io.Serializable; + +/** + * Matchers that are useful when writing Dataflow tests. + */ +public class DataflowMatchers { + /** + * Matcher for {@link ByteString} that prints the strings in UTF8. + */ + public static class ByteStringMatcher extends TypeSafeMatcher + implements Serializable { + private ByteString expected; + private ByteStringMatcher(ByteString expected) { + this.expected = expected; + } + + public static ByteStringMatcher byteStringEq(ByteString expected) { + return new ByteStringMatcher(expected); + } + + @Override + public void describeTo(Description description) { + description + .appendText("ByteString(") + .appendText(expected.toStringUtf8()) + .appendText(")"); + } + + @Override + public void describeMismatchSafely(ByteString actual, Description description) { + description + .appendText("was ByteString(") + .appendText(actual.toStringUtf8()) + .appendText(")"); + } + + @Override + protected boolean matchesSafely(ByteString actual) { + return actual.equals(expected); + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java index 7b3ddde901333..37c11c284ce0f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/dataflow/CustomSourcesTest.java @@ -619,8 +619,9 @@ public void testUnboundedSplits() throws Exception { @Test public void testReadUnboundedReader() throws Exception { - StreamingModeExecutionContext context = new StreamingModeExecutionContext( - "stageName", new ConcurrentHashMap(), null); + StreamingModeExecutionContext context = new StreamingModeExecutionContext("stageName", + new ConcurrentHashMap(), /*stateNameMap=*/null, + /*stateCache=*/null); DataflowPipelineOptions options = PipelineOptionsFactory.create().as(DataflowPipelineOptions.class); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CachingSideInputReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CachingSideInputReaderTest.java index 37e9511cf4d34..791d762dc6ef3 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CachingSideInputReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/CachingSideInputReaderTest.java @@ -25,8 +25,8 @@ import com.google.cloud.dataflow.sdk.testing.PCollectionViewTesting; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.util.PCollectionViewWindow; -import com.google.cloud.dataflow.sdk.util.Sized; -import com.google.cloud.dataflow.sdk.util.SizedSideInputReader; +import com.google.cloud.dataflow.sdk.util.WeightedSideInputReader; +import com.google.cloud.dataflow.sdk.util.WeightedValue; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; @@ -46,7 +46,7 @@ public class CachingSideInputReaderTest { private static boolean isCached( - Cache, Sized> cache, + Cache, WeightedValue> cache, PCollectionView view, BoundedWindow window) { return null != cache.getIfPresent(PCollectionViewWindow.of(view, window)); } @@ -69,13 +69,13 @@ private static boolean isCached( private static final int MAXIMUM_CACHE_SIZE = 1000; /** A {@link Cache} that is set up before each test. */ - private Cache, Sized> defaultCache; + private Cache, WeightedValue> defaultCache; @Before public void setupCache() { defaultCache = CacheBuilder.newBuilder() .maximumWeight(MAXIMUM_CACHE_SIZE) - .weigher(SizedWeigher.withBaseWeight(1)) + .weigher(Weighers.fixedWeightKeys(1)) .build(); } @@ -83,10 +83,10 @@ public void setupCache() { public void testCachingSideInputReaderAgreesWithUnderlyingReaderForSmallItem() throws Exception { // A SideInputReader that vends fixed contents for LENGTH_VIEW_FOR_DEFAULT_TAG // with a chosen size that fits in the maximum size of the cache. - SizedSideInputReader reader = SizedDirectSideInputReader.withContents( + WeightedSideInputReader reader = WeightedDirectSideInputReader.withContents( ImmutableMap.of( UNTYPED_ITERABLE_TAG, - Sized.of( + WeightedValue.of( PCollectionViewTesting.contentsInDefaultWindow("some", "small", "collection"), MAXIMUM_CACHE_SIZE - 100))); @@ -108,10 +108,10 @@ public void testCachingSideInputReaderAgreesWithUnderlyingReaderForSmallItem() t public void testCachingSideInputReaderAgreesWithUnderlyingReaderForLargeItem() throws Exception { // A SideInputReader that vends fixed contents for LENGTH_VIEW_FOR_DEFAULT_TAG // with a chosen size that exceeds the maximum size of the cache. - SizedSideInputReader reader = SizedDirectSideInputReader.withContents( + WeightedSideInputReader reader = WeightedDirectSideInputReader.withContents( ImmutableMap.of( UNTYPED_ITERABLE_TAG, - Sized.of( + WeightedValue.of( PCollectionViewTesting.contentsInDefaultWindow("some", "large", "collection"), MAXIMUM_CACHE_SIZE + 100))); @@ -134,10 +134,10 @@ public void testCachingSideInputReaderAgreesWithUnderlyingReaderForLargeItem() t public void testCachingSideInputReaderCachesSmallItem() throws Exception { // A SideInputReader that vends fixed contents for LENGTH_VIEW_FOR_DEFAULT_TAG // with a chosen size that fits in the maximum size of the cache. - SizedSideInputReader reader = SizedDirectSideInputReader.withContents( + WeightedSideInputReader reader = WeightedDirectSideInputReader.withContents( ImmutableMap.of( UNTYPED_ITERABLE_TAG, - Sized.of( + WeightedValue.of( PCollectionViewTesting.contentsInDefaultWindow("hello", "goodbye"), MAXIMUM_CACHE_SIZE - 1000))); @@ -160,10 +160,10 @@ public void testCachingSideInputReaderCachesSmallItem() throws Exception { public void testCachingSideInputReaderDoesNotCacheLargeItem() throws Exception { // A SideInputReader that vends fixed contents for LENGTH_VIEW_FOR_DEFAULT_TAG // with a chosen size that exceeds in the maximum size of the cache. - SizedSideInputReader reader = SizedDirectSideInputReader.withContents( + WeightedSideInputReader reader = WeightedDirectSideInputReader.withContents( ImmutableMap.of( UNTYPED_ITERABLE_TAG, - Sized.of( + WeightedValue.of( PCollectionViewTesting.contentsInDefaultWindow("hello", "goodbye"), MAXIMUM_CACHE_SIZE + 100))); @@ -188,9 +188,10 @@ public void testCachingSideInputReaderEmpty() throws Exception { PCollectionView view = PCollectionViewTesting.testingView( tag, new PCollectionViewTesting.LengthViewFn(), StringUtf8Coder.of()); - CachingSideInputReader sideInputReader = CachingSideInputReader.of( - SizedDirectSideInputReader.withContents(ImmutableMap., Sized>of()), - defaultCache); + CachingSideInputReader sideInputReader = + CachingSideInputReader.of(WeightedDirectSideInputReader.withContents( + ImmutableMap., WeightedValue>of()), + defaultCache); assertFalse(sideInputReader.contains(view)); assertTrue(sideInputReader.isEmpty()); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReaderTest.java index c3ad2bbd23e6c..90c55273a528b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowSideInputReaderTest.java @@ -34,7 +34,7 @@ import com.google.cloud.dataflow.sdk.util.BatchModeExecutionContext; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.ExecutionContext; -import com.google.cloud.dataflow.sdk.util.Sized; +import com.google.cloud.dataflow.sdk.util.WeightedValue; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; @@ -161,10 +161,10 @@ public void testDataflowSideInputReaderNotEmpty() throws Exception { @Test public void testDataflowSideInputReaderFilteredRead() throws Exception { assertTrue(defaultSideInputReader.contains(DEFAULT_LENGTH_VIEW)); - Sized sizedValue = defaultSideInputReader.getSized( + WeightedValue sizedValue = defaultSideInputReader.getWeighted( DEFAULT_LENGTH_VIEW, PCollectionViewTesting.DEFAULT_NONEMPTY_WINDOW); assertThat(sizedValue.getValue(), equalTo(DEFAULT_SOURCE_LENGTH)); - assertThat(sizedValue.getSize(), equalTo(DEFAULT_SOURCE_LENGTH * windowedLongBytes())); + assertThat(sizedValue.getWeight(), equalTo(DEFAULT_SOURCE_LENGTH * windowedLongBytes())); } /** @@ -176,15 +176,15 @@ public void testDataflowSideInputReaderRepeatedRead() throws Exception { DataflowSideInputReader sideInputReader = DataflowSideInputReader.of( Collections.singletonList(defaultSideInputInfo), options, executionContext); - Sized firstRead = sideInputReader.getSized( + WeightedValue firstRead = sideInputReader.getWeighted( DEFAULT_LENGTH_VIEW, PCollectionViewTesting.DEFAULT_NONEMPTY_WINDOW); // A repeated read should yield the same size. - Sized repeatedRead = sideInputReader.getSized( + WeightedValue repeatedRead = sideInputReader.getWeighted( DEFAULT_LENGTH_VIEW, PCollectionViewTesting.DEFAULT_NONEMPTY_WINDOW); assertThat(repeatedRead.getValue(), equalTo(firstRead.getValue())); - assertThat(repeatedRead.getSize(), equalTo(firstRead.getSize())); + assertThat(repeatedRead.getWeight(), equalTo(firstRead.getWeight())); } @@ -194,10 +194,10 @@ public void testDataflowSideInputReaderMiss() throws Exception { Collections.singletonList(defaultSideInputInfo), options, executionContext); // Reading an empty window yields the size of 0 elements. - Sized emptyWindowValue = sideInputReader.getSized( + WeightedValue emptyWindowValue = sideInputReader.getWeighted( DEFAULT_LENGTH_VIEW, PCollectionViewTesting.DEFAULT_EMPTY_WINDOW); assertThat(emptyWindowValue.getValue(), equalTo(0L)); - assertThat(emptyWindowValue.getSize(), equalTo(0L)); + assertThat(emptyWindowValue.getWeight(), equalTo(0L)); } /** diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContextTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContextTest.java index c0062ba9284bd..9c4f272e17e74 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContextTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/StreamingModeExecutionContextTest.java @@ -76,12 +76,14 @@ private static TupleTag>> newStringTag() { @Test public void testTimerInternalsSetTimer() { - StreamingModeExecutionContext executionContext = new StreamingModeExecutionContext( - "stageName", null, new ConcurrentHashMap()); + StreamingModeExecutionContext executionContext = new StreamingModeExecutionContext("stageName", + null, new ConcurrentHashMap(), + new WindmillStateCache().forComputation("comp")); Windmill.WorkItemCommitRequest.Builder outputBuilder = Windmill.WorkItemCommitRequest.newBuilder(); - executionContext.start(null, + executionContext.start( + Windmill.WorkItem.newBuilder().setKey(ByteString.EMPTY).setWorkToken(17L).build(), new Instant(1000), // input watermark null, // output watermark stateReader, stateFetcher, outputBuilder); @@ -108,7 +110,7 @@ public void testTimerInternalsSetTimer() { @Test public void testSideInputReaderReconstituted() { StreamingModeExecutionContext executionContext = - new StreamingModeExecutionContext("stageName", null, null); + new StreamingModeExecutionContext("stageName", null, null, null); PCollectionView preview1 = PCollectionViewTesting.testingView( newStringTag(), new ConstantViewFn("view1"), StringUtf8Coder.of()); @@ -160,7 +162,8 @@ public void testReaderCache() throws Exception { ConcurrentHashMap readerCache = new ConcurrentHashMap(); StreamingModeExecutionContext context = - new StreamingModeExecutionContext("stageName", readerCache, null); + new StreamingModeExecutionContext("stageName", readerCache, /*stateNameMap=*/null, + /*stateCache=*/null); UnboundedSource.UnboundedReader reader1 = new CountingSource(Integer.MAX_VALUE).createReader(options, null); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SizedDirectSideInputReader.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WeightedDirectSideInputReader.java similarity index 60% rename from sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SizedDirectSideInputReader.java rename to sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WeightedDirectSideInputReader.java index daccfd071cd21..9667df2eddf78 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/SizedDirectSideInputReader.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WeightedDirectSideInputReader.java @@ -20,8 +20,8 @@ import com.google.cloud.dataflow.sdk.util.DirectSideInputReader; import com.google.cloud.dataflow.sdk.util.PTuple; import com.google.cloud.dataflow.sdk.util.SideInputReader; -import com.google.cloud.dataflow.sdk.util.Sized; -import com.google.cloud.dataflow.sdk.util.SizedSideInputReader; +import com.google.cloud.dataflow.sdk.util.WeightedSideInputReader; +import com.google.cloud.dataflow.sdk.util.WeightedValue; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; @@ -29,29 +29,30 @@ import java.util.Map; /** - * A {@link SizedSideInputReader} with explicitly provided sizes for all values. + * A {@link WeightedSideInputReader} with explicitly provided sizes for all values. */ -class SizedDirectSideInputReader extends SizedSideInputReader.Defaults { +class WeightedDirectSideInputReader extends WeightedSideInputReader.Defaults { private final SideInputReader subReader; - private final Map, Long> sizes; + private final Map, Long> weights; /** - * Returns a {@link SizedDirectSideInputReader} containing the contents in the provided + * Returns a {@link WeightedDirectSideInputReader} containing the contents in the provided * {@code Map}. A {@link DirectSideInputReader} will be used for the actual retrieval logic; this * class merely does the size bookkeeping. */ - public static SizedDirectSideInputReader withContents( - Map, Sized> sizedContents) { - return new SizedDirectSideInputReader(sizedContents); + public static WeightedDirectSideInputReader withContents( + Map, WeightedValue> sizedContents) { + return new WeightedDirectSideInputReader(sizedContents); } - private SizedDirectSideInputReader(Map, Sized> sizedContents) { - sizes = new HashMap<>(); + private WeightedDirectSideInputReader( + Map, WeightedValue> sizedContents) { + weights = new HashMap<>(); PTuple values = PTuple.empty(); - for (Map.Entry, Sized> entry : sizedContents.entrySet()) { + for (Map.Entry, WeightedValue> entry : sizedContents.entrySet()) { values = values.and(entry.getKey(), entry.getValue().getValue()); - sizes.put(entry.getKey(), entry.getValue().getSize()); + weights.put(entry.getKey(), entry.getValue().getWeight()); } subReader = DirectSideInputReader.of(values); } @@ -67,9 +68,9 @@ public boolean contains(PCollectionView view) { } @Override - public Sized getSized(PCollectionView view, BoundedWindow window) { - return Sized.of( + public WeightedValue getWeighted(PCollectionView view, BoundedWindow window) { + return WeightedValue.of( subReader.get(view, window), - sizes.get(view.getTagInternal())); + weights.get(view.getTagInternal())); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateCacheTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateCacheTest.java new file mode 100644 index 0000000000000..c7ad7b3eac486 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateCacheTest.java @@ -0,0 +1,210 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.util.state.State; +import com.google.cloud.dataflow.sdk.util.state.StateNamespace; +import com.google.cloud.dataflow.sdk.util.state.StateNamespaces; +import com.google.cloud.dataflow.sdk.util.state.StateTag; +import com.google.protobuf.ByteString; + +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.util.Objects; + +/** + * Tests for {@link WindmillStateCache}. + */ +@RunWith(JUnit4.class) +public class WindmillStateCacheTest { + private static final String COMPUTATION = "computation"; + private static final ByteString KEY = ByteString.copyFromUtf8("key"); + private static final String STATE_FAMILY = "family"; + + private static class TestStateTag implements StateTag { + final String id; + + TestStateTag(String id) { + this.id = id; + } + + @Override + public void appendTo(Appendable appendable) throws IOException { + appendable.append(id); + } + + @Override + public String getId() { + return id; + } + + @Override + public TestState bind(StateBinder binder) { + throw new UnsupportedOperationException(); + } + + @Override + public String toString() { + return "Tag(" + id + ")"; + } + + @Override + public boolean equals(Object other) { + return (other instanceof TestStateTag) && Objects.equals(((TestStateTag) other).id, id); + } + + @Override + public int hashCode() { + return Objects.hash(id); + } + } + + private static class TestState implements State { + String value = null; + + TestState(String value) { + this.value = value; + } + + public String getValue() { + return value; + } + + @Override + public void clear() { + this.value = null; + } + + @Override + public boolean equals(Object other) { + return (other instanceof TestState) && Objects.equals(((TestState) other).value, value); + } + + @Override + public int hashCode() { + return Objects.hash(value); + } + + @Override + public String toString() { + return "State(" + value + ")"; + } + } + + private static StateNamespace windowNamespace(long start) { + return StateNamespaces.window( + IntervalWindow.getCoder(), new IntervalWindow(new Instant(start), new Instant(start + 1))); + } + + private static StateNamespace triggerNamespace(long start, int triggerIdx) { + return StateNamespaces.windowAndTrigger(IntervalWindow.getCoder(), + new IntervalWindow(new Instant(start), new Instant(start + 1)), triggerIdx); + } + + WindmillStateCache cache; + WindmillStateCache.ForKey keyCache; + + @Before + public void setUp() { + cache = new WindmillStateCache(); + keyCache = cache.forComputation(COMPUTATION).forKey(KEY, STATE_FAMILY, 0L); + assertEquals(0, cache.getWeight()); + } + + @Test + public void testBasic() throws Exception { + assertNull(keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertNull(keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); + assertNull(keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); + assertNull(keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag2"))); + + keyCache.put(StateNamespaces.global(), new TestStateTag("tag1"), new TestState("g1"), 2); + assertEquals(5, cache.getWeight()); + keyCache.put(windowNamespace(0), new TestStateTag("tag2"), new TestState("w2"), 2); + assertEquals(10, cache.getWeight()); + keyCache.put(triggerNamespace(0, 0), new TestStateTag("tag3"), new TestState("t3"), 2); + assertEquals(12, cache.getWeight()); + keyCache.put(triggerNamespace(0, 0), new TestStateTag("tag2"), new TestState("t2"), 2); + assertEquals(14, cache.getWeight()); + + assertEquals( + new TestState("g1"), keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertEquals(new TestState("w2"), keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); + assertEquals( + new TestState("t3"), keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); + assertEquals( + new TestState("t2"), keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag2"))); + } + + /** + * Verifies that values are cached in the appropriate namespaces. + */ + @Test + public void testInvalidation() throws Exception { + assertNull(keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + keyCache.put(StateNamespaces.global(), new TestStateTag("tag1"), new TestState("g1"), 2); + assertEquals(5, cache.getWeight()); + assertEquals( + new TestState("g1"), keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + + keyCache = cache.forComputation(COMPUTATION).forKey(KEY, STATE_FAMILY, 1L); + assertEquals(5, cache.getWeight()); + assertNull(keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertEquals(0, cache.getWeight()); + } + + /** + * Verifies that the cache is invalidated when the cache token changes. + */ + @Test + public void testEviction() throws Exception { + keyCache.put(windowNamespace(0), new TestStateTag("tag2"), new TestState("w2"), 2); + assertEquals(5, cache.getWeight()); + keyCache.put(triggerNamespace(0, 0), new TestStateTag("tag3"), new TestState("t3"), 2000000000); + assertEquals(0, cache.getWeight()); + // Eviction is atomic across the whole window. + assertNull(keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); + assertNull(keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); + } + + /** + * Verifies that caches are kept indedently per-key. + */ + @Test + public void testMultipleKeys() throws Exception { + WindmillStateCache.ForKey keyCache1 = cache.forComputation("comp1").forKey( + ByteString.copyFromUtf8("key1"), STATE_FAMILY, 0L); + WindmillStateCache.ForKey keyCache2 = cache.forComputation("comp1").forKey( + ByteString.copyFromUtf8("key2"), STATE_FAMILY, 0L); + WindmillStateCache.ForKey keyCache3 = cache.forComputation("comp2").forKey( + ByteString.copyFromUtf8("key1"), STATE_FAMILY, 0L); + + keyCache1.put(StateNamespaces.global(), new TestStateTag("tag1"), new TestState("g1"), 2); + assertEquals( + new TestState("g1"), keyCache1.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertNull(keyCache2.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertNull(keyCache3.get(StateNamespaces.global(), new TestStateTag("tag1"))); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java index a63d9a0417cea..febc376647545 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java @@ -15,10 +15,12 @@ */ package com.google.cloud.dataflow.sdk.runners.worker; +import static com.google.cloud.dataflow.sdk.DataflowMatchers.ByteStringMatcher.byteStringEq; import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.eq; import static org.mockito.Mockito.never; import static org.mockito.Mockito.when; @@ -53,10 +55,12 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; +import java.util.ArrayList; import java.util.Arrays; import java.util.concurrent.TimeUnit; @@ -79,6 +83,7 @@ public class WindmillStateInternalsTest { private WindmillStateReader mockReader; private WindmillStateInternals underTest; + private WindmillStateCache cache; @Mock private Supplier readStateSupplier; @@ -94,7 +99,10 @@ private static ByteString key(String prefix, StateNamespace namespace, String ad @Before public void setUp() { MockitoAnnotations.initMocks(this); - underTest = new WindmillStateInternals(STATE_FAMILY, true, mockReader, readStateSupplier); + cache = new WindmillStateCache(); + underTest = new WindmillStateInternals(STATE_FAMILY, mockReader, + cache.forComputation("comp").forKey(ByteString.EMPTY, STATE_FAMILY, 17L), + readStateSupplier); } private void waitAndSet(final SettableFuture future, final T value, final long millis) { @@ -111,6 +119,15 @@ public void run() { }).run(); } + private WindmillStateReader.WeightedList weightedList(String... elems) { + WindmillStateReader.WeightedList result = + new WindmillStateReader.WeightedList(new ArrayList(elems.length)); + for (String elem : elems) { + result.addWeighted(elem, elem.length()); + } + return result; + } + @Test public void testBagAddBeforeRead() throws Exception { StateTag> addr = StateTags.bag("bag", StringUtf8Coder.of()); @@ -208,8 +225,6 @@ public void testBagAddPersist() throws Exception { assertEquals(1, listUpdates.getValuesCount()); assertEquals("hello", listUpdates.getValues(0).getData().substring(1).toStringUtf8()); - // Blind adds should not need to read the future. - Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @@ -238,10 +253,6 @@ public void testBagClearPersist() throws Exception { assertEquals(1, listUpdates.getValuesCount()); assertEquals("world", listUpdates.getValues(0).getData().substring(1).toStringUtf8()); - // Clear should need to read the future. - Mockito.verify(mockReader) - .listFuture(key(NAMESPACE, "bag"), STATE_FAMILY, StringUtf8Coder.of()); - Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @@ -260,29 +271,6 @@ public void testBagPersistEmpty() throws Exception { assertEquals(1, commitBuilder.getListUpdatesCount()); } - @Test - public void testBagNoStateFamilies() throws Exception { - underTest = new WindmillStateInternals(STATE_FAMILY, false, mockReader, readStateSupplier); - - StateTag> addr = StateTags.bag("bag", StringUtf8Coder.of()); - BagState bag = underTest.state(NAMESPACE, addr); - - bag.add("hello"); - bag.clear(); - bag.add("world"); - - Windmill.WorkItemCommitRequest.Builder commitBuilder = - Windmill.WorkItemCommitRequest.newBuilder(); - underTest.persist(commitBuilder); - - // Clear should need to read the future. - Mockito.verify(mockReader) - .listFuture(key(STATE_FAMILY, NAMESPACE, "bag"), "", StringUtf8Coder.of()); - Mockito.verify(mockReader).startBatchAndBlock(); - Mockito.verifyNoMoreInteractions(mockReader); - } - - @Test public void testCombiningAddBeforeRead() throws Exception { CombiningValueState value = underTest.state(NAMESPACE, COMBINING_ADDR); @@ -331,7 +319,9 @@ public void testCombiningIsEmpty() throws Exception { when(mockReader.listFuture(COMBINING_KEY, STATE_FAMILY, accumCoder)) .thenReturn(future); StateContents result = value.isEmpty(); - Mockito.verify(mockReader).listFuture(COMBINING_KEY, STATE_FAMILY, accumCoder); + ArgumentCaptor byteString = ArgumentCaptor.forClass(ByteString.class); + Mockito.verify(mockReader).listFuture(byteString.capture(), eq(STATE_FAMILY), eq(accumCoder)); + assertThat(byteString.getValue(), byteStringEq(COMBINING_KEY)); waitAndSet(future, Arrays.asList(new int[] {29}), 200); assertThat(result.read(), Matchers.is(false)); @@ -374,8 +364,6 @@ public void testCombiningAddPersist() throws Exception { CoderUtils.decodeFromByteArray( accumCoder, listUpdates.getValues(0).getData().substring(1).toByteArray())[0]); - // Blind adds should not need to read the future. - Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @@ -443,9 +431,6 @@ public void testCombiningClearPersist() throws Exception { CoderUtils.decodeFromByteArray( accumCoder, listUpdates.getValues(0).getData().substring(1).toByteArray())[0]); - // Blind adds should not need to read the future. - Mockito.verify(mockReader).listFuture(COMBINING_KEY, STATE_FAMILY, accumCoder); - Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @@ -538,6 +523,8 @@ public void testWatermarkClearBeforeRead() throws Exception { Mockito.verifyNoMoreInteractions(mockReader); } + + /* @Test public void testWatermarkIsEmptyWindmillHasData() throws Exception { StateTag addr = StateTags.watermarkStateInternal( @@ -582,6 +569,7 @@ public void testWatermarkIsEmptyAfterClear() throws Exception { bag.add(new Instant(1000)); assertThat(result.read(), Matchers.is(false)); } + */ @Test public void testWatermarkPersistEarliest() throws Exception { @@ -602,8 +590,6 @@ public void testWatermarkPersistEarliest() throws Exception { assertEquals(key(NAMESPACE, "watermark"), watermarkHold.getTag()); assertEquals(TimeUnit.MILLISECONDS.toMicros(1000), watermarkHold.getTimestamps(0)); - // Blind adds should not need to read the future. - Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @@ -629,9 +615,7 @@ public void testWatermarkPersistLatestEmpty() throws Exception { assertEquals(key(NAMESPACE, "watermark"), watermarkHold.getTag()); assertEquals(TimeUnit.MILLISECONDS.toMicros(2000), watermarkHold.getTimestamps(0)); - // Blind adds should not need to read the future. Mockito.verify(mockReader).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); - Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @@ -657,9 +641,7 @@ public void testWatermarkPersistLatestWindmillWins() throws Exception { assertEquals(key(NAMESPACE, "watermark"), watermarkHold.getTag()); assertEquals(TimeUnit.MILLISECONDS.toMicros(4000), watermarkHold.getTimestamps(0)); - // Blind adds should not need to read the future. Mockito.verify(mockReader).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); - Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @@ -685,9 +667,7 @@ public void testWatermarkPersistLatestLocalAdditionsWin() throws Exception { assertEquals(key(NAMESPACE, "watermark"), watermarkHold.getTag()); assertEquals(TimeUnit.MILLISECONDS.toMicros(2000), watermarkHold.getTimestamps(0)); - // Blind adds should not need to read the future. Mockito.verify(mockReader).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); - Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @@ -711,7 +691,6 @@ public void testWatermarkPersistEndOfWindow() throws Exception { assertEquals(TimeUnit.MILLISECONDS.toMicros(2000), watermarkHold.getTimestamps(0)); // Blind adds should not need to read the future. - Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @@ -735,12 +714,8 @@ public void testWatermarkClearPersist() throws Exception { Windmill.WatermarkHold clearAndUpdate = commitBuilder.getWatermarkHolds(0); assertEquals(key(NAMESPACE, "watermark"), clearAndUpdate.getTag()); assertEquals(1, clearAndUpdate.getTimestampsCount()); - assertEquals(key(NAMESPACE, "watermark"), clearAndUpdate.getTag()); - assertEquals(1, clearAndUpdate.getTimestampsCount()); assertEquals(TimeUnit.MILLISECONDS.toMicros(1000), clearAndUpdate.getTimestamps(0)); - // Clearing requires reading the future. - Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @@ -761,39 +736,6 @@ public void testWatermarkPersistEmpty() throws Exception { assertEquals(1, commitBuilder.getWatermarkHoldsCount()); } - @Test - public void testWatermarkNoStateFamiliesEarliest() throws Exception { - underTest = new WindmillStateInternals(STATE_FAMILY, false, mockReader, readStateSupplier); - - StateTag addr = StateTags.watermarkStateInternal( - "watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); - WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); - bag.get(); - Mockito.verify(mockReader).watermarkFuture(key(STATE_FAMILY, NAMESPACE, "watermark"), ""); - } - - @Test - public void testWatermarkNoStateFamiliesLatest() throws Exception { - underTest = new WindmillStateInternals(STATE_FAMILY, false, mockReader, readStateSupplier); - - StateTag addr = StateTags.watermarkStateInternal( - "watermark", OutputTimeFns.outputAtLatestInputTimestamp()); - WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); - bag.get(); - Mockito.verify(mockReader).watermarkFuture(key(STATE_FAMILY, NAMESPACE, "watermark"), ""); - } - - @Test - public void testWatermarkNoStateFamiliesEndOfWindow() throws Exception { - underTest = new WindmillStateInternals(STATE_FAMILY, false, mockReader, readStateSupplier); - - StateTag addr = StateTags.watermarkStateInternal( - "watermark", OutputTimeFns.outputAtLatestInputTimestamp()); - WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); - bag.get(); - Mockito.verify(mockReader).watermarkFuture(key(STATE_FAMILY, NAMESPACE, "watermark"), ""); - } - @Test public void testValueSetBeforeRead() throws Exception { StateTag> addr = StateTags.value("value", StringUtf8Coder.of()); @@ -846,10 +788,6 @@ public void testValueSetPersist() throws Exception { assertEquals("Hi", valueUpdate.getValue().getData().toStringUtf8()); assertTrue(valueUpdate.isInitialized()); - // Setting a value requires a read to prevent blind writes. - Mockito.verify(mockReader) - .valueFuture(key(NAMESPACE, "value"), STATE_FAMILY, StringUtf8Coder.of()); - Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @@ -870,10 +808,6 @@ public void testValueClearPersist() throws Exception { assertEquals(key(NAMESPACE, "value"), valueUpdate.getTag()); assertEquals(0, valueUpdate.getValue().getData().size()); - // Setting a value requires a read to prevent blind writes. - Mockito.verify(mockReader) - .valueFuture(key(NAMESPACE, "value"), STATE_FAMILY, StringUtf8Coder.of()); - Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @@ -888,24 +822,157 @@ public void testValueNoChangePersist() throws Exception { assertEquals(0, commitBuilder.getValueUpdatesCount()); - // No changes shouldn't require getting any futures - Mockito.verify(mockReader).startBatchAndBlock(); Mockito.verifyNoMoreInteractions(mockReader); } @Test - public void testValueNoStateFamilies() throws Exception { - underTest = new WindmillStateInternals(STATE_FAMILY, false, mockReader, readStateSupplier); - + public void testCachedValue() throws Exception { StateTag> addr = StateTags.value("value", StringUtf8Coder.of()); ValueState value = underTest.state(NAMESPACE, addr); - SettableFuture future = SettableFuture.create(); - when(mockReader.valueFuture(key(STATE_FAMILY, NAMESPACE, "value"), "", StringUtf8Coder.of())) + assertEquals(0, cache.getWeight()); + + value.set("Hi"); + underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); + + assertEquals(2, cache.getWeight()); + + value = underTest.state(NAMESPACE, addr); + assertEquals("Hi", value.get().read()); + value.clear(); + underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); + + assertEquals(0, cache.getWeight()); + + value = underTest.state(NAMESPACE, addr); + assertEquals(null, value.get().read()); + + Mockito.verifyNoMoreInteractions(mockReader); + } + + @Test + public void testCachedBag() throws Exception { + StateTag> addr = StateTags.bag("bag", StringUtf8Coder.of()); + BagState bag = underTest.state(NAMESPACE, addr); + + assertEquals(0, cache.getWeight()); + + SettableFuture> future = SettableFuture.create(); + when(mockReader.listFuture(key(NAMESPACE, "bag"), STATE_FAMILY, StringUtf8Coder.of())) .thenReturn(future); - waitAndSet(future, "World", 200); - assertEquals("World", value.get().read()); + StateContents> result = bag.get(); + + assertEquals(0, cache.getWeight()); + + bag.add("hello"); + waitAndSet(future, weightedList("world"), 200); + assertThat(result.read(), Matchers.containsInAnyOrder("hello", "world")); + + underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); + + assertEquals(10, cache.getWeight()); + + bag = underTest.state(NAMESPACE, addr); + bag.add("goodbye"); + assertThat(bag.get().read(), Matchers.containsInAnyOrder("hello", "world", "goodbye")); + bag.clear(); + bag.add("new"); + + underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); + + assertEquals(3, cache.getWeight()); + + bag = underTest.state(NAMESPACE, addr); + bag.add("new2"); + assertThat(bag.get().read(), Matchers.containsInAnyOrder("new", "new2")); + bag.clear(); + bag.add("new3"); + + underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); + + assertEquals(4, cache.getWeight()); + + bag = underTest.state(NAMESPACE, addr); + assertThat(bag.get().read(), Matchers.containsInAnyOrder("new3")); + + Mockito.verify(mockReader) + .listFuture(key(NAMESPACE, "bag"), STATE_FAMILY, StringUtf8Coder.of()); + Mockito.verifyNoMoreInteractions(mockReader); + } + + @Test + public void testCachedWatermarkHold() throws Exception { + StateTag addr = StateTags.watermarkStateInternal( + "watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); + WatermarkStateInternal bag = underTest.state(NAMESPACE, addr); + + SettableFuture future = SettableFuture.create(); + when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)).thenReturn(future); + + assertEquals(0, cache.getWeight()); + + StateContents result = bag.get(); + + bag.add(new Instant(3000)); + waitAndSet(future, new Instant(2000), 200); + assertThat(result.read(), Matchers.equalTo(new Instant(2000))); + + underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); + + assertEquals(8, cache.getWeight()); + + bag = underTest.state(NAMESPACE, addr); + assertThat(bag.get().read(), Matchers.equalTo(new Instant(2000))); + bag.clear(); + + underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); + + assertEquals(8, cache.getWeight()); + + bag = underTest.state(NAMESPACE, addr); + assertEquals(null, bag.get().read()); + + Mockito.verify(mockReader).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); + Mockito.verifyNoMoreInteractions(mockReader); + } + + @Test + public void testCachedCombining() throws Exception { + CombiningValueState value = underTest.state(NAMESPACE, COMBINING_ADDR); + + SettableFuture> future = SettableFuture.create(); + when(mockReader.listFuture(key(NAMESPACE, "combining"), STATE_FAMILY, accumCoder)) + .thenReturn(future); + + assertEquals(0, cache.getWeight()); + + StateContents result = value.get(); + + value.add(1); + waitAndSet(future, Arrays.asList(new int[]{2}), 200); + assertThat(result.read(), Matchers.equalTo(3)); + + underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); + + assertEquals(1, cache.getWeight()); + + value = underTest.state(NAMESPACE, COMBINING_ADDR); + assertThat(value.get().read(), Matchers.equalTo(3)); + value.add(3); + assertThat(value.get().read(), Matchers.equalTo(6)); + value.clear(); + + underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); + + assertEquals(0, cache.getWeight()); + + value = underTest.state(NAMESPACE, COMBINING_ADDR); + assertThat(value.get().read(), Matchers.equalTo(0)); + + Mockito.verify(mockReader) + .listFuture(key(NAMESPACE, "combining"), STATE_FAMILY, accumCoder); + Mockito.verifyNoMoreInteractions(mockReader); } private void disableCompactOnWrite() { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ApiSurfaceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ApiSurfaceTest.java index d70c329be4a50..e995b821de69f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ApiSurfaceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ApiSurfaceTest.java @@ -47,6 +47,7 @@ public void testOurApiSurface() throws Exception { ApiSurface checkedApiSurface = ApiSurface.getSdkApiSurface() .pruningClassName("com.google.cloud.dataflow.sdk.runners.worker.StateFetcher") .pruningClassName("com.google.cloud.dataflow.sdk.util.common.ReflectHelpers") + .pruningClassName("com.google.cloud.dataflow.sdk.DataflowMatchers") .pruningClassName("com.google.cloud.dataflow.sdk.TestUtils") .pruningClassName("com.google.cloud.dataflow.sdk.WindowMatchers"); @@ -183,4 +184,3 @@ public void testExposedArrayCycle() throws Exception { assertExposed(ExposedArrayCycle.class, Exposed.class); } } - From 55cf870891b1ab080043008278139fa549f81796 Mon Sep 17 00:00:00 2001 From: dhalperi Date: Mon, 21 Dec 2015 18:30:29 -0800 Subject: [PATCH 1246/1541] DataflowPipelineRunnerTest: make pass if user has default project set In commit b6c4f8f3f7701a4f0190508e65eeb4aa9d1a4b88, I made DataflowPipelineOptions use the default project from gcloud in some cases it wasn't already being used. However, this change made a test fail on developer machines where this codepath would be used. Fix the test by explicitly setting the project to null, preventing the extraction of credentials from the gcloud configuration. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110733418 --- .../cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java index 3a29d82040954..cccdccff28941 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunnerTest.java @@ -519,6 +519,9 @@ public void testNoProjectFails() { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setRunner(DataflowPipelineRunner.class); + // Explicitly set to null to prevent the default instance factory from reading credentials + // from a user's environment, causing this test to fail. + options.setProject(null); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Project id"); From e436a1a7b2d99f923adc3ae07169ebaad04bf951 Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 21 Dec 2015 19:55:30 -0800 Subject: [PATCH 1247/1541] Touch up KV javadoc ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110736748 --- .../google/cloud/dataflow/sdk/values/KV.java | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java index 7601c30637be2..5143e063f2a51 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java @@ -29,24 +29,24 @@ /** * An immutable key/value pair. * - *

Various {@link PTransform}s like {@link GroupByKey} and {@link Combine#perKey} - * work on {@link PCollection}s of KVs. + *

Various {@link PTransform PTransforms} like {@link GroupByKey} and {@link Combine#perKey} + * operate on {@link PCollection PCollections} of {@link KV KVs}. * * @param the type of the key * @param the type of the value */ public class KV implements Serializable { - /** Returns a KV with the given key and value. */ + /** Returns a {@link KV} with the given key and value. */ public static KV of(K key, V value) { return new KV<>(key, value); } - /** Returns the key of this KV. */ + /** Returns the key of this {@link KV}. */ public K getKey() { return key; } - /** Returns the value of this KV. */ + /** Returns the value of this {@link KV}. */ public V getValue() { return value; } @@ -76,7 +76,11 @@ public boolean equals(Object other) { && Objects.deepEquals(this.value, otherKv.value); } - /** Orders the {@link KV} by the key. A null key is less than any non-null key. */ + /** + * Orders the {@link KV} by the key. + * + *

A {@code null} key is less than any non-{@code null} key. + */ public static class OrderByKey, V> implements SerializableComparator> { @Override @@ -91,7 +95,11 @@ public int compare(KV a, KV b) { } } - /** Orders the {@link KV} by the value. A null value is less than any non-null value. */ + /** + * Orders the {@link KV} by the value. + * + *

A {@code null} value is less than any non-{@code null} value. + */ public static class OrderByValue> implements SerializableComparator> { @Override From dcc3fe84bb671d61fa218425294f41223c4f4af7 Mon Sep 17 00:00:00 2001 From: dhalperi Date: Tue, 22 Dec 2015 09:54:18 -0800 Subject: [PATCH 1248/1541] RetryHttpRequestInitializer: verify that SocketTimeoutExceptions are handled ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110778488 --- .../util/RetryHttpRequestInitializerTest.java | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializerTest.java index a914b06f096b5..7097190802588 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/RetryHttpRequestInitializerTest.java @@ -16,7 +16,10 @@ package com.google.cloud.dataflow.sdk.util; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.anyString; @@ -36,9 +39,15 @@ import com.google.api.client.http.LowLevelHttpResponse; import com.google.api.client.json.JsonFactory; import com.google.api.client.json.jackson2.JacksonFactory; +import com.google.api.client.testing.http.MockHttpTransport; +import com.google.api.client.testing.http.MockLowLevelHttpRequest; import com.google.api.client.util.NanoClock; import com.google.api.client.util.Sleeper; +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.storage.Storage; +import com.google.common.collect.ImmutableList; import org.hamcrest.Matchers; import org.junit.After; @@ -53,8 +62,10 @@ import org.mockito.stubbing.Answer; import java.io.IOException; +import java.net.SocketTimeoutException; import java.security.PrivateKey; import java.util.Arrays; +import java.util.concurrent.atomic.AtomicLong; /** * Tests for RetryHttpRequestInitializer. @@ -240,4 +251,46 @@ public Integer answer(InvocationOnMock invocation) { verify(mockLowLevelRequest, times(retries)).execute(); verify(mockLowLevelResponse, times(retries)).getStatusCode(); } + + /** + * Tests that when RPCs fail with {@link SocketTimeoutException}, the IO exception handler + * is invoked. + */ + @Test + public void testIOExceptionHandlerIsInvokedOnTimeout() throws Exception { + // Counts the number of calls to execute the HTTP request. + final AtomicLong executeCount = new AtomicLong(); + + // 10 is a private internal constant in the Google API Client library. See + // com.google.api.client.http.HttpRequest#setNumberOfRetries + // TODO: update this test once the private internal constant is public. + final int defaultNumberOfRetries = 10; + + // A mock HTTP request that always throws SocketTimeoutException. + MockHttpTransport transport = + new MockHttpTransport.Builder().setLowLevelHttpRequest(new MockLowLevelHttpRequest() { + @Override + public LowLevelHttpResponse execute() throws IOException { + executeCount.incrementAndGet(); + throw new SocketTimeoutException("Fake forced timeout exception"); + } + }).build(); + + // A sample HTTP request to BigQuery that uses both default Transport and default + // RetryHttpInitializer. + Bigquery b = new Bigquery.Builder( + transport, Transport.getJsonFactory(), new RetryHttpRequestInitializer()).build(); + BigQueryTableInserter inserter = new BigQueryTableInserter(b); + TableReference t = new TableReference() + .setProjectId("project").setDatasetId("dataset").setTableId("table"); + + try { + inserter.insertAll(t, ImmutableList.of(new TableRow())); + fail(); + } catch (Throwable e) { + assertThat(e, Matchers.instanceOf(RuntimeException.class)); + assertThat(e.getCause(), Matchers.instanceOf(SocketTimeoutException.class)); + assertEquals(1 + defaultNumberOfRetries, executeCount.get()); + } + } } From 2cb3eac4e6a05fd64a9b9550a9d9e4788df707f1 Mon Sep 17 00:00:00 2001 From: klk Date: Tue, 22 Dec 2015 10:23:51 -0800 Subject: [PATCH 1249/1541] Touch up javadoc for values package ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110780949 --- .../google/cloud/dataflow/sdk/values/KV.java | 4 +- .../cloud/dataflow/sdk/values/PBegin.java | 12 +- .../dataflow/sdk/values/PCollection.java | 118 +++++++++--------- .../dataflow/sdk/values/PCollectionList.java | 59 +++++---- .../dataflow/sdk/values/PCollectionTuple.java | 23 ++-- .../dataflow/sdk/values/PCollectionView.java | 21 ++-- .../cloud/dataflow/sdk/values/PDone.java | 9 +- .../cloud/dataflow/sdk/values/PInput.java | 15 ++- .../cloud/dataflow/sdk/values/POutput.java | 31 ++--- .../dataflow/sdk/values/POutputValueBase.java | 19 +-- .../cloud/dataflow/sdk/values/PValue.java | 9 +- .../cloud/dataflow/sdk/values/PValueBase.java | 44 +++---- .../dataflow/sdk/values/TimestampedValue.java | 8 +- .../cloud/dataflow/sdk/values/TupleTag.java | 10 +- .../dataflow/sdk/values/TupleTagList.java | 36 +++--- .../dataflow/sdk/values/TypeDescriptor.java | 12 +- .../dataflow/sdk/values/TypedPValue.java | 41 +++--- .../dataflow/sdk/values/package-info.java | 48 ++++--- 18 files changed, 272 insertions(+), 247 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java index 5143e063f2a51..23cee07cfe051 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/KV.java @@ -77,7 +77,7 @@ public boolean equals(Object other) { } /** - * Orders the {@link KV} by the key. + * A {@link Comparator} that orders {@link KV KVs} by the natural ordering of their keys. * *

A {@code null} key is less than any non-{@code null} key. */ @@ -96,7 +96,7 @@ public int compare(KV a, KV b) { } /** - * Orders the {@link KV} by the value. + * A {@link Comparator} that orders {@link KV KVs} by the natural ordering of their values. * *

A {@code null} value is less than any non-{@code null} value. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PBegin.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PBegin.java index 852c7084bec89..23ac3aed32d88 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PBegin.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PBegin.java @@ -25,14 +25,14 @@ import java.util.Collections; /** - * {@code PBegin} is used as the "input" to a root {@link PTransform} that is the first - * operation in a {@link Pipeline}, such as {@link Read TextIO.Read} or {@link Create}. + * {@link PBegin} is the "input" to a root {@link PTransform}, such as {@link Read Read} or + * {@link Create}. * *

Typically created by calling {@link Pipeline#begin} on a Pipeline. */ public class PBegin implements PInput { /** - * Returns a {@code PBegin} in the given {@code Pipeline}. + * Returns a {@link PBegin} in the given {@link Pipeline}. */ public static PBegin in(Pipeline pipeline) { return new PBegin(pipeline); @@ -40,7 +40,7 @@ public static PBegin in(Pipeline pipeline) { /** * Like {@link #apply(String, PTransform)} but defaulting to the name - * of the {@code PTransform}. + * of the {@link PTransform}. */ public OutputT apply( PTransform t) { @@ -48,7 +48,7 @@ public OutputT apply( } /** - * Applies the given {@code PTransform} to this input {@code PBegin}, + * Applies the given {@link PTransform} to this input {@link PBegin}, * using {@code name} to identify this specific application of the transform. * This name is used in various places, including the monitoring UI, logging, * and to stably identify this application node in the job graph. @@ -77,7 +77,7 @@ public void finishSpecifying() { ///////////////////////////////////////////////////////////////////////////// /** - * Constructs a {@code PBegin} in the given {@code Pipeline}. + * Constructs a {@link PBegin} in the given {@link Pipeline}. */ protected PBegin(Pipeline pipeline) { this.pipeline = pipeline; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java index 15e6345b49f15..6fffddfeb9606 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollection.java @@ -18,76 +18,76 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.PubsubIO; +import com.google.cloud.dataflow.sdk.io.Read; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.WindowingStrategy; /** - * A {@code PCollection} is an immutable collection of values of type - * {@code T}. A {@code PCollection} can contain either a bounded or unbounded - * number of elements. Bounded and unbounded {@code PCollection}s are produced - * as the output of {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s - * (including root PTransforms like - * {@link com.google.cloud.dataflow.sdk.io.TextIO.Read}, - * {@link com.google.cloud.dataflow.sdk.io.PubsubIO.Read} and - * {@link com.google.cloud.dataflow.sdk.transforms.Create}), and can + * A {@link PCollection PCollection<T>} is an immutable collection of values of type + * {@code T}. A {@link PCollection} can contain either a bounded or unbounded + * number of elements. Bounded and unbounded {@link PCollection PCollections} are produced + * as the output of {@link PTransform PTransforms} + * (including root PTransforms like {@link Read} and {@link Create}), and can * be passed as the inputs of other PTransforms. * *

Some root transforms produce bounded {@code PCollections} and others - * produce unbounded ones. For example, - * {@link com.google.cloud.dataflow.sdk.io.TextIO.Read} reads a static set - * of files, so it produces a bounded {@code PCollection}. - * {@link com.google.cloud.dataflow.sdk.io.PubsubIO.Read}, on the other hand, - * receives a potentially infinite stream of Pubsub messages, so it produces - * an unbounded {@code PCollection}. + * produce unbounded ones. For example, {@link TextIO.Read} reads a static set + * of files, so it produces a bounded {@link PCollection}. + * {@link PubsubIO.Read}, on the other hand, receives a potentially infinite stream + * of Pubsub messages, so it produces an unbounded {@link PCollection}. * - *

Each element in a {@code PCollection} may have an associated implicit + *

Each element in a {@link PCollection} may have an associated implicit * timestamp. Readers assign timestamps to elements when they create - * {@code PCollection}s, and other {@code PTransform}s propagate these - * timestamps from their input to their output. For example, PubsubIO.Read - * assigns pubsub message timestamps to elements, and TextIO.Read assigns - * the default value {@code Long.MIN_VALUE} to elements. User code can + * {@link PCollection PCollections}, and other {@link PTransform PTransforms} propagate these + * timestamps from their input to their output. For example, {@link PubsubIO.Read} + * assigns pubsub message timestamps to elements, and {@link TextIO.Read} assigns + * the default value {@link BoundedWindow#TIMESTAMP_MIN_VALUE} to elements. User code can * explicitly assign timestamps to elements with * {@link com.google.cloud.dataflow.sdk.transforms.DoFn.Context#outputWithTimestamp}. * - *

Additionally, a {@code PCollection} has an associated + *

Additionally, a {@link PCollection} has an associated * {@link WindowFn} and each element is assigned to a set of windows. * By default, the windowing function is {@link GlobalWindows} * and all elements are assigned into a single default window. * This default can be overridden with the {@link Window} - * {@code PTransform}. Dataflow pipelines run in classic batch MapReduce style - * with the default GlobalWindow strategy if timestamps are ignored. + * {@link PTransform}. * - *

See the individual {@code PTransform} subclasses for specific information + *

See the individual {@link PTransform} subclasses for specific information * on how they propagate timestamps and windowing. * - * @param the type of the elements of this PCollection + * @param the type of the elements of this {@link PCollection} */ public class PCollection extends TypedPValue { /** - * The PCollection IsBounded property. + * The enumeration of cases for whether a {@link PCollection} is bounded. */ public enum IsBounded { /** - * {@code PCollection} contains bounded data elements, such as - * {@code PCollection}s from {@code TextIO}, {@code BigQueryIO}, - * {@code Create} e.t.c. + * Indicates that a {@link PCollection} contains bounded data elements, such as + * {@link PCollection PCollections} from {@link TextIO}, {@link BigQueryIO}, + * {@link Create} e.t.c. */ BOUNDED, /** - * {@code PCollection} contains unbounded data elements, such as - * {@code PCollection}s from {@code PubsubIO}. + * Indicates that a {@link PCollection} contains unbounded data elements, such as + * {@link PCollection PCollections} from {@link PubsubIO}. */ UNBOUNDED; /** * Returns the composed IsBounded property. * - *

The composed property is BOUNDED only if all components are BOUNDED. - * Otherwise, it is UNBOUNDED. + *

The composed property is {@link #BOUNDED} only if all components are {@link #BOUNDED}. + * Otherwise, it is {@link #UNBOUNDED}. */ public IsBounded and(IsBounded that) { if (this == BOUNDED && that == BOUNDED) { @@ -99,10 +99,10 @@ public IsBounded and(IsBounded that) { } /** - * Returns the name of this PCollection. + * Returns the name of this {@link PCollection}. * - *

By default, the name of a PCollection is based on the name of the - * PTransform that produces it. It can be specified explicitly by + *

By default, the name of a {@link PCollection} is based on the name of the + * {@link PTransform} that produces it. It can be specified explicitly by * calling {@link #setName}. * * @throws IllegalStateException if the name hasn't been set yet @@ -113,11 +113,11 @@ public String getName() { } /** - * Sets the name of this PCollection. Returns {@code this}. + * Sets the name of this {@link PCollection}. Returns {@code this}. * - * @throws IllegalStateException if this PCollection has already been - * finalized and is no longer settable, e.g., by having - * {@code apply()} called on it + * @throws IllegalStateException if this {@link PCollection} has already been + * finalized and may no longer be set. + * Once {@link #apply} has been called, this will be the case. */ @Override public PCollection setName(String name) { @@ -126,11 +126,11 @@ public PCollection setName(String name) { } /** - * Returns the Coder used by this PCollection to encode and decode + * Returns the {@link Coder} used by this {@link PCollection} to encode and decode * the values stored in it. * - * @throws IllegalStateException if the Coder hasn't been set, and - * couldn't be inferred + * @throws IllegalStateException if the {@link Coder} hasn't been set, and + * couldn't be inferred. */ @Override public Coder getCoder() { @@ -138,12 +138,12 @@ public Coder getCoder() { } /** - * Sets the Coder used by this PCollection to encode and decode the - * values stored in it. Returns {@code this}. + * Sets the {@link Coder} used by this {@link PCollection} to encode and decode the + * values stored in it. Returns {@code this}. * - * @throws IllegalStateException if this PCollection has already - * been finalized and is no longer settable, e.g., by having - * {@code apply()} called on it + * @throws IllegalStateException if this {@link PCollection} has already + * been finalized and may no longer be set. + * Once {@link #apply} has been called, this will be the case. */ @Override public PCollection setCoder(Coder coder) { @@ -154,16 +154,20 @@ public PCollection setCoder(Coder coder) { /** * Like {@link IsBounded#apply(String, PTransform)} but defaulting to the name * of the {@link PTransform}. + * + * @return the output of the applied {@link PTransform} */ public OutputT apply(PTransform, OutputT> t) { return Pipeline.applyTransform(this, t); } /** - * Applies the given {@code PTransform} to this input {@code PCollection}, + * Applies the given {@link PTransform} to this input {@link PCollection}, * using {@code name} to identify this specific application of the transform. * This name is used in various places, including the monitoring UI, logging, * and to stably identify this application node in the job graph. + * + * @return the output of the applied {@link PTransform} */ public OutputT apply( String name, PTransform, OutputT> t) { @@ -171,7 +175,7 @@ public OutputT apply( } /** - * Returns the {@link WindowingStrategy} of this {@code PCollection}. + * Returns the {@link WindowingStrategy} of this {@link PCollection}. */ public WindowingStrategy getWindowingStrategy() { return windowingStrategy; @@ -186,7 +190,7 @@ public IsBounded isBounded() { /** * {@link WindowingStrategy} that will be used for merging windows and triggering output in this - * {@code PCollection} and subsequence {@code PCollection}s produced from this one. + * {@link PCollection} and subsequence {@link PCollection PCollections} produced from this one. * *

By default, no merging is performed. */ @@ -199,10 +203,10 @@ private PCollection(Pipeline p) { } /** - * Sets the {@code TypeDescriptor} for this {@code PCollection}, so that - * the enclosing {@code PCollectionTuple}, {@code PCollectionList}, - * or {@code PTransform>}, etc., can provide - * more detailed reflective information. + * Sets the {@link TypeDescriptor TypeDescriptor<T>} for this + * {@link PCollection PCollection<T>}. This may allow the enclosing + * {@link PCollectionTuple}, {@link PCollectionList}, or {@code PTransform>}, + * etc., to provide more detailed reflective information. */ @Override public PCollection setTypeDescriptorInternal(TypeDescriptor typeDescriptor) { @@ -211,7 +215,7 @@ public PCollection setTypeDescriptorInternal(TypeDescriptor typeDescriptor } /** - * Sets the {@link WindowingStrategy} of this {@code PCollection}. + * Sets the {@link WindowingStrategy} of this {@link PCollection}. * *

For use by primitive transformations only. */ @@ -221,7 +225,7 @@ public PCollection setWindowingStrategyInternal(WindowingStrategy windo } /** - * Sets the {@link PCollection.IsBounded} of this {@code PCollection}. + * Sets the {@link PCollection.IsBounded} of this {@link PCollection}. * *

For use by internal transformations only. */ @@ -231,7 +235,7 @@ public PCollection setIsBoundedInternal(IsBounded isBounded) { } /** - * Creates and returns a new PCollection for a primitive output. + * Creates and returns a new {@link PCollection} for a primitive output. * *

For use by primitive transformations only. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionList.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionList.java index 867ca5100aef1..b99af020bfc8d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionList.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionList.java @@ -18,7 +18,9 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; +import com.google.cloud.dataflow.sdk.transforms.Flatten; import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.Partition; import com.google.common.collect.ImmutableList; import java.util.ArrayList; @@ -28,12 +30,10 @@ import java.util.List; /** - * A {@code PCollectionList} is an immutable list of homogeneously - * typed {@code PCollection}s. A PCollectionList is used, for + * A {@link PCollectionList PCollectionList<T>} is an immutable list of homogeneously + * typed {@link PCollection PCollection<T>s}. A {@link PCollectionList} is used, for * instance, as the input to - * {@link com.google.cloud.dataflow.sdk.transforms.Flatten} or the - * output of - * {@link com.google.cloud.dataflow.sdk.transforms.Partition}. + * {@link Flatten} or the output of {@link Partition}. * *

PCollectionLists can be created and accessed like follows: *

 {@code
@@ -57,13 +57,13 @@
  * List> allPcs = pcs.getAll();
  * } 
* - * @param the type of the elements of all the PCollections in this list + * @param the type of the elements of all the {@link PCollection PCollections} in this list */ public class PCollectionList implements PInput, POutput { /** - * Returns an empty PCollectionList that is part of the given Pipeline. + * Returns an empty {@link PCollectionList} that is part of the given {@link Pipeline}. * - *

Longer PCollectionLists can be created by calling + *

Longer {@link PCollectionList PCollectionLists} can be created by calling * {@link #and} on the result. */ public static PCollectionList empty(Pipeline pipeline) { @@ -71,9 +71,9 @@ public static PCollectionList empty(Pipeline pipeline) { } /** - * Returns a singleton PCollectionList containing the given PCollection. + * Returns a singleton {@link PCollectionList} containing the given {@link PCollection}. * - *

Longer PCollectionLists can be created by calling + *

Longer {@link PCollectionList PCollectionLists} can be created by calling * {@link #and} on the result. */ public static PCollectionList of(PCollection pc) { @@ -81,12 +81,13 @@ public static PCollectionList of(PCollection pc) { } /** - * Returns a PCollectionList containing the given PCollections, in order. + * Returns a {@link PCollectionList} containing the given {@link PCollection PCollections}, + * in order. * *

The argument list cannot be empty. * - *

All the PCollections in the resulting PCollectionList must be - * part of the same Pipeline. + *

All the {@link PCollection PCollections} in the resulting {@link PCollectionList} must be + * part of the same {@link Pipeline}. * *

Longer PCollectionLists can be created by calling * {@link #and} on the result. @@ -102,11 +103,11 @@ public static PCollectionList of(Iterable> pcs) { } /** - * Returns a new PCollectionList that has all the PCollections of - * this PCollectionList plus the given PCollection appended to the end. + * Returns a new {@link PCollectionList} that has all the {@link PCollection PCollections} of + * this {@link PCollectionList} plus the given {@link PCollection} appended to the end. * - *

All the PCollections in the resulting PCollectionList must be - * part of the same Pipeline. + *

All the {@link PCollection PCollections} in the resulting {@link PCollectionList} must be + * part of the same {@link Pipeline}. */ public PCollectionList and(PCollection pc) { if (pc.getPipeline() != pipeline) { @@ -121,12 +122,12 @@ public PCollectionList and(PCollection pc) { } /** - * Returns a new PCollectionList that has all the PCollections of - * this PCollectionList plus the given PCollections appended to the end, - * in order. + * Returns a new {@link PCollectionList} that has all the {@link PCollection PCollections} of + * this {@link PCollectionList} plus the given {@link PCollection PCollections} appended to the + * end, in order. * - *

All the PCollections in the resulting PCollectionList must be - * part of the same Pipeline. + *

All the {@link PCollections} in the resulting {@link PCollectionList} must be + * part of the same {@link Pipeline}. */ public PCollectionList and(Iterable> pcs) { List> copy = new ArrayList<>(pcollections); @@ -141,15 +142,16 @@ public PCollectionList and(Iterable> pcs) { } /** - * Returns the number of PCollections in this PCollectionList. + * Returns the number of {@link PCollection PCollections} in this {@link PCollectionList}. */ public int size() { return pcollections.size(); } /** - * Returns the PCollection at the given index (origin zero). Throws - * IndexOutOfBounds if the index is out of the range + * Returns the {@link PCollection} at the given index (origin zero). + * + * @throws IndexOutOfBoundsException if the index is out of the range * {@code [0..size()-1]}. */ public PCollection get(int index) { @@ -157,7 +159,8 @@ public PCollection get(int index) { } /** - * Returns an immutable List of all the PCollections in this PCollectionList. + * Returns an immutable List of all the {@link PCollection PCollections} in this + * {@link PCollectionList}. */ public List> getAll() { return pcollections; @@ -173,10 +176,12 @@ public OutputT apply( } /** - * Applies the given {@code PTransform} to this input {@code PCollectionList}, + * Applies the given {@link PTransform} to this input {@link PCollectionList}, * using {@code name} to identify this specific application of the transform. * This name is used in various places, including the monitoring UI, logging, * and to stably identify this application node in the job graph. + * + * @return the output of the applied {@link PTransform} */ public OutputT apply( String name, PTransform, OutputT> t) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java index 04926392f98c0..58550e4182c74 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionTuple.java @@ -19,6 +19,7 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.util.WindowingStrategy; import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded; import com.google.common.collect.ImmutableMap; @@ -29,17 +30,17 @@ import java.util.Map; /** - * A {@code PCollectionTuple} is an immutable tuple of - * heterogeneously-typed {@link PCollection}s, "keyed" by - * {@link TupleTag}s. A PCollectionTuple can be used as the input or + * A {@link PCollectionTuple} is an immutable tuple of + * heterogeneously-typed {@link PCollection PCollections}, "keyed" by + * {@link TupleTag TupleTags}. A {@link PCollectionTuple} can be used as the input or * output of a - * {@link com.google.cloud.dataflow.sdk.transforms.PTransform} taking + * {@link PTransform} taking * or producing multiple PCollection inputs or outputs that can be of * different types, for instance a - * {@link com.google.cloud.dataflow.sdk.transforms.ParDo} with side + * {@link ParDo} with side * outputs. * - *

A {@code PCollectionTuple} can be created and accessed like follows: + *

A {@link PCollectionTuple} can be created and accessed like follows: *

 {@code
  * PCollection pc1 = ...;
  * PCollection pc2 = ...;
@@ -74,7 +75,7 @@
  */
 public class PCollectionTuple implements PInput, POutput {
   /**
-   * Returns an empty {@code PCollectionTuple} that is part of the given {@link Pipeline}.
+   * Returns an empty {@link PCollectionTuple} that is part of the given {@link Pipeline}.
    *
    * 

A {@link PCollectionTuple} containing additional elements can be created by calling * {@link #and} on the result. @@ -87,7 +88,7 @@ public static PCollectionTuple empty(Pipeline pipeline) { * Returns a singleton {@link PCollectionTuple} containing the given * {@link PCollection} keyed by the given {@link TupleTag}. * - *

A {@code PCollectionTuple} containing additional elements can be created by calling + *

A {@link PCollectionTuple} containing additional elements can be created by calling * {@link #and} on the result. */ public static PCollectionTuple of(TupleTag tag, PCollection pc) { @@ -152,6 +153,8 @@ public Map, PCollection> getAll() { /** * Like {@link #apply(String, PTransform)} but defaulting to the name * of the {@link PTransform}. + * + * @return the output of the applied {@link PTransform} */ public OutputT apply( PTransform t) { @@ -159,10 +162,12 @@ public OutputT apply( } /** - * Applies the given {@code PTransform} to this input {@code PCollectionTuple}, + * Applies the given {@link PTransform} to this input {@link PCollectionTuple}, * using {@code name} to identify this specific application of the transform. * This name is used in various places, including the monitoring UI, logging, * and to stably identify this application node in the job graph. + * + * @return the output of the applied {@link PTransform} */ public OutputT apply( String name, PTransform t) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionView.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionView.java index e57a9fbc659d5..515e21ba6df97 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionView.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PCollectionView.java @@ -17,30 +17,29 @@ package com.google.cloud.dataflow.sdk.values; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.View; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowingStrategy; import java.io.Serializable; /** - * A {@link PCollectionView PCollectionView<T>} is an immutable view of a - * {@link PCollection} as a value of type {@code T} that can be accessed e.g. as - * a side input to a {@link com.google.cloud.dataflow.sdk.transforms.DoFn}. + * A {@link PCollectionView PCollectionView<T>} is an immutable view of a {@link PCollection} + * as a value of type {@code T} that can be accessed + * as a side input to a {@link ParDo} transform. * - *

A {@code PCollectionView} should always be the output of a + *

A {@link PCollectionView} should always be the output of a * {@link com.google.cloud.dataflow.sdk.transforms.PTransform}. It is the joint responsibility of * this transform and each {@link com.google.cloud.dataflow.sdk.runners.PipelineRunner} to implement * the view in a runner-specific manner. * - *

The most common case is using the {@link com.google.cloud.dataflow.sdk.transforms.View} - * transforms to prepare a {@link PCollection} for use as a side input to - * {@link com.google.cloud.dataflow.sdk.transforms.ParDo}. See - * {@link com.google.cloud.dataflow.sdk.transforms.View#asSingleton()}, - * {@link com.google.cloud.dataflow.sdk.transforms.View#asIterable()}, and - * {@link com.google.cloud.dataflow.sdk.transforms.View#asMap()} for more detail on specific views + *

The most common case is using the {@link View} transforms to prepare a {@link PCollection} + * for use as a side input to {@link ParDo}. See {@link View#asSingleton()}, + * {@link View#asIterable()}, and {@link View#asMap()} for more detail on specific views * available in the SDK. * - * @param the type of the value(s) accessible via this {@code PCollectionView} + * @param the type of the value(s) accessible via this {@link PCollectionView} */ public interface PCollectionView extends PValue, Serializable { /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PDone.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PDone.java index f6a2af3e4cf3a..39a00616bf715 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PDone.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PDone.java @@ -17,20 +17,19 @@ package com.google.cloud.dataflow.sdk.values; import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.transforms.PTransform; import java.util.Collection; import java.util.Collections; /** - * {@code PDone} is the output of a - * {@link com.google.cloud.dataflow.sdk.transforms.PTransform} that - * doesn't have a non-trival result, e.g., a Write. No more - * transforms can be applied to it. + * {@link PDone} is the output of a {@link PTransform} that has a trivial result, + * such as a {@link Write}. */ public class PDone extends POutputValueBase { /** - * Creates a {@code PDone} in the given {@code Pipeline}. + * Creates a {@link PDone} in the given {@link Pipeline}. */ public static PDone in(Pipeline pipeline) { return new PDone(pipeline); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PInput.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PInput.java index 39cfaabc735e5..89b097a65318f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PInput.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PInput.java @@ -26,17 +26,20 @@ */ public interface PInput { /** - * Returns the owning {@link Pipeline} of this {@code PInput}. + * Returns the owning {@link Pipeline} of this {@link PInput}. */ public Pipeline getPipeline(); /** - * Expands this {@code PInput} into a list of its component input {@link PValue}s. + * Expands this {@link PInput} into a list of its component output + * {@link PValue PValues}. * - *

A {@link PValue} expands to itself. - * - *

A tuple or list of {@link PValue}s (e.g., {@link PCollectionTuple}, - * and {@link PCollectionList}) expands to its component {@link PValue}s. + *

    + *
  • A {@link PValue} expands to itself.
  • + *
  • A tuple or list of {@link PValue PValues} (such as + * {@link PCollectionTuple} or {@link PCollectionList}) + * expands to its component {@code PValue PValues}.
  • + *
* *

Not intended to be invoked directly by user code. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutput.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutput.java index 8e9ccbe34323e..f99bc0b09ddae 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutput.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutput.java @@ -28,19 +28,20 @@ public interface POutput { /** - * Returns the owning {@link Pipeline} of this {@code POutput}. + * Returns the owning {@link Pipeline} of this {@link POutput}. */ public Pipeline getPipeline(); /** - * Expands this {@code POutput} into a list of its component output - * {@code PValue}s. + * Expands this {@link POutput} into a list of its component output + * {@link PValue PValues}. * - *

A {@link PValue} expands to itself. - * - *

A tuple or list of {@code PValue}s (e.g., - * {@link PCollectionTuple}, and - * {@link PCollectionList}) expands to its component {@code PValue}s. + *

    + *
  • A {@link PValue} expands to itself.
  • + *
  • A tuple or list of {@link PValue PValues} (such as + * {@link PCollectionTuple} or {@link PCollectionList}) + * expands to its component {@code PValue PValues}.
  • + *
* *

Not intended to be invoked directly by user code. */ @@ -55,20 +56,20 @@ public interface POutput { * *

This is not intended to be invoked by user code, but * is automatically invoked as part of applying the - * producing {@code PTransform}. + * producing {@link PTransform}. */ public void recordAsOutput(AppliedPTransform transform); /** - * As part of applying the producing {@code PTransform}, finalizes this + * As part of applying the producing {@link PTransform}, finalizes this * output to make it ready for being used as an input and for running. * - *

This includes ensuring that all {@code PCollection}s - * have {@code Coder}s specified or defaulted. + *

This includes ensuring that all {@link PCollection PCollections} + * have {@link Coder Coders} specified or defaulted. * - *

Automatically invoked whenever this {@code POutput} is used - * as a {@code PInput} to another {@code PTransform}, or if never - * used as a {@code PInput}, when {@link Pipeline#run} + *

Automatically invoked whenever this {@link POutput} is used + * as a {@link PInput} to another {@link PTransform}, or if never + * used as a {@link PInput}, when {@link Pipeline#run} * is called, so users do not normally call this explicitly. */ public void finishSpecifyingOutput(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutputValueBase.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutputValueBase.java index 3f28677b3c683..69e04c3436423 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutputValueBase.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/POutputValueBase.java @@ -18,13 +18,14 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; +import com.google.cloud.dataflow.sdk.transforms.PTransform; /** - * A {@code POutputValueBase} is the abstract base class of + * A {@link POutputValueBase} is the abstract base class of * {@code PTransform} outputs. * - *

A {@code PValueBase} that adds tracking of its producing - * {@code AppliedPTransform}. + *

A {@link PValueBase} that adds tracking of its producing + * {@link AppliedPTransform}. * *

For internal use. */ @@ -38,7 +39,7 @@ protected POutputValueBase(Pipeline pipeline) { /** * No-arg constructor for Java serialization only. - * The resulting {@code POutputValueBase} is unlikely to be + * The resulting {@link POutputValueBase} is unlikely to be * valid. */ protected POutputValueBase() { @@ -51,7 +52,7 @@ public Pipeline getPipeline() { } /** - * Returns the {@code AppliedPTransform} that this {@code POutputValueBase} + * Returns the {@link AppliedPTransform} that this {@link POutputValueBase} * is an output of. * *

For internal use only. @@ -61,8 +62,8 @@ public Pipeline getPipeline() { } /** - * Records that this {@code POutputValueBase} is an output with the - * given name of the given {@code AppliedPTransform}. + * Records that this {@link POutputValueBase} is an output with the + * given name of the given {@link AppliedPTransform}. * *

To be invoked only by {@link POutput#recordAsOutput} * implementations. Not to be invoked directly by user code. @@ -87,7 +88,7 @@ public void recordAsOutput(AppliedPTransform transform) { } /** - * Default behavior for {@code finishSpecifyingOutput()} is + * Default behavior for {@link #finishSpecifyingOutput()} is * to do nothing. Override if your {@link PValue} requires * finalization. */ @@ -95,7 +96,7 @@ public void recordAsOutput(AppliedPTransform transform) { public void finishSpecifyingOutput() { } /** - * The {@code PTransform} that produces this {@code POutputValueBase}. + * The {@link PTransform} that produces this {@link POutputValueBase}. */ private AppliedPTransform producingTransform; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValue.java index 5196f965f48ee..eb95a23f50f43 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValue.java @@ -17,21 +17,20 @@ package com.google.cloud.dataflow.sdk.values; import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; +import com.google.cloud.dataflow.sdk.transforms.PTransform; /** - * A {@code PValue} is the interface to values that can be - * input and output from {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s. + * The interface for values that can be input to and output from {@link PTransform PTransforms}. */ public interface PValue extends POutput, PInput { /** - * Returns the name of this {@code PValue}. + * Returns the name of this {@link PValue}. */ public String getName(); /** - * Returns the {@code AppliedPTransform} that this {@code POutputValueBase} - * is an output of. + * Returns the {@link AppliedPTransform} that this {@link PValue} is an output of. * *

For internal use only. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValueBase.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValueBase.java index 99cb08c7b2468..7e57204f33060 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValueBase.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/PValueBase.java @@ -24,26 +24,24 @@ import java.util.Collections; /** - * A {@code PValueBase} is an abstract base class that provides + * A {@link PValueBase} is an abstract base class that provides * sensible default implementations for methods of {@link PValue}. * In particular, this includes functionality for getting/setting: * *

    - *
  • The {@code Pipeline} that the {@code PValue} is - * part of. - *
  • Whether the {@code PValue} has bee finalized (as an input - * or an output), after which its properties can - * no longer be changed. + *
  • The {@link Pipeline} that the {@link PValue} is part of.
  • + *
  • Whether the {@link PValue} has bee finalized (as an input + * or an output), after which its properties can no longer be changed.
  • *
* *

For internal use. */ public abstract class PValueBase extends POutputValueBase implements PValue { /** - * Returns the name of this {@code PValueBase}. + * Returns the name of this {@link PValueBase}. * - *

By default, the name of a {@code PValueBase} is based on the - * name of the {@code PTransform} that produces it. It can be + *

By default, the name of a {@link PValueBase} is based on the + * name of the {@link PTransform} that produces it. It can be * specified explicitly by calling {@link #setName}. * * @throws IllegalStateException if the name hasn't been set yet @@ -57,11 +55,10 @@ public String getName() { } /** - * Sets the name of this {@code PValueBase}. Returns {@code this}. + * Sets the name of this {@link PValueBase}. Returns {@code this}. * - * @throws IllegalStateException if this {@code PValueBase} has - * already been finalized and is no longer settable, e.g., by having - * {@code apply()} called on it + * @throws IllegalStateException if this {@link PValueBase} has + * already been finalized and may no longer be set. */ public PValueBase setName(String name) { if (finishedSpecifying) { @@ -80,7 +77,7 @@ protected PValueBase(Pipeline pipeline) { /** * No-arg constructor for Java serialization only. - * The resulting {@code PValueBase} is unlikely to be + * The resulting {@link PValueBase} is unlikely to be * valid. */ protected PValueBase() { @@ -88,12 +85,12 @@ protected PValueBase() { } /** - * The name of this {@code PValueBase}, or null if not yet set. + * The name of this {@link PValueBase}, or null if not yet set. */ private String name; /** - * Whether this {@code PValueBase} has been finalized, and its core + * Whether this {@link PValueBase} has been finalized, and its core * properties, e.g., name, can no longer be changed. */ private boolean finishedSpecifying = false; @@ -104,9 +101,9 @@ public void recordAsOutput(AppliedPTransform transform) { } /** - * Records that this {@code POutputValueBase} is an output with the - * given name of the given {@code AppliedPTransform} in the given - * {@code Pipeline}. + * Records that this {@link POutputValueBase} is an output with the + * given name of the given {@link AppliedPTransform} in the given + * {@link Pipeline}. * *

To be invoked only by {@link POutput#recordAsOutput} * implementations. Not to be invoked directly by user code. @@ -120,7 +117,7 @@ protected void recordAsOutput(AppliedPTransform transform, } /** - * Returns whether this {@code PValueBase} has been finalized, and + * Returns whether this {@link PValueBase} has been finalized, and * its core properties, e.g., name, can no longer be changed. * *

For internal use only. @@ -147,11 +144,10 @@ public String toString() { } /** - * Returns a {@code String} capturing the kind of this - * {@code PValueBase}. + * Returns a {@link String} capturing the kind of this + * {@link PValueBase}. * - *

By default, uses the base name of this {@code PValueBase}'s - * class as its kind string. + *

By default, uses the base name of the current class as its kind string. */ protected String getKindString() { return StringUtils.approximateSimpleName(getClass()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java index 58447933059da..1085d44b135c5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TimestampedValue.java @@ -37,10 +37,10 @@ import java.util.Objects; /** - * An immutable (value, timestamp) pair. + * An immutable pair of a value and a timestamp. * - *

Used for assigning initial timestamps to values inserted into a pipeline - * with {@link com.google.cloud.dataflow.sdk.transforms.Create#timestamped}. + *

The timestamp of a value determines many properties, such as its assignment to + * windows and whether the value is late (with respect to the watermark of a {@link PCollection}). * * @param the type of the value */ @@ -83,7 +83,7 @@ public String toString() { ///////////////////////////////////////////////////////////////////////////// /** - * A {@link Coder} for {@code TimestampedValue}. + * A {@link Coder} for {@link TimestampedValue}. */ public static class TimestampedValueCoder extends StandardCoder> { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java index e52de62319d75..74949211325c7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTag.java @@ -31,16 +31,16 @@ import java.util.Random; /** - * A {@code TupleTag} is a typed tag to use as the key of a + * A {@link TupleTag} is a typed tag to use as the key of a * heterogeneously typed tuple, like {@link PCollectionTuple}. * Its generic type parameter allows tracking * the static type of things stored in tuples. * - *

To aid in assigning default {@code Coder}s for results of - * side outputs of {@code ParDo}, an output - * {@code TupleTag} should be instantiated with an extra {@code {}} so + *

To aid in assigning default {@link Coder Coders} for results of + * side outputs of {@link ParDo}, an output + * {@link TupleTag} should be instantiated with an extra {@code {}} so * it is an instance of an anonymous subclass without generic type - * parameters. Input {@code TupleTag}s require no such extra + * parameters. Input {@link TupleTag TupleTags} require no such extra * instantiation (although it doesn't hurt). For example: * *

 {@code
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTagList.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTagList.java
index b7e7ae26a249a..f019fc26e4cb0 100644
--- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTagList.java
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TupleTagList.java
@@ -16,6 +16,7 @@
 
 package com.google.cloud.dataflow.sdk.values;
 
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
 import com.google.common.collect.ImmutableList;
 
 import java.io.Serializable;
@@ -24,12 +25,12 @@
 import java.util.List;
 
 /**
- * A {@code TupleTagList} is an immutable list of heterogeneously
- * typed {@link TupleTag}s.  A TupleTagList is used, for instance, to
+ * A {@link TupleTagList} is an immutable list of heterogeneously
+ * typed {@link TupleTag TupleTags}. A {@link TupleTagList} is used, for instance, to
  * specify the tags of the side outputs of a
- * {@link com.google.cloud.dataflow.sdk.transforms.ParDo}.
+ * {@link ParDo}.
  *
- * 

TupleTagLists can be created and accessed like follows: + *

A {@link TupleTagList} can be created and accessed like follows: *

 {@code
  * TupleTag tag1 = ...;
  * TupleTag tag2 = ...;
@@ -53,9 +54,9 @@
  */
 public class TupleTagList implements Serializable {
   /**
-   * Returns an empty TupleTagList.
+   * Returns an empty {@link TupleTagList}.
    *
-   * 

Longer TupleTagLists can be created by calling + *

Longer {@link TupleTagList TupleTagLists} can be created by calling * {@link #and} on the result. */ public static TupleTagList empty() { @@ -63,9 +64,9 @@ public static TupleTagList empty() { } /** - * Returns a singleton TupleTagList containing the given TupleTag. + * Returns a singleton {@link TupleTagList} containing the given {@link TupleTag}. * - *

Longer TupleTagLists can be created by calling + *

Longer {@link TupleTagList TupleTagLists} can be created by calling * {@link #and} on the result. */ public static TupleTagList of(TupleTag tag) { @@ -73,9 +74,9 @@ public static TupleTagList of(TupleTag tag) { } /** - * Returns a TupleTagList containing the given TupleTags, in order. + * Returns a {@link TupleTagList} containing the given {@link TupleTag TupleTags}, in order. * - *

Longer TupleTagLists can be created by calling + *

Longer {@link TupleTagList TupleTagLists} can be created by calling * {@link #and} on the result. */ public static TupleTagList of(List> tags) { @@ -83,8 +84,8 @@ public static TupleTagList of(List> tags) { } /** - * Returns a new TupleTagList that has all the TupleTags of - * this TupleTagList plus the given TupleTag appended to the end. + * Returns a new {@link TupleTagList} that has all the {@link TupleTag TupleTags} of + * this {@link TupleTagList} plus the given {@link TupleTag} appended to the end. */ public TupleTagList and(TupleTag tag) { return new TupleTagList( @@ -95,8 +96,8 @@ public TupleTagList and(TupleTag tag) { } /** - * Returns a new TupleTagList that has all the TupleTags of - * this TupleTagList plus the given TupleTags appended to the end, + * Returns a new {@link TupleTagList} that has all the {@link TupleTag TupleTags} of + * this {@link TupleTagList} plus the given {@link TupleTag TupleTags} appended to the end, * in order. */ public TupleTagList and(List> tags) { @@ -115,8 +116,9 @@ public int size() { } /** - * Returns the TupleTag at the given index (origin zero). Throws - * IndexOutOfBounds if the index is out of the range + * Returns the {@link TupleTag} at the given index (origin zero). + * + * @throws IndexOutOfBoundsException if the index is out of the range * {@code [0..size()-1]}. */ public TupleTag get(int index) { @@ -124,7 +126,7 @@ public TupleTag get(int index) { } /** - * Returns an immutable List of all the TupleTags in this TupleTagList. + * Returns an immutable List of all the {@link TupleTag TupleTags} in this {@link TupleTagList}. */ public List> getAll() { return tupleTags; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypeDescriptor.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypeDescriptor.java index 09008c69fcee6..47d2cd5a5f96d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypeDescriptor.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypeDescriptor.java @@ -49,7 +49,7 @@ public abstract class TypeDescriptor implements Serializable { private final TypeToken token; /** - * Creates a TypeDescriptor wrapping the provided token. + * Creates a {@link TypeDescriptor} wrapping the provided token. * This constructor is private so Guava types do not leak. */ private TypeDescriptor(TypeToken token) { @@ -93,14 +93,14 @@ public static TypeDescriptor of(Type type) { } /** - * Returns the {@code Type} represented by this {@link TypeDescriptor}. + * Returns the {@link Type} represented by this {@link TypeDescriptor}. */ public Type getType() { return token.getType(); } /** - * Returns the {@code Class} underlying the {@code Type} represented by + * Returns the {@link Class} underlying the {@link Type} represented by * this {@link TypeDescriptor}. */ public Class getRawType() { @@ -130,8 +130,8 @@ public final boolean isArray() { } /** - * Returns a {@code TypeVariable} for the named type parameter. Throws - * {@code IllegalArgumentException} if a type variable by the requested type parameter is not + * Returns a {@link TypeVariable} for the named type parameter. Throws + * {@link IllegalArgumentException} if a type variable by the requested type parameter is not * found. * *

For example, {@code new TypeDescriptor(){}.getTypeParameter("T")} returns a @@ -186,7 +186,7 @@ public List> getArgumentTypes(Method method) { } /** - * Returns a {@code TypeDescriptor} representing the given + * Returns a {@link TypeDescriptor} representing the given * type, with type variables resolved according to the specialization * in this type. * diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypedPValue.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypedPValue.java index 433a6baba38fd..9b210b20b265a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypedPValue.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/TypedPValue.java @@ -26,21 +26,22 @@ import com.google.cloud.dataflow.sdk.transforms.ParDo; /** - * A {@code TypedPValue} is the abstract base class of things that - * store some number of values of type {@code T}. Because we know - * the type {@code T}, this is the layer of the inheritance hierarchy where - * we store a coder for objects of type {@code T} + * A {@link TypedPValue TypedPValue<T>} is the abstract base class of things that + * store some number of values of type {@code T}. * - * @param the type of the values stored in this {@code TypedPValue} + *

Because we know the type {@code T}, this is the layer of the inheritance hierarchy where + * we store a coder for objects of type {@code T}. + * + * @param the type of the values stored in this {@link TypedPValue} */ public abstract class TypedPValue extends PValueBase implements PValue { /** - * Returns the Coder used by this TypedPValue to encode and decode + * Returns the {@link Coder} used by this {@link TypedPValue} to encode and decode * the values stored in it. * - * @throws IllegalStateException if the Coder hasn't been set, and - * couldn't be inferred + * @throws IllegalStateException if the {@link Coder} hasn't been set, and + * couldn't be inferred. */ public Coder getCoder() { if (coder == null) { @@ -50,10 +51,10 @@ public Coder getCoder() { } /** - * Sets the Coder used by this TypedPValue to encode and decode the - * values stored in it. Returns {@code this}. + * Sets the {@link Coder} used by this {@link TypedPValue} to encode and decode the + * values stored in it. Returns {@code this}. * - * @throws IllegalStateException if this TypedPValue has already + * @throws IllegalStateException if this {@link TypedPValue} has already * been finalized and is no longer settable, e.g., by having * {@code apply()} called on it */ @@ -71,10 +72,10 @@ public TypedPValue setCoder(Coder coder) { } /** - * After building, finalizes this PValue to make it ready for - * running. Automatically invoked whenever the PValue is "used" + * After building, finalizes this {@link PValue} to make it ready for + * running. Automatically invoked whenever the {@link PValue} is "used" * (e.g., when apply() is called on it) and when the Pipeline is - * run (useful if this is a PValue with no consumers). + * run (useful if this is a {@link PValue} with no consumers). */ @Override public void finishSpecifying() { @@ -88,7 +89,7 @@ public void finishSpecifying() { // Internal details below here. /** - * The Coder used by this TypedPValue to encode and decode the + * The {@link Coder} used by this {@link TypedPValue} to encode and decode the * values stored in it, or null if not specified nor inferred yet. */ private Coder coder; @@ -100,7 +101,7 @@ protected TypedPValue(Pipeline p) { private TypeDescriptor typeDescriptor; /** - * Returns a {@code TypeDescriptor} with some reflective information + * Returns a {@link TypeDescriptor TypeDescriptor<T>} with some reflective information * about {@code T}, if possible. May return {@code null} if no information * is available. Subclasses may override this to enable better * {@code Coder} inference. @@ -110,8 +111,8 @@ public TypeDescriptor getTypeDescriptor() { } /** - * Sets the {@code TypeDescriptor} associated with this class. Better - * reflective type information will lead to better {@code Coder} + * Sets the {@link TypeDescriptor TypeDescriptor<T>} associated with this class. Better + * reflective type information will lead to better {@link Coder} * inference. */ public TypedPValue setTypeDescriptorInternal(TypeDescriptor typeDescriptor) { @@ -121,8 +122,8 @@ public TypedPValue setTypeDescriptorInternal(TypeDescriptor typeDescriptor /** * If the coder is not explicitly set, this sets the coder for - * this {@code TypedPValue} to the best coder that can be inferred - * based upon the known {@code TypeDescriptor}. By default, this is null, + * this {@link TypedPValue} to the best coder that can be inferred + * based upon the known {@link TypeDescriptor}. By default, this is null, * but can and should be improved by subclasses. */ @SuppressWarnings({"unchecked", "rawtypes"}) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/package-info.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/package-info.java index 6a3bf003bcce0..b8ca756f0ab45 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/package-info.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/values/package-info.java @@ -18,25 +18,35 @@ * Defines {@link com.google.cloud.dataflow.sdk.values.PCollection} and other classes for * representing data in a {@link com.google.cloud.dataflow.sdk.Pipeline}. * - *

A {@link com.google.cloud.dataflow.sdk.values.PCollection} is an immutable collection of - * values of type {@code T} and is the main representation for data. - * A {@link com.google.cloud.dataflow.sdk.values.PCollectionTuple} is a tuple of PCollections - * used in cases where PTransforms take or return multiple PCollections. - * - *

A {@link com.google.cloud.dataflow.sdk.values.PCollectionTuple} is an immutable tuple of - * heterogeneously-typed {@link com.google.cloud.dataflow.sdk.values.PCollection}s, "keyed" by - * {@link com.google.cloud.dataflow.sdk.values.TupleTag}s. - * A PCollectionTuple can be used as the input or - * output of a - * {@link com.google.cloud.dataflow.sdk.transforms.PTransform} taking - * or producing multiple PCollection inputs or outputs that can be of - * different types, for instance a - * {@link com.google.cloud.dataflow.sdk.transforms.ParDo} with side - * outputs. - * - *

A {@link com.google.cloud.dataflow.sdk.values.PCollectionView} is an immutable view of a - * PCollection that can be accessed from a DoFn and other user Fns - * as a side input. + *

In particular, see these collection abstractions: * + *

    + *
  • {@link com.google.cloud.dataflow.sdk.values.PCollection} - an immutable collection of + * values of type {@code T} and the main representation for data in Dataflow.
  • + *
  • {@link com.google.cloud.dataflow.sdk.values.PCollectionView} - an immutable view of a + * {@link com.google.cloud.dataflow.sdk.values.PCollection} that can be accessed as a + * side input of a {@link com.google.cloud.dataflow.sdk.transforms.ParDo} + * {@link com.google.cloud.dataflow.sdk.transforms.PTransform}.
  • + *
  • {@link com.google.cloud.dataflow.sdk.values.PCollectionTuple} - a heterogeneous tuple of + * {@link com.google.cloud.dataflow.sdk.values.PCollection PCollections} + * used in cases where a {@link com.google.cloud.dataflow.sdk.transforms.PTransform} takes + * or returns multiple + * {@link com.google.cloud.dataflow.sdk.values.PCollection PCollections}.
  • + *
  • {@link com.google.cloud.dataflow.sdk.values.PCollectionList} - a homogeneous list of + * {@link com.google.cloud.dataflow.sdk.values.PCollection PCollections} used, for example, + * as input to {@link com.google.cloud.dataflow.sdk.transforms.Flatten}.
  • + *
+ * + *

And these classes for individual values play particular roles in Dataflow: + * + *

    + *
  • {@link com.google.cloud.dataflow.sdk.values.KV} - a key/value pair that is used by + * keyed transforms, most notably {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey}. + *
  • + *
  • {@link com.google.cloud.dataflow.sdk.values.TimestampedValue} - a timestamp/value pair + * that is used for windowing and handling out-of-order data in streaming execution.
  • + *
+ * + *

For further details, see the documentation for each class in this package. */ package com.google.cloud.dataflow.sdk.values; From 7f7f377251f81249ee3ac863d210c9efefac76c1 Mon Sep 17 00:00:00 2001 From: hdeist Date: Tue, 22 Dec 2015 11:18:44 -0800 Subject: [PATCH 1250/1541] Add more tests to DoFnTesterTest ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110785121 --- .../sdk/transforms/DoFnTesterTest.java | 136 ++++++++++++++++-- 1 file changed, 126 insertions(+), 10 deletions(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/DoFnTesterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/DoFnTesterTest.java index f2cb40c2e84f8..32b38acffbd5f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/DoFnTesterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/DoFnTesterTest.java @@ -15,23 +15,127 @@ */ package com.google.cloud.dataflow.sdk.transforms; +import static org.hamcrest.CoreMatchers.hasItems; import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import java.util.List; + /** * Tests for {@link DoFnTester}. */ @RunWith(JUnit4.class) public class DoFnTesterTest { + @Test - public void getAggregatorValuesShouldGetValueOfCounter() { + public void processElement() { CounterDoFn counterDoFn = new CounterDoFn(); + DoFnTester tester = DoFnTester.of(counterDoFn); + + tester.processElement(1L); - DoFnTester tester = DoFnTester.of(counterDoFn); + List take = tester.takeOutputElements(); + + assertThat(take, hasItems("1")); + + // Following takeOutputElements(), neither takeOutputElements() + // nor peekOutputElements() return anything. + assertTrue(tester.takeOutputElements().isEmpty()); + assertTrue(tester.peekOutputElements().isEmpty()); + + // processElement() caused startBundle() to be called, but finishBundle() was never called. + CounterDoFn deserializedDoFn = (CounterDoFn) tester.fn; + assertTrue(deserializedDoFn.wasStartBundleCalled()); + assertFalse(deserializedDoFn.wasFinishBundleCalled()); + } + + @Test + public void processElementsWithPeeks() { + CounterDoFn counterDoFn = new CounterDoFn(); + DoFnTester tester = DoFnTester.of(counterDoFn); + + // Explicitly call startBundle(). + tester.startBundle(); + + // verify startBundle() was called but not finishBundle(). + CounterDoFn deserializedDoFn = (CounterDoFn) tester.fn; + assertTrue(deserializedDoFn.wasStartBundleCalled()); + assertFalse(deserializedDoFn.wasFinishBundleCalled()); + + // process a couple of elements. + tester.processElement(1L); + tester.processElement(2L); + + // peek the first 2 outputs. + List peek = tester.peekOutputElements(); + assertThat(peek, hasItems("1", "2")); + + // process a couple more. + tester.processElement(3L); + tester.processElement(4L); + + // peek all the outputs so far. + peek = tester.peekOutputElements(); + assertThat(peek, hasItems("1", "2", "3", "4")); + // take the outputs. + List take = tester.takeOutputElements(); + assertThat(take, hasItems("1", "2", "3", "4")); + + // Following takeOutputElements(), neither takeOutputElements() + // nor peekOutputElements() return anything. + assertTrue(tester.peekOutputElements().isEmpty()); + assertTrue(tester.takeOutputElements().isEmpty()); + + // verify finishBundle() hasn't been called yet. + assertTrue(deserializedDoFn.wasStartBundleCalled()); + assertFalse(deserializedDoFn.wasFinishBundleCalled()); + + // process a couple more. + tester.processElement(5L); + tester.processElement(6L); + + // peek and take now have only the 2 last outputs. + peek = tester.peekOutputElements(); + assertThat(peek, hasItems("5", "6")); + take = tester.takeOutputElements(); + assertThat(take, hasItems("5", "6")); + + tester.finishBundle(); + + // verify finishBundle() was called. + assertTrue(deserializedDoFn.wasStartBundleCalled()); + assertTrue(deserializedDoFn.wasFinishBundleCalled()); + } + + @Test + public void processBatch() { + CounterDoFn counterDoFn = new CounterDoFn(); + DoFnTester tester = DoFnTester.of(counterDoFn); + + // processBatch() returns all the output like takeOutputElements(). + List take = tester.processBatch(1L, 2L, 3L, 4L); + + assertThat(take, hasItems("1", "2", "3", "4")); + + // peek now returns nothing. + assertTrue(tester.peekOutputElements().isEmpty()); + + // verify startBundle() and finishBundle() were both called. + CounterDoFn deserializedDoFn = (CounterDoFn) tester.fn; + assertTrue(deserializedDoFn.wasStartBundleCalled()); + assertTrue(deserializedDoFn.wasFinishBundleCalled()); + } + + @Test + public void getAggregatorValuesShouldGetValueOfCounter() { + CounterDoFn counterDoFn = new CounterDoFn(); + DoFnTester tester = DoFnTester.of(counterDoFn); tester.processBatch(1L, 2L, 4L, 8L); Long aggregatorVal = tester.getAggregatorValue(counterDoFn.agg); @@ -42,8 +146,7 @@ public void getAggregatorValuesShouldGetValueOfCounter() { @Test public void getAggregatorValuesWithEmptyCounterShouldSucceed() { CounterDoFn counterDoFn = new CounterDoFn(); - - DoFnTester tester = DoFnTester.of(counterDoFn); + DoFnTester tester = DoFnTester.of(counterDoFn); tester.processBatch(); Long aggregatorVal = tester.getAggregatorValue(counterDoFn.agg); // empty bundle @@ -53,7 +156,7 @@ public void getAggregatorValuesWithEmptyCounterShouldSucceed() { @Test public void getAggregatorValuesInStartFinishBundleShouldGetValues() { CounterDoFn fn = new CounterDoFn(1L, 2L); - DoFnTester tester = DoFnTester.of(fn); + DoFnTester tester = DoFnTester.of(fn); tester.processBatch(0L, 0L); Long aggValue = tester.getAggregatorValue(fn.agg); @@ -61,12 +164,14 @@ public void getAggregatorValuesInStartFinishBundleShouldGetValues() { } /** - * A DoFn that adds values to an aggregator in processElement. + * A DoFn that adds values to an aggregator and converts input to String in processElement. */ - private static class CounterDoFn extends DoFn { + private static class CounterDoFn extends DoFn { Aggregator agg = createAggregator("ctr", new Sum.SumLongFn()); private final long startBundleVal; private final long finishBundleVal; + private boolean startBundleCalled; + private boolean finishBundleCalled; public CounterDoFn() { this(0L, 0L); @@ -78,18 +183,29 @@ public CounterDoFn(long start, long finish) { } @Override - public void startBundle(DoFn.Context c) { + public void startBundle(Context c) { agg.addValue(startBundleVal); + startBundleCalled = true; } @Override - public void processElement(DoFn.ProcessContext c) throws Exception { + public void processElement(ProcessContext c) throws Exception { agg.addValue(c.element()); + c.output(c.element().toString()); } @Override - public void finishBundle(DoFn.Context c) { + public void finishBundle(Context c) { agg.addValue(finishBundleVal); + finishBundleCalled = true; + } + + boolean wasStartBundleCalled() { + return startBundleCalled; + } + + boolean wasFinishBundleCalled() { + return finishBundleCalled; } } } From d23aff317d275fb0c3ca491b9c3c91fcd6f0476a Mon Sep 17 00:00:00 2001 From: sgmc Date: Tue, 22 Dec 2015 13:14:46 -0800 Subject: [PATCH 1251/1541] Add OAuth2 addresses to GcpOptions ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=110793208 --- .../dataflow/sdk/options/GcpOptions.java | 23 +++++++++++++++++++ .../cloud/dataflow/sdk/util/Credentials.java | 3 +++ 2 files changed, 26 insertions(+) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java index ef36cda04ae09..1ab544809b916 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/GcpOptions.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.options; import com.google.api.client.auth.oauth2.Credential; +import com.google.api.client.googleapis.auth.oauth2.GoogleOAuthConstants; import com.google.cloud.dataflow.sdk.util.CredentialFactory; import com.google.cloud.dataflow.sdk.util.GcpCredentialFactory; import com.google.cloud.dataflow.sdk.util.InstanceBuilder; @@ -259,4 +260,26 @@ public Credential create(PipelineOptions options) { } } } + + /** + * The token server URL to use for OAuth 2 authentication. Normally, the default is sufficient, + * but some specialized use cases may want to override this value. + */ + @Description("The token server URL to use for OAuth 2 authentication. Normally, the default " + + "is sufficient, but some specialized use cases may want to override this value.") + @Default.String(GoogleOAuthConstants.TOKEN_SERVER_URL) + @Hidden + String getTokenServerUrl(); + void setTokenServerUrl(String value); + + /** + * The authorization server URL to use for OAuth 2 authentication. Normally, the default is + * sufficient, but some specialized use cases may want to override this value. + */ + @Description("The authorization server URL to use for OAuth 2 authentication. Normally, the " + + "default is sufficient, but some specialized use cases may want to override this value.") + @Default.String(GoogleOAuthConstants.AUTHORIZATION_SERVER_URL) + @Hidden + String getAuthorizationServerEncodedUrl(); + void setAuthorizationServerEncodedUrl(String value); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java index 1af539163bdf8..671b131554ead 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/Credentials.java @@ -24,6 +24,7 @@ import com.google.api.client.googleapis.auth.oauth2.GoogleCredential; import com.google.api.client.googleapis.auth.oauth2.GoogleOAuthConstants; import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport; +import com.google.api.client.http.GenericUrl; import com.google.api.client.http.HttpTransport; import com.google.api.client.json.JsonFactory; import com.google.api.client.json.jackson2.JacksonFactory; @@ -175,6 +176,8 @@ private static Credential getCredentialFromClientSecrets( GoogleAuthorizationCodeFlow flow = new GoogleAuthorizationCodeFlow.Builder( httpTransport, jsonFactory, clientSecrets, scopes) .setDataStoreFactory(dataStoreFactory) + .setTokenServerUrl(new GenericUrl(options.getTokenServerUrl())) + .setAuthorizationServerEncodedUrl(options.getAuthorizationServerEncodedUrl()) .build(); // The credentialId identifies the credential if we're using a persistent From 2cec5a7e4e7acd0c3e09723f29081899c07c61be Mon Sep 17 00:00:00 2001 From: lcwik Date: Mon, 28 Dec 2015 16:45:38 -0800 Subject: [PATCH 1252/1541] Make ParDo.withSideInputs cumulative Previously when building a ParDo, withSideInputs would repace the currently built set of side inputs with the newly passed in set. This change makes the builder cumulative so ParDo.withSideInputs(A).withSideInputs(B) is equivalent to ParDo.withSideInputs({A, B}). ----Release Notes---- Modified ParDo.withSideInputs such that successive calls are cumulative. [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111039573 --- .../cloud/dataflow/sdk/transforms/ParDo.java | 34 ++++-- .../dataflow/sdk/transforms/ParDoTest.java | 112 +++++++++++++++++- 2 files changed, 134 insertions(+), 12 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java index 245dae01a7123..43644d4e737db 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java @@ -607,7 +607,7 @@ public Unbound named(String name) { /** * Returns a new {@link ParDo} transform that's like this - * transform but with the specified side inputs. + * transform but with the specified additional side inputs. * Does not modify this transform. The resulting transform is * still incomplete. * @@ -620,7 +620,7 @@ public Unbound withSideInputs(PCollectionView... sideInputs) { /** * Returns a new {@link ParDo} transform that is like this - * transform but with the specified side inputs. Does not modify + * transform but with the specified additional side inputs. Does not modify * this transform. The resulting transform is still incomplete. * *

See the discussion of Side Inputs above and on @@ -628,7 +628,10 @@ public Unbound withSideInputs(PCollectionView... sideInputs) { */ public Unbound withSideInputs( Iterable> sideInputs) { - return new Unbound(name, ImmutableList.copyOf(sideInputs)); + ImmutableList.Builder> builder = ImmutableList.builder(); + builder.addAll(this.sideInputs); + builder.addAll(sideInputs); + return new Unbound(name, builder.build()); } /** @@ -710,7 +713,7 @@ public Bound named(String name) { /** * Returns a new {@link ParDo} {@link PTransform} that's like this - * {@link PTransform} but with the specified side inputs. Does not + * {@link PTransform} but with the specified additional side inputs. Does not * modify this {@link PTransform}. * *

See the discussion of Side Inputs above and on @@ -722,7 +725,7 @@ public Bound withSideInputs(PCollectionView... sideInputs) { /** * Returns a new {@link ParDo} {@link PTransform} that's like this - * {@link PTransform} but with the specified side inputs. Does not + * {@link PTransform} but with the specified additional side inputs. Does not * modify this {@link PTransform}. * *

See the discussion of Side Inputs above and on @@ -730,7 +733,10 @@ public Bound withSideInputs(PCollectionView... sideInputs) { */ public Bound withSideInputs( Iterable> sideInputs) { - return new Bound<>(name, ImmutableList.copyOf(sideInputs), fn); + ImmutableList.Builder> builder = ImmutableList.builder(); + builder.addAll(this.sideInputs); + builder.addAll(sideInputs); + return new Bound<>(name, builder.build(), fn); } /** @@ -839,7 +845,7 @@ public UnboundMulti withSideInputs( /** * Returns a new multi-output {@link ParDo} transform that's like - * this transform but with the specified side inputs. Does not + * this transform but with the specified additional side inputs. Does not * modify this transform. The resulting transform is still * incomplete. * @@ -848,8 +854,11 @@ public UnboundMulti withSideInputs( */ public UnboundMulti withSideInputs( Iterable> sideInputs) { + ImmutableList.Builder> builder = ImmutableList.builder(); + builder.addAll(this.sideInputs); + builder.addAll(sideInputs); return new UnboundMulti<>( - name, ImmutableList.copyOf(sideInputs), + name, builder.build(), mainOutputTag, sideOutputTags); } @@ -924,7 +933,7 @@ public BoundMulti named(String name) { /** * Returns a new multi-output {@link ParDo} {@link PTransform} - * that's like this {@link PTransform} but with the specified side + * that's like this {@link PTransform} but with the specified additional side * inputs. Does not modify this {@link PTransform}. * *

See the discussion of Side Inputs above and on @@ -937,7 +946,7 @@ public BoundMulti withSideInputs( /** * Returns a new multi-output {@link ParDo} {@link PTransform} - * that's like this {@link PTransform} but with the specified side + * that's like this {@link PTransform} but with the specified additional side * inputs. Does not modify this {@link PTransform}. * *

See the discussion of Side Inputs above and on @@ -945,8 +954,11 @@ public BoundMulti withSideInputs( */ public BoundMulti withSideInputs( Iterable> sideInputs) { + ImmutableList.Builder> builder = ImmutableList.builder(); + builder.addAll(this.sideInputs); + builder.addAll(sideInputs); return new BoundMulti<>( - name, ImmutableList.copyOf(sideInputs), + name, builder.build(), mainOutputTag, sideOutputTags, fn); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java index 50afbf993418e..f3f9bde92d6d1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java @@ -600,7 +600,7 @@ public void testParDoWithSideInputs() { PCollectionView sideInputUnread = pipeline .apply("CreateSideInputUnread", Create.of(-3333)) .apply("ViewSideInputUnread", View.asSingleton()); - PCollectionView sideInput2 = pipeline + PCollectionView sideInput2 = pipeline .apply("CreateSideInput2", Create.of(222)) .apply("ViewSideInput2", View.asSingleton()); @@ -619,6 +619,116 @@ public void testParDoWithSideInputs() { pipeline.run(); } + @Test + @Category(RunnableOnService.class) + public void testParDoWithSideInputsIsCumulative() { + Pipeline pipeline = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + PCollectionView sideInput1 = pipeline + .apply("CreateSideInput1", Create.of(11)) + .apply("ViewSideInput1", View.asSingleton()); + PCollectionView sideInputUnread = pipeline + .apply("CreateSideInputUnread", Create.of(-3333)) + .apply("ViewSideInputUnread", View.asSingleton()); + PCollectionView sideInput2 = pipeline + .apply("CreateSideInput2", Create.of(222)) + .apply("ViewSideInput2", View.asSingleton()); + + PCollection output = pipeline + .apply(Create.of(inputs)) + .apply(ParDo.withSideInputs(sideInput1) + .withSideInputs(sideInputUnread) + .withSideInputs(sideInput2) + .of(new TestDoFn( + Arrays.asList(sideInput1, sideInput2), + Arrays.>asList()))); + + DataflowAssert.that(output) + .satisfies(ParDoTest.HasExpectedOutput + .forInput(inputs) + .andSideInputs(11, 222)); + + pipeline.run(); + } + + @Test + @Category(RunnableOnService.class) + public void testMultiOutputParDoWithSideInputs() { + Pipeline pipeline = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + final TupleTag mainOutputTag = new TupleTag("main"){}; + final TupleTag sideOutputTag = new TupleTag("sideOutput"){}; + + PCollectionView sideInput1 = pipeline + .apply("CreateSideInput1", Create.of(11)) + .apply("ViewSideInput1", View.asSingleton()); + PCollectionView sideInputUnread = pipeline + .apply("CreateSideInputUnread", Create.of(-3333)) + .apply("ViewSideInputUnread", View.asSingleton()); + PCollectionView sideInput2 = pipeline + .apply("CreateSideInput2", Create.of(222)) + .apply("ViewSideInput2", View.asSingleton()); + + PCollectionTuple outputs = pipeline + .apply(Create.of(inputs)) + .apply(ParDo.withSideInputs(sideInput1) + .withSideInputs(sideInputUnread) + .withSideInputs(sideInput2) + .withOutputTags(mainOutputTag, TupleTagList.of(sideOutputTag)) + .of(new TestDoFn( + Arrays.asList(sideInput1, sideInput2), + Arrays.>asList()))); + + DataflowAssert.that(outputs.get(mainOutputTag)) + .satisfies(ParDoTest.HasExpectedOutput + .forInput(inputs) + .andSideInputs(11, 222)); + + pipeline.run(); + } + + @Test + @Category(RunnableOnService.class) + public void testMultiOutputParDoWithSideInputsIsCumulative() { + Pipeline pipeline = TestPipeline.create(); + + List inputs = Arrays.asList(3, -42, 666); + + final TupleTag mainOutputTag = new TupleTag("main"){}; + final TupleTag sideOutputTag = new TupleTag("sideOutput"){}; + + PCollectionView sideInput1 = pipeline + .apply("CreateSideInput1", Create.of(11)) + .apply("ViewSideInput1", View.asSingleton()); + PCollectionView sideInputUnread = pipeline + .apply("CreateSideInputUnread", Create.of(-3333)) + .apply("ViewSideInputUnread", View.asSingleton()); + PCollectionView sideInput2 = pipeline + .apply("CreateSideInput2", Create.of(222)) + .apply("ViewSideInput2", View.asSingleton()); + + PCollectionTuple outputs = pipeline + .apply(Create.of(inputs)) + .apply(ParDo.withSideInputs(sideInput1) + .withSideInputs(sideInputUnread) + .withSideInputs(sideInput2) + .withOutputTags(mainOutputTag, TupleTagList.of(sideOutputTag)) + .of(new TestDoFn( + Arrays.asList(sideInput1, sideInput2), + Arrays.>asList()))); + + DataflowAssert.that(outputs.get(mainOutputTag)) + .satisfies(ParDoTest.HasExpectedOutput + .forInput(inputs) + .andSideInputs(11, 222)); + + pipeline.run(); + } + @Test public void testParDoReadingFromUnknownSideInput() { Pipeline pipeline = TestPipeline.create(); From 441cbe22eb8b399eba6990793e1f5efa9d0c169e Mon Sep 17 00:00:00 2001 From: klk Date: Tue, 29 Dec 2015 11:45:34 -0800 Subject: [PATCH 1253/1541] Tidy some worker code ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111082740 --- .../sdk/runners/worker/AssignWindowsParDoFn.java | 6 +++--- .../sdk/runners/worker/CombineValuesFn.java | 16 ++++++++-------- .../runners/worker/DefaultParDoFnFactory.java | 2 +- .../worker/GroupAlsoByWindowsParDoFn.java | 2 +- .../sdk/runners/worker/InMemoryReader.java | 2 +- .../sdk/runners/worker/NormalParDoFn.java | 4 ++-- .../sdk/runners/worker/ParDoFnFactory.java | 2 +- .../worker/ReifyTimestampAndWindowsParDoFn.java | 2 +- .../sdk/runners/worker/InMemoryReaderTest.java | 2 +- .../worker/WindmillStateInternalsTest.java | 2 ++ 10 files changed, 21 insertions(+), 19 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java index 43492792ce221..153fcf5f3085f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/AssignWindowsParDoFn.java @@ -50,7 +50,7 @@ static AssignWindowsParDoFn of( AssignWindowsDoFn fn, String stepName, String transformName, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { @@ -72,7 +72,7 @@ public ParDoFn create( @Nullable List sideInputInfos, @Nullable List multiOutputInfos, int numOutputs, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { @@ -119,7 +119,7 @@ private AssignWindowsParDoFn( AssignWindowsDoFn fn, String stepName, String transformName, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) { super( diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java index 28e42cb3fc1d2..8b7b53c6e0ba3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/CombineValuesFn.java @@ -68,7 +68,7 @@ static CombineValuesFn of( String phase, String stepName, String transformName, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { @@ -90,7 +90,7 @@ public ParDoFn create( @Nullable List sideInputInfos, @Nullable List multiOutputInfos, int numOutputs, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { @@ -126,19 +126,19 @@ public ParDoFn create( @Override protected DoFnInfo getDoFnInfo() { - DoFn doFn = null; + DoFn doFn = null; switch (phase) { case CombinePhase.ALL: - doFn = new CombineValuesDoFn(combineFn); + doFn = new CombineValuesDoFn<>(combineFn); break; case CombinePhase.ADD: - doFn = new AddInputsDoFn(combineFn); + doFn = new AddInputsDoFn<>(combineFn); break; case CombinePhase.MERGE: - doFn = new MergeAccumulatorsDoFn(combineFn); + doFn = new MergeAccumulatorsDoFn<>(combineFn); break; case CombinePhase.EXTRACT: - doFn = new ExtractOutputDoFn(combineFn); + doFn = new ExtractOutputDoFn<>(combineFn); break; default: throw new IllegalArgumentException( @@ -156,7 +156,7 @@ private CombineValuesFn( String phase, String stepName, String transformName, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) { super( diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DefaultParDoFnFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DefaultParDoFnFactory.java index 0595ee1f37d28..b11d8327a5ea5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DefaultParDoFnFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/DefaultParDoFnFactory.java @@ -55,7 +55,7 @@ public ParDoFn create( List sideInputInfos, List multiOutputInfos, int numOutputs, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java index 4bb468f290c14..8203b08b0886a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/GroupAlsoByWindowsParDoFn.java @@ -88,7 +88,7 @@ public ParDoFn create( @Nullable List sideInputInfos, @Nullable List multiOutputInfos, int numOutputs, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java index b9a60c16ee498..650719047d618 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReader.java @@ -100,7 +100,7 @@ public InMemoryReaderIterator() { @Override protected boolean hasNextImpl() { - return tracker.tryReturnRecordAt(true, (long) nextIndex); + return tracker.tryReturnRecordAt(true, nextIndex); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java index 2df5a1e8ce91d..7155580c8309d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/NormalParDoFn.java @@ -55,7 +55,7 @@ static NormalParDoFn of( List outputTags, String stepName, String transformName, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) { return new NormalParDoFn( @@ -84,7 +84,7 @@ public ParDoFn create( @Nullable List sideInputInfos, @Nullable List multiOutputInfos, int numOutputs, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactory.java index b80709f2142eb..cfb96645c79fd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ParDoFnFactory.java @@ -44,7 +44,7 @@ ParDoFn create( List sideInputInfos, List multiOutputInfos, int numOutputs, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReifyTimestampAndWindowsParDoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReifyTimestampAndWindowsParDoFn.java index c5edfefebbdaa..3e9e1a6e54f23 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReifyTimestampAndWindowsParDoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/ReifyTimestampAndWindowsParDoFn.java @@ -67,7 +67,7 @@ public ParDoFn create( @Nullable List sideInputInfos, @Nullable List multiOutputInfos, int numOutputs, - DataflowExecutionContext executionContext, + DataflowExecutionContext executionContext, CounterSet.AddCounterMutator addCounterMutator, StateSampler stateSampler) throws Exception { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java index f69d95a5d1cf6..dc02053a9c03b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/InMemoryReaderTest.java @@ -196,7 +196,7 @@ public void testParallelism() throws Exception { InMemoryReader inMemoryReader = new InMemoryReader<>(encodedElements(elements, coder), 1L, 4L, coder); int count = 0; - for (Reader.ReaderIterator iterator = inMemoryReader.iterator(); + for (Reader.ReaderIterator iterator = inMemoryReader.iterator(); iterator.hasNext(); iterator.next()) { assertTrue(iterator.getRemainingParallelism() >= 1); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java index febc376647545..58afedb55419e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/WindmillStateInternalsTest.java @@ -978,6 +978,7 @@ public void testCachedCombining() throws Exception { private void disableCompactOnWrite() { WindmillStateInternals.COMPACT_NOW.set( new Supplier() { + @Override public Boolean get() { return false; } @@ -987,6 +988,7 @@ public Boolean get() { private void forceCompactOnWrite() { WindmillStateInternals.COMPACT_NOW.set( new Supplier() { + @Override public Boolean get() { return true; } From ae47163147e12c783a2bfff027b9c21a948df232 Mon Sep 17 00:00:00 2001 From: davor Date: Tue, 29 Dec 2015 15:12:23 -0800 Subject: [PATCH 1254/1541] DataflowWorkProgressUpdaterTest: fix data races in the test itself ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111093921 --- .../DataflowWorkProgressUpdaterTest.java | 79 +++++++++---------- 1 file changed, 37 insertions(+), 42 deletions(-) diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java index a8c0315cbd22f..9f600eabb2680 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/DataflowWorkProgressUpdaterTest.java @@ -85,14 +85,17 @@ public class DataflowWorkProgressUpdaterTest { static class TestMapTaskExecutor extends MapTaskExecutor { ApproximateReportedProgress progress = null; + List> metrics = new ArrayList<>(); + CounterSet counters; public TestMapTaskExecutor(CounterSet counters) { super(new ArrayList(), counters, new StateSampler("test", counters.getAddCounterMutator())); + this.counters = counters; } @Override - public Reader.Progress getWorkerProgress() { + public synchronized Reader.Progress getWorkerProgress() { return cloudProgressToReaderProgress(progress); } @@ -107,9 +110,28 @@ public Reader.DynamicSplitResult requestDynamicSplit(Reader.DynamicSplitRequest cloudPositionToReaderPosition(split.getPosition())); } - public void setWorkerProgress(ApproximateReportedProgress progress) { + public synchronized void setWorkerProgress(ApproximateReportedProgress progress) { this.progress = progress; } + + @Override + public synchronized Collection> getOutputMetrics() { + return metrics; + } + + public synchronized void setUpMetrics(int n) { + metrics = new ArrayList<>(); + for (int i = 0; i < n; i++) { + metrics.add(makeMetric(i)); + } + } + + public synchronized void setUpCounters(int n) { + counters.clear(); + for (int i = 0; i < n; i++) { + counters.add(makeCounter(i)); + } + } } private static final String PROJECT_ID = "TEST_PROJECT_ID"; @@ -127,8 +149,6 @@ public void setWorkerProgress(ApproximateReportedProgress progress) { @Mock private DataflowWorker.WorkUnitClient workUnitClient; - private CounterSet counters; - private List> metrics; private TestMapTaskExecutor worker; private WorkItem workItem; private DataflowWorkerHarnessOptions options; @@ -143,14 +163,7 @@ public void initMocksAndWorkflowServiceAndWorkerAndWork() { options.setJobId(JOB_ID); options.setWorkerId(WORKER_ID); - metrics = new ArrayList<>(); - counters = new CounterSet(); - worker = new TestMapTaskExecutor(counters) { - @Override - public Collection> getOutputMetrics() { - return metrics; - } - }; + worker = new TestMapTaskExecutor(new CounterSet()); workItem = new WorkItem(); workItem.setProjectId(PROJECT_ID); @@ -181,9 +194,9 @@ protected long getLeaseRenewalLatencyMargin() { public void workProgressUpdaterUpdates() throws Exception { when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) .thenReturn(generateServiceState(System.currentTimeMillis() + 2000, 1000, null, 2L)); - setUpCounters(2); - setUpMetrics(3); - setUpProgress(approximateProgressAtIndex(1L)); + worker.setUpCounters(2); + worker.setUpMetrics(3); + worker.setWorkerProgress(approximateProgressAtIndex(1L)); progressUpdater.startReportingProgress(); // The initial update should be sent after 300. verify(workUnitClient, timeout(400)) @@ -197,9 +210,9 @@ public void workProgressUpdaterUpdates() throws Exception { // and actual dynamic split result. @Test(timeout = 10000) public void workProgressUpdaterAdaptsProgressInterval() throws Exception { - setUpCounters(3); - setUpMetrics(2); - setUpProgress(approximateProgressAtIndex(1L)); + worker.setUpCounters(3); + worker.setUpMetrics(2); + worker.setWorkerProgress(approximateProgressAtIndex(1L)); // In tests below, we allow 500ms leeway. @@ -216,9 +229,9 @@ public void workProgressUpdaterAdaptsProgressInterval() throws Exception { new ExpectedDataflowWorkItemStatus().withCounters(3).withMetrics(2).withProgress( approximateProgressAtIndex(1L)).withReportIndex(1L))); - setUpCounters(5); - setUpMetrics(6); - setUpProgress(approximateProgressAtIndex(2L)); + worker.setUpCounters(5); + worker.setUpMetrics(6); + worker.setWorkerProgress(approximateProgressAtIndex(2L)); when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) .thenReturn(generateServiceState(System.currentTimeMillis() + 3000, 2000, null, 3L)); // The second update should be sent after ~1000ms (previous requested report interval). @@ -233,7 +246,7 @@ public void workProgressUpdaterAdaptsProgressInterval() throws Exception { // After the request is sent, reset cached dynamic split result to null. assertNull(progressUpdater.getDynamicSplitResultToReport()); - setUpProgress(approximateProgressAtIndex(3L)); + worker.setWorkerProgress(approximateProgressAtIndex(3L)); when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) .thenReturn(generateServiceState(System.currentTimeMillis() + 1000, 3000, null, 4L)); @@ -242,7 +255,7 @@ public void workProgressUpdaterAdaptsProgressInterval() throws Exception { new ExpectedDataflowWorkItemStatus().withProgress(approximateProgressAtIndex(3L)) .withReportIndex(3L))); - setUpProgress(approximateProgressAtIndex(4L)); + worker.setWorkerProgress(approximateProgressAtIndex(4L)); when(workUnitClient.reportWorkItemStatus(any(WorkItemStatus.class))) .thenReturn(generateServiceState(System.currentTimeMillis() + 4000, 3000, null, 5L)); @@ -266,7 +279,7 @@ public void workProgressUpdaterSendsLastPendingUpdateWhenStopped() throws Except .thenReturn(generateServiceState( System.currentTimeMillis() + 2000, 1000, positionAtIndex(2L), 2L)); - setUpProgress(approximateProgressAtIndex(1L)); + worker.setWorkerProgress(approximateProgressAtIndex(1L)); progressUpdater.startReportingProgress(); // The initial update should be sent after 300 msec. @@ -298,13 +311,6 @@ public void workProgressUpdaterSendsLastPendingUpdateWhenStopped() throws Except verifyNoMoreInteractions(workUnitClient); } - private void setUpCounters(int n) { - counters.clear(); - for (int i = 0; i < n; i++) { - counters.add(makeCounter(i)); - } - } - private static Counter makeCounter(int i) { if (i % 3 == 0) { return Counter.longs(COUNTER_NAME + i, COUNTER_KINDS[0]) @@ -325,17 +331,6 @@ private static Metric makeMetric(int i) { return new DoubleMetric(String.valueOf(i), i); } - private void setUpMetrics(int n) { - metrics = new ArrayList<>(); - for (int i = 0; i < n; i++) { - metrics.add(makeMetric(i)); - } - } - - private void setUpProgress(ApproximateReportedProgress progress) { - worker.setWorkerProgress(progress); - } - private WorkItemServiceState generateServiceState(long leaseExpirationTimestamp, int progressReportIntervalMs, Position suggestedStopPosition, long nextReportIndex) { From 31a8b88b627f2bdc0eb128c4874f70175fb906ef Mon Sep 17 00:00:00 2001 From: klk Date: Tue, 29 Dec 2015 20:42:44 -0800 Subject: [PATCH 1255/1541] Tidy some utility code ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111107107 --- .../cloud/dataflow/sdk/util/DoFnRunner.java | 21 ++++++++++++------- .../cloud/dataflow/sdk/util/VarInt.java | 2 +- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java index 4a8c105edb4da..b501e90208767 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/DoFnRunner.java @@ -224,7 +224,7 @@ private static class DoFnContext final TupleTag mainOutputTag; final StepContext stepContext; final CounterSet.AddCounterMutator addCounterMutator; - final WindowFn windowFn; + final WindowFn windowFn; /** * The set of known output tags, some of which may be undeclared, so we can throw an @@ -240,7 +240,7 @@ public DoFnContext(PipelineOptions options, List> sideOutputTags, StepContext stepContext, CounterSet.AddCounterMutator addCounterMutator, - WindowFn windowFn) { + WindowFn windowFn) { fn.super(); this.options = options; this.fn = fn; @@ -267,8 +267,8 @@ public PipelineOptions getPipelineOptions() { return options; } - WindowedValue makeWindowedValue( - T output, Instant timestamp, Collection windows, PaneInfo pane) { + WindowedValue makeWindowedValue( + T output, Instant timestamp, Collection windows, PaneInfo pane) { final Instant inputTimestamp = timestamp; if (timestamp == null) { @@ -277,7 +277,11 @@ WindowedValue makeWindowedValue( if (windows == null) { try { - windows = windowFn.assignWindows(windowFn.new AssignContext() { + // The windowFn can never succeed at accessing the element, so its type does not + // matter here + @SuppressWarnings("unchecked") + WindowFn objectWindowFn = (WindowFn) windowFn; + windows = objectWindowFn.assignWindows(objectWindowFn.new AssignContext() { @Override public Object element() { throw new UnsupportedOperationException( @@ -562,7 +566,7 @@ public void writePCollectionViewData( Iterable> data, Coder elemCoder) throws IOException { @SuppressWarnings("unchecked") - Coder windowCoder = context.windowFn.windowCoder(); + Coder windowCoder = (Coder) context.windowFn.windowCoder(); context.stepContext.writePCollectionViewData( tag, data, IterableCoder.of(WindowedValue.getFullCoder(elemCoder, windowCoder)), @@ -577,8 +581,9 @@ public StateInternals stateInternals() { } @Override - protected Aggregator createAggregatorInternal(String name, - CombineFn combiner) { + protected Aggregator + createAggregatorInternal( + String name, CombineFn combiner) { return context.createAggregatorInternal(name, combiner); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/VarInt.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/VarInt.java index f2e9c8bd74088..af039112eabac 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/VarInt.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/VarInt.java @@ -32,7 +32,7 @@ public class VarInt { private static long convertIntToLongNoSignExtend(int v) { - return ((long) v) & 0xFFFFFFFFL; + return v & 0xFFFFFFFFL; } /** From c837233cee865a7fb3bf1a232edf447aa2fa8fda Mon Sep 17 00:00:00 2001 From: lcwik Date: Wed, 30 Dec 2015 13:32:55 -0800 Subject: [PATCH 1256/1541] Update the overview page for the javadoc Added relevant links to various docs and talked about the @Experimental API ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111148749 --- javadoc/overview.html | 31 +++++++++++++++++++++++++++++++ sdk/pom.xml | 4 ++-- 2 files changed, 33 insertions(+), 2 deletions(-) create mode 100644 javadoc/overview.html diff --git a/javadoc/overview.html b/javadoc/overview.html new file mode 100644 index 0000000000000..4ffd33f22fcad --- /dev/null +++ b/javadoc/overview.html @@ -0,0 +1,31 @@ + + + + Google Cloud Dataflow Java SDK + + +

The Google Cloud Dataflow SDK for Java provides a simple and elegant + programming model to express your data processing pipelines; + see our product page + for more information and getting started instructions.

+ +

The easiest way to use the Google Cloud Dataflow SDK for Java is via + one of the released artifacts from the + + Maven Central Repository. + See our + release notes for more information about each released version.

+ +

Version numbers use the form major.minor.incremental + and are incremented as follows:

+

    +
  • major version for incompatible API changes
  • +
  • minor version for new functionality added in a backward-compatible manner
  • +
  • incremental version for forward-compatible bug fixes
  • +
+ +

Please note that APIs marked + {@link com.google.cloud.dataflow.sdk.annotations.Experimental @Experimental} + may change at any point and are not guaranteed to remain compatible across versions.

+ + diff --git a/sdk/pom.xml b/sdk/pom.xml index 002b2d8e4f727..3a083ade1e75a 100644 --- a/sdk/pom.xml +++ b/sdk/pom.xml @@ -218,8 +218,8 @@ maven-javadoc-plugin Google Cloud Dataflow SDK ${project.version} API - Google Cloud Dataflow SDK ${project.version} API - ../overview.html + Google Cloud Dataflow SDK for Java, version ${project.version} + ../javadoc/overview.html com.google.cloud.dataflow.sdk -exclude com.google.cloud.dataflow.sdk.runners.worker:com.google.cloud.dataflow.sdk.runners.dataflow:com.google.cloud.dataflow.sdk.util ${dataflow.javadoc_opts} From 1b0d4173e6d8d933d3fc300ccefa70e1cf95e33f Mon Sep 17 00:00:00 2001 From: jeremiele Date: Wed, 30 Dec 2015 15:08:43 -0800 Subject: [PATCH 1257/1541] Ensures that BoundedSource and UnboundedSource are Serializable. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111153948 --- .../google/cloud/dataflow/sdk/io/Read.java | 5 +- .../dataflow/sdk/io/FileBasedSourceTest.java | 2 +- .../cloud/dataflow/sdk/io/ReadTest.java | 144 ++++++++++++++++++ 3 files changed, 148 insertions(+), 3 deletions(-) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ReadTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java index f4936a2375fa7..710fd643b3633 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.runners.dataflow.CustomSources; import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; import com.google.cloud.dataflow.sdk.util.WindowingStrategy; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded; @@ -100,7 +101,7 @@ public static class Bounded extends PTransform> { private Bounded(@Nullable String name, BoundedSource source) { super(name); - this.source = source; + this.source = SerializableUtils.ensureSerializable(source); } /** @@ -165,7 +166,7 @@ public static class Unbounded extends PTransform> { private Unbounded(@Nullable String name, UnboundedSource source) { super(name); - this.source = source; + this.source = SerializableUtils.ensureSerializable(source); } /** diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/FileBasedSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/FileBasedSourceTest.java index 914bfcd4f76b2..d50c6a63b9a17 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/FileBasedSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/FileBasedSourceTest.java @@ -79,7 +79,7 @@ public class FileBasedSourceTest { *

E.g., if {@code splitHeader} is "h" and the lines of the file are: h, a, b, h, h, c, then * the records in this source are a,b,c, and records a and c are split points. */ - class TestFileBasedSource extends FileBasedSource { + static class TestFileBasedSource extends FileBasedSource { final String splitHeader; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ReadTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ReadTest.java new file mode 100644 index 0000000000000..8dc517a8e8f6a --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ReadTest.java @@ -0,0 +1,144 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.io; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.UnboundedSource.CheckpointMark; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * Tests for {@link Read}. + */ +@RunWith(JUnit4.class) +public class ReadTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void failsWhenCustomBoundedSourceIsNotSerializable() { + thrown.expect(IllegalArgumentException.class); + Read.from(new NotSerializableBoundedSource()); + } + + @Test + public void succeedsWhenCustomBoundedSourceIsSerializable() { + Read.from(new SerializableBoundedSource()); + } + + @Test + public void failsWhenCustomUnboundedSourceIsNotSerializable() { + thrown.expect(IllegalArgumentException.class); + Read.from(new NotSerializableUnboundedSource()); + } + + @Test + public void succeedsWhenCustomUnboundedSourceIsSerializable() { + Read.from(new SerializableUnboundedSource()); + } + + private abstract static class CustomBoundedSource extends BoundedSource { + @Override + public List> splitIntoBundles( + long desiredBundleSizeBytes, PipelineOptions options) throws Exception { + return null; + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + return 0; + } + + @Override + public boolean producesSortedKeys(PipelineOptions options) throws Exception { + return false; + } + + @Override + public BoundedReader createReader(PipelineOptions options) throws IOException { + return null; + } + + @Override + public void validate() {} + + @Override + public Coder getDefaultOutputCoder() { + return null; + } + } + + private static class NotSerializableBoundedSource extends CustomBoundedSource { + @SuppressWarnings("unused") + private final NotSerializableClass notSerializableClass = new NotSerializableClass(); + } + + private static class SerializableBoundedSource extends CustomBoundedSource {} + + private abstract static class CustomUnboundedSource + extends UnboundedSource { + @Override + public List> generateInitialSplits( + int desiredNumSplits, PipelineOptions options) throws Exception { + return null; + } + + @Override + public UnboundedReader createReader( + PipelineOptions options, NoOpCheckpointMark checkpointMark) { + return null; + } + + @Override + @Nullable + public Coder getCheckpointMarkCoder() { + return null; + } + + @Override + public void validate() {} + + @Override + public Coder getDefaultOutputCoder() { + return null; + } + } + + private static class NoOpCheckpointMark implements CheckpointMark { + @Override + public void finalizeCheckpoint() throws IOException {} + } + + private static class NotSerializableUnboundedSource extends CustomUnboundedSource { + @SuppressWarnings("unused") + private final NotSerializableClass notSerializableClass = new NotSerializableClass(); + } + + private static class SerializableUnboundedSource extends CustomUnboundedSource {} + + private static class NotSerializableClass {} +} From d19f99f3ce95ddf41b47973119cb0ad09f77cf61 Mon Sep 17 00:00:00 2001 From: malo Date: Wed, 30 Dec 2015 16:59:35 -0800 Subject: [PATCH 1258/1541] Always pass maxNumWorkers in job settings When submitting a job request, always pass the user defined value of maxNumWorkers, even when no autoscaling algorithm has been explicitely chosen. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111159069 --- .../DataflowPipelineWorkerPoolOptions.java | 10 ++-- .../runners/DataflowPipelineTranslator.java | 6 +- .../DataflowPipelineTranslatorTest.java | 55 ++++++++++++++++++- 3 files changed, 63 insertions(+), 8 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java index d089491c3a156..633b2707b4f2f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/DataflowPipelineWorkerPoolOptions.java @@ -85,12 +85,14 @@ public String getAlgorithm() { void setAutoscalingAlgorithm(AutoscalingAlgorithmType value); /** - * The maximum number of workers to use when using workerpool autoscaling. + * The maximum number of workers to use for the workerpool. This options limits the size of the + * workerpool for the lifetime of the job, including + * pipeline updates. * If left unspecified, the Dataflow service will compute a ceiling. */ - @Description("[Experimental] The maximum number of workers to use when using workerpool " - + "autoscaling. If left unspecified, the Dataflow service will compute a ceiling.") - @Experimental(Experimental.Kind.AUTOSCALING) + @Description("The maximum number of workers to use for the workerpool. This options limits the " + + "size of the workerpool for the lifetime of the job, including pipeline updates. " + + "If left unspecified, the Dataflow service will compute a ceiling.") int getMaxNumWorkers(); void setMaxNumWorkers(int value); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index 7956efdfb2e78..f6618f98393fb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -440,12 +440,12 @@ public Job translate(List packages) { if (options.getDiskSizeGb() > 0) { workerPool.setDiskSizeGb(options.getDiskSizeGb()); } + AutoscalingSettings settings = new AutoscalingSettings(); if (options.getAutoscalingAlgorithm() != null) { - AutoscalingSettings settings = new AutoscalingSettings(); settings.setAlgorithm(options.getAutoscalingAlgorithm().getAlgorithm()); - settings.setMaxNumWorkers(options.getMaxNumWorkers()); - workerPool.setAutoscalingSettings(settings); } + settings.setMaxNumWorkers(options.getMaxNumWorkers()); + workerPool.setAutoscalingSettings(settings); List workerPools = new LinkedList<>(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java index 5c6ad9285bc48..272329f950905 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java @@ -225,12 +225,23 @@ public void testScalingAlgorithmMissing() throws IOException { .getJob(); assertEquals(1, job.getEnvironment().getWorkerPools().size()); + // Autoscaling settings are always set. assertNull( job .getEnvironment() .getWorkerPools() .get(0) - .getAutoscalingSettings()); + .getAutoscalingSettings() + .getAlgorithm()); + assertEquals( + 0, + job + .getEnvironment() + .getWorkerPools() + .get(0) + .getAutoscalingSettings() + .getMaxNumWorkers() + .intValue()); } @Test @@ -257,6 +268,48 @@ public void testScalingAlgorithmNone() throws IOException { .get(0) .getAutoscalingSettings() .getAlgorithm()); + assertEquals( + 0, + job + .getEnvironment() + .getWorkerPools() + .get(0) + .getAutoscalingSettings() + .getMaxNumWorkers() + .intValue()); + } + + @Test + public void testMaxNumWorkersIsPassedWhenNoAlgorithmIsSet() throws IOException { + final DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType noScaling = null; + DataflowPipelineOptions options = buildPipelineOptions(); + options.setMaxNumWorkers(42); + options.setAutoscalingAlgorithm(noScaling); + + Pipeline p = buildPipeline(options); + p.traverseTopologically(new RecordingPipelineVisitor()); + Job job = + DataflowPipelineTranslator.fromOptions(options) + .translate(p, Collections.emptyList()) + .getJob(); + + assertEquals(1, job.getEnvironment().getWorkerPools().size()); + assertNull( + job + .getEnvironment() + .getWorkerPools() + .get(0) + .getAutoscalingSettings() + .getAlgorithm()); + assertEquals( + 42, + job + .getEnvironment() + .getWorkerPools() + .get(0) + .getAutoscalingSettings() + .getMaxNumWorkers() + .intValue()); } @Test From caec17bae26bbbb7dfeee901b449257dd2867c6c Mon Sep 17 00:00:00 2001 From: sgmc Date: Wed, 30 Dec 2015 19:17:38 -0800 Subject: [PATCH 1259/1541] Factor AvroUtils into a separate class ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111163754 --- .../cloud/dataflow/sdk/io/AvroSource.java | 93 +--------- .../cloud/dataflow/sdk/util/AvroUtils.java | 137 ++++++++++++++ .../dataflow/sdk/util/AvroUtilsTest.java | 170 ++++++++++++++++++ 3 files changed, 316 insertions(+), 84 deletions(-) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AvroUtils.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AvroUtilsTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroSource.java index 421b9b365f031..6d9279ad82929 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroSource.java @@ -20,7 +20,8 @@ import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.PipelineRunner; -import com.google.cloud.dataflow.sdk.util.IOChannelUtils; +import com.google.cloud.dataflow.sdk.util.AvroUtils; +import com.google.cloud.dataflow.sdk.util.AvroUtils.AvroMetadata; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.base.Preconditions; @@ -45,7 +46,6 @@ import java.nio.ByteBuffer; import java.nio.channels.Channels; import java.nio.channels.ReadableByteChannel; -import java.util.Arrays; import java.util.Collection; import java.util.zip.Inflater; import java.util.zip.InflaterInputStream; @@ -241,81 +241,6 @@ public void validate() { super.validate(); } - /** - * Avro file metadata. Visible for testing. - */ - static class Metadata { - byte[] syncMarker; - String codec; - String schema; - - public Metadata(byte[] syncMarker, String codec, String schema) { - this.syncMarker = syncMarker; - this.codec = codec; - this.schema = schema; - } - } - - /** - * Reads the {@link Metadata} from the header of an Avro file. Throws an IOException if the file - * is an invalid format. - * - *

This method parses the header of an Avro - * - * Object Container File. - */ - static Metadata readMetadataFromFile(String fileName) throws IOException { - String codec = null; - String schema = null; - byte[] syncMarker; - try (InputStream stream = - Channels.newInputStream(IOChannelUtils.getFactory(fileName).open(fileName))) { - BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(stream, null); - - // The header of an object container file begins with a four-byte magic number, followed - // by the file metadata (including the schema and codec), encoded as a map. Finally, the - // header ends with the file's 16-byte sync marker. - // See https://avro.apache.org/docs/1.7.7/spec.html#Object+Container+Files for details on - // the encoding of container files. - - // Read the magic number. - byte[] magic = new byte[DataFileConstants.MAGIC.length]; - decoder.readFixed(magic); - if (!Arrays.equals(magic, DataFileConstants.MAGIC)) { - throw new IOException("Missing Avro file signature: " + fileName); - } - - // Read the metadata to find the codec and schema. - ByteBuffer valueBuffer = ByteBuffer.allocate(512); - long numRecords = decoder.readMapStart(); - while (numRecords > 0) { - for (long recordIndex = 0; recordIndex < numRecords; recordIndex++) { - String key = decoder.readString(); - // readBytes() clears the buffer and returns a buffer where: - // - position is the start of the bytes read - // - limit is the end of the bytes read - valueBuffer = decoder.readBytes(valueBuffer); - byte[] bytes = new byte[valueBuffer.remaining()]; - valueBuffer.get(bytes); - if (key.equals(DataFileConstants.CODEC)) { - codec = new String(bytes, "UTF-8"); - } else if (key.equals(DataFileConstants.SCHEMA)) { - schema = new String(bytes, "UTF-8"); - } - } - numRecords = decoder.mapNext(); - } - if (codec == null) { - codec = DataFileConstants.NULL_CODEC; - } - - // Finally, read the sync marker. - syncMarker = new byte[DataFileConstants.SYNC_SIZE]; - decoder.readFixed(syncMarker); - } - return new Metadata(syncMarker, codec, schema); - } - @Override public AvroSource createForSubrangeOfFile(String fileName, long start, long end) { byte[] syncMarker = this.syncMarker; @@ -327,21 +252,21 @@ public AvroSource createForSubrangeOfFile(String fileName, long start, long e // for a subrange of a file, we can initialize these values. When the resulting AvroSource // is further split, they do not need to be read again. if (codec == null || syncMarker == null || fileSchemaString == null) { - Metadata metadata; + AvroMetadata metadata; try { Collection files = FileBasedSource.expandFilePattern(fileName); Preconditions.checkArgument(files.size() <= 1, "More than 1 file matched %s"); - metadata = readMetadataFromFile(fileName); + metadata = AvroUtils.readMetadataFromFile(fileName); } catch (IOException e) { throw new RuntimeException("Error reading metadata from file " + fileName, e); } - codec = metadata.codec; - syncMarker = metadata.syncMarker; - fileSchemaString = metadata.schema; + codec = metadata.getCodec(); + syncMarker = metadata.getSyncMarker(); + fileSchemaString = metadata.getSchemaString(); // If the source was created with a null schema, use the schema that we read from the file's // metadata. if (readSchemaString == null) { - readSchemaString = metadata.schema; + readSchemaString = metadata.getSchemaString(); } } return new AvroSource(fileName, getMinBundleSize(), start, end, readSchemaString, type, @@ -551,7 +476,7 @@ public AvroReader(AvroSource source) { } @Override - public AvroSource getCurrentSource() { + public synchronized AvroSource getCurrentSource() { return (AvroSource) super.getCurrentSource(); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AvroUtils.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AvroUtils.java new file mode 100644 index 0000000000000..b50fe8a7805bf --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/AvroUtils.java @@ -0,0 +1,137 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import org.apache.avro.file.DataFileConstants; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DecoderFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.util.Arrays; + +/** + * A set of utilities for working with Avro files. + * + *

These utilities are based on the Avro 1.7.7 specification. + */ +public class AvroUtils { + + /** + * Avro file metadata. + */ + public static class AvroMetadata { + private byte[] syncMarker; + private String codec; + private String schemaString; + + AvroMetadata(byte[] syncMarker, String codec, String schemaString) { + this.syncMarker = syncMarker; + this.codec = codec; + this.schemaString = schemaString; + } + + /** + * The JSON-encoded schema + * string for the file. + */ + public String getSchemaString() { + return schemaString; + } + + /** + * The codec of the + * file. + */ + public String getCodec() { + return codec; + } + + /** + * The 16-byte sync marker for the file. See the documentation for + * Object + * Container File for more information. + */ + public byte[] getSyncMarker() { + return syncMarker; + } + } + + /** + * Reads the {@link AvroMetadata} from the header of an Avro file. + * + *

This method parses the header of an Avro + * + * Object Container File. + * + * @throws IOException if the file is an invalid format. + */ + public static AvroMetadata readMetadataFromFile(String fileName) throws IOException { + String codec = null; + String schemaString = null; + byte[] syncMarker; + try (InputStream stream = + Channels.newInputStream(IOChannelUtils.getFactory(fileName).open(fileName))) { + BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(stream, null); + + // The header of an object container file begins with a four-byte magic number, followed + // by the file metadata (including the schema and codec), encoded as a map. Finally, the + // header ends with the file's 16-byte sync marker. + // See https://avro.apache.org/docs/1.7.7/spec.html#Object+Container+Files for details on + // the encoding of container files. + + // Read the magic number. + byte[] magic = new byte[DataFileConstants.MAGIC.length]; + decoder.readFixed(magic); + if (!Arrays.equals(magic, DataFileConstants.MAGIC)) { + throw new IOException("Missing Avro file signature: " + fileName); + } + + // Read the metadata to find the codec and schema. + ByteBuffer valueBuffer = ByteBuffer.allocate(512); + long numRecords = decoder.readMapStart(); + while (numRecords > 0) { + for (long recordIndex = 0; recordIndex < numRecords; recordIndex++) { + String key = decoder.readString(); + // readBytes() clears the buffer and returns a buffer where: + // - position is the start of the bytes read + // - limit is the end of the bytes read + valueBuffer = decoder.readBytes(valueBuffer); + byte[] bytes = new byte[valueBuffer.remaining()]; + valueBuffer.get(bytes); + if (key.equals(DataFileConstants.CODEC)) { + codec = new String(bytes, "UTF-8"); + } else if (key.equals(DataFileConstants.SCHEMA)) { + schemaString = new String(bytes, "UTF-8"); + } + } + numRecords = decoder.mapNext(); + } + if (codec == null) { + codec = DataFileConstants.NULL_CODEC; + } + + // Finally, read the sync marker. + syncMarker = new byte[DataFileConstants.SYNC_SIZE]; + decoder.readFixed(syncMarker); + } + return new AvroMetadata(syncMarker, codec, schemaString); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AvroUtilsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AvroUtilsTest.java new file mode 100644 index 0000000000000..68570bfd7a873 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/AvroUtilsTest.java @@ -0,0 +1,170 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.junit.Assert.assertEquals; + +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.DefaultCoder; +import com.google.cloud.dataflow.sdk.util.AvroUtils.AvroMetadata; +import com.google.common.base.MoreObjects; + +import org.apache.avro.Schema; +import org.apache.avro.file.CodecFactory; +import org.apache.avro.file.DataFileConstants; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.io.DatumWriter; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Random; + +/** + * Tests for AvroUtils. + */ +@RunWith(JUnit4.class) +public class AvroUtilsTest { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + private static final int DEFAULT_RECORD_COUNT = 10000; + + /** + * Generates an input Avro file containing the given records in the temporary directory and + * returns the full path of the file. + */ + @SuppressWarnings("deprecation") // test of internal functionality + private String generateTestFile(String filename, List elems, AvroCoder coder, + String codec) throws IOException { + File tmpFile = tmpFolder.newFile(filename); + String path = tmpFile.toString(); + + FileOutputStream os = new FileOutputStream(tmpFile); + DatumWriter datumWriter = coder.createDatumWriter(); + try (DataFileWriter writer = new DataFileWriter<>(datumWriter)) { + writer.setCodec(CodecFactory.fromString(codec)); + writer.create(coder.getSchema(), os); + for (T elem : elems) { + writer.append(elem); + } + } + return path; + } + + @Test + public void testReadMetadataWithCodecs() throws Exception { + // Test reading files generated using all codecs. + String codecs[] = {DataFileConstants.NULL_CODEC, DataFileConstants.BZIP2_CODEC, + DataFileConstants.DEFLATE_CODEC, DataFileConstants.SNAPPY_CODEC, + DataFileConstants.XZ_CODEC}; + List expected = createRandomRecords(DEFAULT_RECORD_COUNT); + + for (String codec : codecs) { + String filename = generateTestFile( + codec, expected, AvroCoder.of(Bird.class), codec); + AvroMetadata metadata = AvroUtils.readMetadataFromFile(filename); + assertEquals(codec, metadata.getCodec()); + } + } + + @Test + public void testReadSchemaString() throws Exception { + List expected = createRandomRecords(DEFAULT_RECORD_COUNT); + String codec = DataFileConstants.NULL_CODEC; + String filename = generateTestFile( + codec, expected, AvroCoder.of(Bird.class), codec); + AvroMetadata metadata = AvroUtils.readMetadataFromFile(filename); + // By default, parse validates the schema, which is what we want. + Schema schema = new Schema.Parser().parse(metadata.getSchemaString()); + assertEquals(4, schema.getFields().size()); + } + + /** + * Class used as the record type in tests. + */ + @DefaultCoder(AvroCoder.class) + static class Bird { + long number; + String species; + String quality; + long quantity; + + public Bird() {} + + public Bird(long number, String species, String quality, long quantity) { + this.number = number; + this.species = species; + this.quality = quality; + this.quantity = quantity; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(Bird.class) + .addValue(number) + .addValue(species) + .addValue(quantity) + .addValue(quality) + .toString(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof Bird) { + Bird other = (Bird) obj; + return Objects.equals(species, other.species) && Objects.equals(quality, other.quality) + && quantity == other.quantity && number == other.number; + } + return false; + } + + @Override + public int hashCode() { + return Objects.hash(number, species, quality, quantity); + } + } + + /** + * Create a list of n random records. + */ + private static List createRandomRecords(long n) { + String[] qualities = { + "miserable", "forelorn", "fidgity", "squirrelly", "fanciful", "chipper", "lazy"}; + String[] species = {"pigeons", "owls", "gulls", "hawks", "robins", "jays"}; + Random random = new Random(0); + + List records = new ArrayList<>(); + for (long i = 0; i < n; i++) { + Bird bird = new Bird(); + bird.quality = qualities[random.nextInt(qualities.length)]; + bird.species = species[random.nextInt(species.length)]; + bird.number = i; + bird.quantity = random.nextLong(); + records.add(bird); + } + return records; + } +} From 52e6b4db2545d14bd5e8fa46a74dcc126e9965e5 Mon Sep 17 00:00:00 2001 From: amyu Date: Mon, 4 Jan 2016 08:06:38 -0800 Subject: [PATCH 1260/1541] Change the 'gaming' injector credentials check logic In the 'gaming' injector, change the credentials 'exit' logic to check client auth. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111322434 --- .../dataflow/examples/complete/game/injector/InjectorUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/InjectorUtils.java b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/InjectorUtils.java index 06c38646274f2..55982df933e3a 100644 --- a/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/InjectorUtils.java +++ b/examples/src/main/java8/com/google/cloud/dataflow/examples/complete/game/injector/InjectorUtils.java @@ -49,7 +49,7 @@ public static Pubsub getClient(final HttpTransport httpTransport, if (credential.createScopedRequired()) { credential = credential.createScoped(PubsubScopes.all()); } - if (credential.getServiceAccountId() == null) { + if (credential.getClientAuthentication() != null) { System.out.println("\n***Warning! You are not using service account credentials to " + "authenticate.\nYou need to use service account credentials for this example," + "\nsince user-level credentials do not have enough pubsub quota,\nand so you will run " From 454f221fff5a352a6dbbbcbae59d13b75cb2ecac Mon Sep 17 00:00:00 2001 From: vladum Date: Mon, 4 Jan 2016 10:51:41 -0800 Subject: [PATCH 1261/1541] Increment environment major version to 4 ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111336513 --- .../cloud/dataflow/sdk/runners/DataflowPipelineRunner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index b1a53446ba3ac..25084474114a9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -145,7 +145,7 @@ public class DataflowPipelineRunner extends PipelineRunner private DataflowPipelineRunnerHooks hooks; // Environment version information - private static final String ENVIRONMENT_MAJOR_VERSION = "3"; + private static final String ENVIRONMENT_MAJOR_VERSION = "4"; // The limit of CreateJob request size. private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024; From 47b397ca3019e902ea59183704ca0e4676b512f4 Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 4 Jan 2016 13:24:38 -0800 Subject: [PATCH 1262/1541] Tidy PCollectionViews ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111350932 --- .../cloud/dataflow/sdk/util/PCollectionViews.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PCollectionViews.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PCollectionViews.java index b81732a37cf60..e5308aa58a966 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PCollectionViews.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/PCollectionViews.java @@ -96,7 +96,7 @@ public static PCollectionView> mapView WindowingStrategy windowingStrategy, Coder> valueCoder) { - return new MapPCollectionView(pipeline, windowingStrategy, valueCoder); + return new MapPCollectionView(pipeline, windowingStrategy, valueCoder); } /** @@ -107,7 +107,7 @@ public static PCollectionView Pipeline pipeline, WindowingStrategy windowingStrategy, Coder> valueCoder) { - return new MultimapPCollectionView(pipeline, windowingStrategy, valueCoder); + return new MultimapPCollectionView(pipeline, windowingStrategy, valueCoder); } /** @@ -229,7 +229,7 @@ private static class MultimapPCollectionView extends PCollectionViewBase, Map>, W> { public MultimapPCollectionView( Pipeline pipeline, - WindowingStrategy, W> windowingStrategy, + WindowingStrategy windowingStrategy, Coder> valueCoder) { super(pipeline, windowingStrategy, valueCoder); } @@ -258,7 +258,7 @@ private static class MapPCollectionView extends PCollectionViewBase, Map, W> { public MapPCollectionView( Pipeline pipeline, - WindowingStrategy, W> windowingStrategy, + WindowingStrategy windowingStrategy, Coder> valueCoder) { super(pipeline, windowingStrategy, valueCoder); } @@ -362,7 +362,7 @@ public TupleTag>> getTagInternal() { // Safe cast: It is required that the rest of the SDK maintain the invariant that // this tag is only used to access the contents of an appropriately typed underlying // PCollection - @SuppressWarnings({"rawtypes, unchecked"}) + @SuppressWarnings({"rawtypes", "unchecked"}) TupleTag>> untypedTag = (TupleTag) tag; return untypedTag; } From 9262be99f0c8d848ed7a26cdccb2040a30e28413 Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 4 Jan 2016 14:01:31 -0800 Subject: [PATCH 1263/1541] Tidy some type variables ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111354348 --- .../dataflow/sdk/transforms/IntraBundleParallelization.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelization.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelization.java index 2bcf28a7915a4..b6497b71c4ef1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelization.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelization.java @@ -160,7 +160,8 @@ public Bound withMaxParallelism(int maxParallelism) { * *

Note that the specified {@code doFn} needs to be thread safe. */ - public Bound of(DoFn doFn) { + public Bound + of(DoFn doFn) { return new Bound<>(doFn, maxParallelism); } From 3bdb31bc6dd0ce92568895cbbf0960d92cd40632 Mon Sep 17 00:00:00 2001 From: klk Date: Mon, 4 Jan 2016 14:44:45 -0800 Subject: [PATCH 1264/1541] Tidy some tests and support code ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111358603 --- .../cookbook/MaxPerKeyExamplesTest.java | 13 +++---- .../dataflow/sdk/transforms/DoFnTester.java | 34 ++++++++++++++----- 2 files changed, 31 insertions(+), 16 deletions(-) diff --git a/examples/src/test/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamplesTest.java b/examples/src/test/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamplesTest.java index 539012d6fbcbf..3deff2a2e3541 100644 --- a/examples/src/test/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamplesTest.java +++ b/examples/src/test/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamplesTest.java @@ -21,6 +21,7 @@ import com.google.cloud.dataflow.examples.cookbook.MaxPerKeyExamples.FormatMaxesFn; import com.google.cloud.dataflow.sdk.transforms.DoFnTester; import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.collect.ImmutableList; import org.hamcrest.CoreMatchers; import org.junit.Assert; @@ -46,17 +47,13 @@ public class MaxPerKeyExamplesTest { .set("month", "6").set("day", "18") .set("year", "2014").set("mean_temp", "45.3") .set("tornado", true); - private static final TableRow[] ROWS_ARRAY = new TableRow[] { - row1, row2, row3 - }; + private static final List TEST_ROWS = ImmutableList.of(row1, row2, row3); private static final KV kv1 = KV.of(6, 85.3); private static final KV kv2 = KV.of(6, 45.3); private static final KV kv3 = KV.of(7, 75.4); - static final KV[] TUPLES_ARRAY = new KV[] { - kv1, kv2, kv3 - }; + private static final List> TEST_KVS = ImmutableList.of(kv1, kv2, kv3); private static final TableRow resultRow1 = new TableRow() .set("month", 6) @@ -70,7 +67,7 @@ public class MaxPerKeyExamplesTest { public void testExtractTempFn() { DoFnTester> extractTempFn = DoFnTester.of(new ExtractTempFn()); - List> results = extractTempFn.processBatch(ROWS_ARRAY); + List> results = extractTempFn.processBatch(TEST_ROWS); Assert.assertThat(results, CoreMatchers.hasItem(kv1)); Assert.assertThat(results, CoreMatchers.hasItem(kv2)); Assert.assertThat(results, CoreMatchers.hasItem(kv3)); @@ -80,7 +77,7 @@ public void testExtractTempFn() { public void testFormatMaxesFn() { DoFnTester, TableRow> formatMaxesFnFn = DoFnTester.of(new FormatMaxesFn()); - List results = formatMaxesFnFn.processBatch(TUPLES_ARRAY); + List results = formatMaxesFnFn.processBatch(TEST_KVS); Assert.assertThat(results, CoreMatchers.hasItem(resultRow1)); Assert.assertThat(results, CoreMatchers.hasItem(resultRow2)); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java index 99fc0a31d0e6a..e24d99448a307 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnTester.java @@ -35,6 +35,7 @@ import com.google.common.collect.Lists; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -152,11 +153,11 @@ public void setSideOutputTags(TupleTagList sideOutputTags) { /** * A convenience operation that first calls {@link #startBundle}, - * then calls {@link #processElement} on each of the arguments, then + * then calls {@link #processElement} on each of the input elements, then * calls {@link #finishBundle}, then returns the result of * {@link #takeOutputElements}. */ - public List processBatch(InputT... inputElements) { + public List processBatch(Iterable inputElements) { startBundle(); for (InputT inputElement : inputElements) { processElement(inputElement); @@ -165,6 +166,22 @@ public List processBatch(InputT... inputElements) { return takeOutputElements(); } + /** + * A convenience method for testing {@link DoFn DoFns} with bundles of elements. + * Logic proceeds as follows: + * + *

    + *
  1. Calls {@link #startBundle}.
  2. + *
  3. Calls {@link #processElement} on each of the arguments.
  4. + *
  5. Calls {@link #finishBundle}.
  6. + *
  7. Returns the result of {@link #takeOutputElements}.
  8. + *
+ */ + @SafeVarargs + public final List processBatch(InputT... inputElements) { + return processBatch(Arrays.asList(inputElements)); + } + /** * Calls {@link DoFn#startBundle} on the {@code DoFn} under test. * @@ -276,10 +293,10 @@ public List peekSideOutputElements(TupleTag tag) { // TODO: Should we return an unmodifiable list? return Lists.transform( outputManager.getOutput(tag), - new Function() { + new Function, T>() { @Override - public T apply(Object input) { - return ((WindowedValue) input).getValue(); + public T apply(WindowedValue input) { + return input.getValue(); }}); } @@ -308,10 +325,11 @@ public List takeSideOutputElements(TupleTag tag) { /** * Returns the value of the provided {@link Aggregator}. */ - public OutputT getAggregatorValue(Aggregator agg) { + public AggregateT getAggregatorValue(Aggregator agg) { @SuppressWarnings("unchecked") - Counter counter = - (Counter) counterSet.getExistingCounter("user-" + STEP_NAME + "-" + agg.getName()); + Counter counter = + (Counter) + counterSet.getExistingCounter("user-" + STEP_NAME + "-" + agg.getName()); return counter.getAggregate(); } From 486b6419135cb2d2091ae72521fdece6c28848f9 Mon Sep 17 00:00:00 2001 From: jdobry Date: Mon, 4 Jan 2016 14:51:31 -0800 Subject: [PATCH 1265/1541] Setting table creation definition in WindowedWordCount example ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111359232 --- .../google/cloud/dataflow/examples/WindowedWordCount.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/examples/src/main/java/com/google/cloud/dataflow/examples/WindowedWordCount.java b/examples/src/main/java/com/google/cloud/dataflow/examples/WindowedWordCount.java index 207d58629c588..fcda15eeca8f7 100644 --- a/examples/src/main/java/com/google/cloud/dataflow/examples/WindowedWordCount.java +++ b/examples/src/main/java/com/google/cloud/dataflow/examples/WindowedWordCount.java @@ -244,7 +244,11 @@ public static void main(String[] args) throws IOException { * The BigQuery output source supports both bounded and unbounded data. */ wordCounts.apply(ParDo.of(new FormatAsTableRowFn())) - .apply(BigQueryIO.Write.to(getTableReference(options)).withSchema(getSchema())); + .apply(BigQueryIO.Write + .to(getTableReference(options)) + .withSchema(getSchema()) + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE)); PipelineResult result = pipeline.run(); From 9a3deebd51c17827a148664fb7039ff42001e79d Mon Sep 17 00:00:00 2001 From: dhalperi Date: Mon, 4 Jan 2016 14:53:57 -0800 Subject: [PATCH 1266/1541] Add ByteCoder ByteCoder encodes a Java Byte directly as a byte, thus a PCollection will be encoded using consumes 1 byte of storage per element. It is deterministic. Also add ByteCoder to the CoderRegistry as the default encoder/decoder for PCollection. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111359449 --- .../cloud/dataflow/sdk/coders/ByteCoder.java | 111 ++++++++++++++++++ .../dataflow/sdk/coders/CoderRegistry.java | 1 + .../dataflow/sdk/coders/ByteCoderTest.java | 91 ++++++++++++++ 3 files changed, 203 insertions(+) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteCoder.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteCoderTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteCoder.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteCoder.java new file mode 100644 index 0000000000000..9f17497d8dc4a --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/ByteCoder.java @@ -0,0 +1,111 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UTFDataFormatException; + +/** + * A {@link ByteCoder} encodes {@link Byte} values in 1 byte using Java serialization. + */ +public class ByteCoder extends AtomicCoder { + + @JsonCreator + public static ByteCoder of() { + return INSTANCE; + } + + ///////////////////////////////////////////////////////////////////////////// + + private static final ByteCoder INSTANCE = new ByteCoder(); + + private ByteCoder() {} + + @Override + public void encode(Byte value, OutputStream outStream, Context context) + throws IOException, CoderException { + if (value == null) { + throw new CoderException("cannot encode a null Byte"); + } + outStream.write(value.byteValue()); + } + + @Override + public Byte decode(InputStream inStream, Context context) + throws IOException, CoderException { + try { + // value will be between 0-255, -1 for EOF + int value = inStream.read(); + if (value == -1) { + throw new EOFException("EOF encountered decoding 1 byte from input stream"); + } + return (byte) value; + } catch (EOFException | UTFDataFormatException exn) { + // These exceptions correspond to decoding problems, so change + // what kind of exception they're branded as. + throw new CoderException(exn); + } + } + + /** + * {@inheritDoc} + * + * {@link ByteCoder} will never throw a {@link Coder.NonDeterministicException}; bytes can always + * be encoded deterministically. + */ + @Override + public void verifyDeterministic() {} + + /** + * {@inheritDoc} + * + * @return {@code true}. This coder is injective. + */ + @Override + public boolean consistentWithEquals() { + return true; + } + + /** + * {@inheritDoc} + * + * @return {@code true}. {@link ByteCoder#getEncodedElementByteSize} returns a constant. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(Byte value, Context context) { + return true; + } + + /** + * {@inheritDoc} + * + * @return {@code 1}, the byte size of a {@link Byte} encoded using Java serialization. + */ + @Override + protected long getEncodedElementByteSize(Byte value, Context context) + throws Exception { + if (value == null) { + throw new CoderException("cannot estimate size for unsupported null value"); + } + return 1; + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java index 82c3a4980e140..a5f77b9429d88 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/coders/CoderRegistry.java @@ -88,6 +88,7 @@ public CoderRegistry() { * Registers standard Coders with this CoderRegistry. */ public void registerStandardCoders() { + registerCoder(Byte.class, ByteCoder.class); registerCoder(Double.class, DoubleCoder.class); registerCoder(Instant.class, InstantCoder.class); registerCoder(Integer.class, VarIntCoder.class); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteCoderTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteCoderTest.java new file mode 100644 index 0000000000000..6cb852e236139 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/coders/ByteCoderTest.java @@ -0,0 +1,91 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.coders; + +import com.google.cloud.dataflow.sdk.testing.CoderProperties; +import com.google.cloud.dataflow.sdk.util.CoderUtils; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Arrays; +import java.util.List; + +/** + * Test case for {@link ByteCoder}. + */ +@RunWith(JUnit4.class) +public class ByteCoderTest { + + private static final Coder TEST_CODER = ByteCoder.of(); + + private static final List TEST_VALUES = Arrays.asList( + (byte) 1, + (byte) 4, + (byte) 6, + (byte) 50, + (byte) 124, + Byte.MAX_VALUE, + Byte.MIN_VALUE); + + @Test + public void testDecodeEncodeEqual() throws Exception { + for (Byte value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeEqual(TEST_CODER, value); + } + } + + // This should never change. The format is fixed by Java. + private static final String EXPECTED_ENCODING_ID = ""; + + @Test + public void testEncodingId() throws Exception { + CoderProperties.coderHasEncodingId(TEST_CODER, EXPECTED_ENCODING_ID); + } + + /** + * Generated data to check that the wire format has not changed. To regenerate, see + * {@link com.google.cloud.dataflow.sdk.coders.PrintBase64Encodings}. + */ + private static final List TEST_ENCODINGS = Arrays.asList( + "AQ", + "BA", + "Bg", + "Mg", + "fA", + "fw", + "gA"); + + @Test + public void testWireFormatEncode() throws Exception { + CoderProperties.coderEncodesBase64(TEST_CODER, TEST_VALUES, TEST_ENCODINGS); + } + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void encodeNullThrowsCoderException() throws Exception { + thrown.expect(CoderException.class); + thrown.expectMessage("cannot encode a null Byte"); + + CoderUtils.encodeToBase64(TEST_CODER, null); + } +} From e02f2e38874bb2250d952c1130b3c9862db2a4a2 Mon Sep 17 00:00:00 2001 From: markshields Date: Tue, 5 Jan 2016 12:24:20 -0800 Subject: [PATCH 1267/1541] Eagerly merge new windows into existing windows If a new element introduces a new window, and that window can be merged into an existing window, make sure we avoid creating any intermediate state for the new window. Instead, redirect all state to the existing window. In the common case of input elements arriving in (roughly) event time order, and using session windows, will ensure state space and state merging cost is O(1) instead of O(n) for n elements per final session. ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111439148 --- .../sdk/transforms/windowing/AfterPane.java | 28 +- .../windowing/AfterProcessingTime.java | 33 +- .../AfterSynchronizedProcessingTime.java | 32 +- .../transforms/windowing/AfterWatermark.java | 32 +- .../dataflow/sdk/util/ActiveWindowSet.java | 111 +++- .../sdk/util/MergingActiveWindowSet.java | 421 +++++++++++--- .../dataflow/sdk/util/NonEmptyPanes.java | 4 +- .../sdk/util/NonMergingActiveWindowSet.java | 49 +- .../cloud/dataflow/sdk/util/ReduceFn.java | 34 +- .../sdk/util/ReduceFnContextFactory.java | 42 +- .../dataflow/sdk/util/ReduceFnRunner.java | 544 ++++++++++-------- .../dataflow/sdk/util/SystemReduceFn.java | 4 +- .../sdk/util/TriggerContextFactory.java | 2 +- .../dataflow/sdk/util/TriggerRunner.java | 56 +- .../dataflow/sdk/util/WatermarkHold.java | 99 ++-- .../sdk/util/state/MergingStateInternals.java | 18 +- .../windowing/AfterProcessingTimeTest.java | 2 +- .../sdk/util/MergingActiveWindowSetTest.java | 175 ++++++ .../dataflow/sdk/util/ReduceFnTester.java | 8 +- .../state/InMemoryStateInternalsTest.java | 6 +- 20 files changed, 1154 insertions(+), 546 deletions(-) create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/MergingActiveWindowSetTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterPane.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterPane.java index 7e56d0f11ca74..bdc04c2c178bb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterPane.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterPane.java @@ -59,6 +59,11 @@ public static AfterPane elementCountAtLeast(int cou return new AfterPane<>(countElems); } + @Override + public void prefetchOnElement(StateContext state) { + state.access(ELEMENTS_IN_PANE_TAG).get(); + } + @Override public TriggerResult onElement(OnElementContext c) throws Exception { CombiningValueState elementsInPane = c.state().access(ELEMENTS_IN_PANE_TAG); @@ -71,6 +76,11 @@ public TriggerResult onElement(OnElementContext c) throws Exception { return count >= countElems ? TriggerResult.FIRE_AND_FINISH : TriggerResult.CONTINUE; } + @Override + public void prefetchOnMerge(MergingStateContext state) { + state.mergingAccess(ELEMENTS_IN_PANE_TAG).get(); + } + @Override public MergeResult onMerge(OnMergeContext c) throws Exception { // If we've already received enough elements and finished in some window, then this trigger @@ -80,8 +90,12 @@ public MergeResult onMerge(OnMergeContext c) throws Exception { } // Otherwise, compute the sum of elements in all the active panes - CombiningValueState elementsInPane = - c.state().accessAcrossMergingWindows(ELEMENTS_IN_PANE_TAG); + CombiningValueState elementsInPane = c.state().mergingAccess(ELEMENTS_IN_PANE_TAG); + // Both InMemoryStateInternals and WindmillStateInternals implement merging access using + // MergeCombiningValue. The implementation of StateContents.read returned by get will + // eagerly compact state on read. Thus after the following we are guaranteed all state from + // merged windows will have been compacted away. + // TODO: Make this more explicit and less fragile with a 'compact' API? long count = elementsInPane.get().read(); return count >= countElems ? MergeResult.FIRE_AND_FINISH : MergeResult.CONTINUE; } @@ -91,16 +105,6 @@ public TriggerResult onTimer(OnTimerContext c) { return TriggerResult.CONTINUE; } - @Override - public void prefetchOnElement(StateContext state) { - state.access(ELEMENTS_IN_PANE_TAG).get(); - } - - @Override - public void prefetchOnMerge(MergingStateContext state) { - state.accessAcrossMergingWindows(ELEMENTS_IN_PANE_TAG).get(); - } - @Override public void prefetchOnTimer(StateContext state) { } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTime.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTime.java index 5fe0eb9942c28..a64bb3437a35d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTime.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTime.java @@ -58,6 +58,12 @@ protected AfterProcessingTime newWith( return new AfterProcessingTime(transforms); } + @Override + public void prefetchOnElement(StateContext state) { + state.access(DELAYED_UNTIL_TAG).get(); + } + + @Override public TriggerResult onElement(OnElementContext c) throws Exception { @@ -72,6 +78,11 @@ public TriggerResult onElement(OnElementContext c) return TriggerResult.CONTINUE; } + @Override + public void prefetchOnMerge(MergingStateContext state) { + state.mergingAccess(DELAYED_UNTIL_TAG).get(); + } + @Override public MergeResult onMerge(OnMergeContext c) throws Exception { // If the processing time timer has fired in any of the windows being merged, it would have @@ -83,7 +94,7 @@ public MergeResult onMerge(OnMergeContext c) throws Exception { // Determine the earliest point across all the windows, and delay to that. CombiningValueState mergingDelays = - c.state().accessAcrossMergingWindows(DELAYED_UNTIL_TAG); + c.state().mergingAccess(DELAYED_UNTIL_TAG); Instant earliestTimer = mergingDelays.get().read(); if (earliestTimer != null) { mergingDelays.clear(); @@ -94,6 +105,11 @@ public MergeResult onMerge(OnMergeContext c) throws Exception { return MergeResult.CONTINUE; } + @Override + public void prefetchOnTimer(StateContext state) { + state.access(DELAYED_UNTIL_TAG).get(); + } + @Override public TriggerResult onTimer(OnTimerContext c) throws Exception { if (c.timeDomain() != TimeDomain.PROCESSING_TIME) { @@ -108,21 +124,6 @@ public TriggerResult onTimer(OnTimerContext c) throws Exception { return TriggerResult.FIRE_AND_FINISH; } - @Override - public void prefetchOnElement(StateContext state) { - state.access(DELAYED_UNTIL_TAG).get(); - } - - @Override - public void prefetchOnMerge(MergingStateContext state) { - state.accessAcrossMergingWindows(DELAYED_UNTIL_TAG).get(); - } - - @Override - public void prefetchOnTimer(StateContext state) { - state.access(DELAYED_UNTIL_TAG).get(); - } - @Override public void clear(TriggerContext c) throws Exception { CombiningValueState delayed = c.state().access(DELAYED_UNTIL_TAG); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java index 8d7ce10b5dc0b..d38b6f8ad49bf 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java @@ -34,6 +34,11 @@ public AfterSynchronizedProcessingTime() { super(null); } + @Override + public void prefetchOnElement(StateContext state) { + state.access(DELAYED_UNTIL_TAG).get(); + } + @Override public TriggerResult onElement(OnElementContext c) throws Exception { @@ -48,6 +53,11 @@ public TriggerResult onElement(OnElementContext c) return TriggerResult.CONTINUE; } + @Override + public void prefetchOnMerge(MergingStateContext state) { + state.mergingAccess(DELAYED_UNTIL_TAG).get(); + } + @Override public MergeResult onMerge(OnMergeContext c) throws Exception { // If the processing time timer has fired in any of the windows being merged, it would have @@ -59,7 +69,7 @@ public MergeResult onMerge(OnMergeContext c) throws Exception { // Otherwise, determine the earliest delay for all of the windows, and delay to that point. CombiningValueState mergingDelays = - c.state().accessAcrossMergingWindows(DELAYED_UNTIL_TAG); + c.state().mergingAccess(DELAYED_UNTIL_TAG); Instant earliestTimer = mergingDelays.get().read(); if (earliestTimer != null) { mergingDelays.clear(); @@ -70,6 +80,11 @@ public MergeResult onMerge(OnMergeContext c) throws Exception { return MergeResult.CONTINUE; } + @Override + public void prefetchOnTimer(StateContext state) { + state.access(DELAYED_UNTIL_TAG).get(); + } + @Override public TriggerResult onTimer(OnTimerContext c) throws Exception { if (c.timeDomain() != TimeDomain.SYNCHRONIZED_PROCESSING_TIME) { @@ -84,21 +99,6 @@ public TriggerResult onTimer(OnTimerContext c) throws Exception { return TriggerResult.FIRE_AND_FINISH; } - @Override - public void prefetchOnElement(StateContext state) { - state.access(DELAYED_UNTIL_TAG).get(); - } - - @Override - public void prefetchOnMerge(MergingStateContext state) { - state.accessAcrossMergingWindows(DELAYED_UNTIL_TAG).get(); - } - - @Override - public void prefetchOnTimer(StateContext state) { - state.access(DELAYED_UNTIL_TAG).get(); - } - @Override public void clear(TriggerContext c) throws Exception { CombiningValueState delayed = c.state().access(DELAYED_UNTIL_TAG); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermark.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermark.java index 8758768486047..5a7e9e782165a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermark.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermark.java @@ -94,6 +94,11 @@ private FromFirstElementInPane( super(delayFunction); } + @Override + public void prefetchOnElement(StateContext state) { + state.access(DELAYED_UNTIL_TAG).get(); + } + @Override public TriggerResult onElement(OnElementContext c) throws Exception { CombiningValueState delayUntilState = c.state().access(DELAYED_UNTIL_TAG); @@ -107,6 +112,11 @@ public TriggerResult onElement(OnElementContext c) throws Exception { return TriggerResult.CONTINUE; } + @Override + public void prefetchOnMerge(MergingStateContext state) { + state.mergingAccess(DELAYED_UNTIL_TAG).get(); + } + @Override public MergeResult onMerge(OnMergeContext c) throws Exception { // If the watermark time timer has fired in any of the windows being merged, it would have @@ -121,7 +131,7 @@ public MergeResult onMerge(OnMergeContext c) throws Exception { // of this first element in each pane). // Determine the earliest point across all the windows, and delay to that. CombiningValueState mergingDelays = - c.state().accessAcrossMergingWindows(DELAYED_UNTIL_TAG); + c.state().mergingAccess(DELAYED_UNTIL_TAG); Instant earliestTimer = mergingDelays.get().read(); if (earliestTimer != null) { mergingDelays.clear(); @@ -132,6 +142,11 @@ public MergeResult onMerge(OnMergeContext c) throws Exception { return MergeResult.CONTINUE; } + @Override + public void prefetchOnTimer(StateContext state) { + state.access(DELAYED_UNTIL_TAG).get(); + } + @Override public TriggerResult onTimer(OnTimerContext c) throws Exception { if (c.timeDomain() != TimeDomain.EVENT_TIME) { @@ -146,21 +161,6 @@ public TriggerResult onTimer(OnTimerContext c) throws Exception { return TriggerResult.FIRE_AND_FINISH; } - @Override - public void prefetchOnElement(StateContext state) { - state.access(DELAYED_UNTIL_TAG).get(); - } - - @Override - public void prefetchOnMerge(MergingStateContext state) { - state.accessAcrossMergingWindows(DELAYED_UNTIL_TAG).get(); - } - - @Override - public void prefetchOnTimer(StateContext state) { - state.access(DELAYED_UNTIL_TAG).get(); - } - @Override public void clear(TriggerContext c) throws Exception { CombiningValueState delayed = c.state().access(DELAYED_UNTIL_TAG); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ActiveWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ActiveWindowSet.java index f0b6a88e2b7f4..c5854ea4ea679 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ActiveWindowSet.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ActiveWindowSet.java @@ -19,65 +19,128 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import java.util.Collection; +import java.util.Set; + +import javax.annotation.Nullable; /** - * Tracks the windows that are active. + * Track which active windows have their state associated with merged-away windows. + * + * When windows are merged we must track which state previously associated with the merged windows + * must now be associated with the result window. Some of that state may be combined eagerly when + * the windows are merged. The rest is combined lazily when the final state is actually + * required when emitting a pane. We keep track of this using an {@link ActiveWindowSet}. + * + *

An element may belong to one or more windows. Each key may have zero or more windows + * corresponding to elements with that key. A window can be in one of five states: + *

    + *
  1. NEW: We have just encountered the window on an incoming element and do not yet know if + * it should be merged into an ACTIVE window since we have not yet called + * {@link WindowFn#mergeWindows}. + *
  2. EPHEMERAL: A NEW window has been merged into an ACTIVE window before any state has been + * associated with that window. Thus the window is neither ACTIVE nor MERGED. These windows + * are not persistently represented since if they reappear the merge function should again + * redirect them to an ACTIVE window. (We could collapse EPHEMERAL into MERGED, but keeping them + * separate cuts down on the number of windows we need to keep track of in the common case + * of SessionWindows over in-order events.) + *
  3. ACTIVE: A NEW window has state associated with it and has not itself been merged away. + * The window may have one or more 'state address' windows under which its non-empty state is + * stored. The true state for an ACTIVE window must be derived by reading all of the state in its + * state address windows. + *
  4. MERGED: An ACTIVE window has been merged into another ACTIVE window after it had state + * associated with it. The window will thus appear as a state address window for exactly one + * ACTIVE window. + *
  5. GARBAGE: The window has been garbage collected. No new elements (even late elements) will + * ever be assigned to that window. These windows are not explicitly represented anywhere. + * (Garbage collection is performed by {@link ReduceFnRunner#onTimer}). + *
* - * @param the types of windows being managed + *

If no windows will ever be merged we can use the dummy implementation {@link + * NonMergingActiveWindowSet}. Otherwise, the actual implementation of this data structure is in + * {@link MergingActiveWindowSet}. + * + * @param the type of window being managed */ public interface ActiveWindowSet { - /** * Callback for {@link #merge}. */ public interface MergeCallback { - void onMerge(Collection mergedWindows, W resultWindow, boolean isResultNew) throws Exception; + /** + * Called when windows are about to be merged. + * + * @param toBeMerged the windows about to be merged. + * @param activeToBeMerged the subset of {@code toBeMerged} corresponding to windows which + * are currently ACTIVE (and about to be merged). The remaining windows have been deemed + * EPHEMERAL. + * @param mergeResult the result window, either a member of {@code toBeMerged} or new. + */ + void onMerge(Collection toBeMerged, Collection activeToBeMerged, W mergeResult) + throws Exception; } + /** + * Remove EPHEMERAL windows since we only need to know about them while processing new elements. + */ + void removeEphemeralWindows(); + /** * Save any state changes needed. */ void persist(); /** - * Add a window to the {@code ActiveWindowSet}. - * - * @return false if the window was definitely not-active before being added, true if it either - * was already active, or the implementation doesn't have enough information to know. + * Return the ACTIVE window into which {@code window} has been merged. + * Return {@code window} itself if it is ACTIVE. Return null if {@code window} has not + * yet been seen. + */ + @Nullable + W representative(W window); + + /** + * Return (a view of) the set of currently ACTIVE windows. + */ + Set getActiveWindows(); + + /** + * Return {@code true} if {@code window} is ACTIVE. + */ + boolean isActive(W window); + + /** + * If {@code window} is not already known to be ACTIVE, MERGED or EPHEMERAL then add it + * as NEW. All NEW windows will be accounted for as ACTIVE, MERGED or EPHEMERAL by a call + * to {@link #merge}. */ - boolean add(W window); + void addNew(W window); /** - * Return true if the window is active. + * If {@code window} is not already known to be ACTIVE, MERGED or EPHEMERAL then add it + * as ACTIVE. */ - boolean contains(W window); + void addActive(W window); /** - * Remove a window from the {@code ActiveWindowSet}. + * Remove {@code window} from the set. */ void remove(W window); /** * Invoke {@link WindowFn#mergeWindows} on the {@code WindowFn} associated with this window set, * merging as many of the active windows as possible. {@code mergeCallback} will be invoked for - * each group of windows that are merged. + * each group of windows that are merged. After this no NEW windows will remain. */ void merge(MergeCallback mergeCallback) throws Exception; /** - * Return the set of windows that were merged to produce {@code window}. If the associated - * {@code WindowFn} never merges windows, then this should return the singleton list containing - * {@code window}. - */ - Iterable sourceWindows(W window); - - /** - * Return the subset of {@code windows} that existed in the original merge tree. + * Return the state address windows for ACTIVE {@code window} from which all state associated + * should be read and merged. */ - Collection originalWindows(Collection windows); + Set readStateAddresses(W window); /** - * Return the number of windows that are currently active. + * Return the state address window of ACTIVE {@code window} into which all new state should be + * written. Always one of the results of {@link #readStateAddresses}. */ - int size(); + W writeStateAddress(W window); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/MergingActiveWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/MergingActiveWindowSet.java index 2d2ba5f370615..ebf1540e6e4bb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/MergingActiveWindowSet.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/MergingActiveWindowSet.java @@ -24,86 +24,184 @@ import com.google.cloud.dataflow.sdk.util.state.StateTag; import com.google.cloud.dataflow.sdk.util.state.StateTags; import com.google.cloud.dataflow.sdk.util.state.ValueState; -import com.google.common.base.Predicates; -import com.google.common.collect.Collections2; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; +import javax.annotation.Nullable; + /** - * Implementation of {@link ActiveWindowSet} used with {@link WindowFn WindowFns} that support - * merging. + * Implementation of {@link ActiveWindowSet} for use with {@link WindowFn}s that support + * merging. In effect maintains an equivalence class of windows (where two windows which have + * been merged are in the same class), but also manages which windows contain state which + * must be merged when a pane is fired. + * + *

Note that this object must be serialized and stored when work units are committed such + * that subsequent work units can recover the equivalence classes etc. * - * @param the types of windows being managed + * @param the type of window being managed */ -public class MergingActiveWindowSet - implements ActiveWindowSet { - +public class MergingActiveWindowSet implements ActiveWindowSet { private final WindowFn windowFn; /** - * A map of live windows to windows that were merged into them. + * A map from ACTIVE windows to their state address windows. Writes to the ACTIVE window + * state can be redirected to any one of the state address windows. Reads need to merge + * from all state address windows. If the set is empty then the window is NEW. * - *

The keys of the map correspond to the set of (merged) windows and the values - * are the no-longer-present windows that were merged into the keys. A given - * window can appear in both the key and value of a single entry, but other at - * most once across all keys and values. + *

    + *
  • The state address windows will be empty if the window is NEW, we don't yet know what other + * windows it may be merged into, and the window does not yet have any state associated with it. + * In this way we can distinguish between MERGED and EPHEMERAL windows when merging. + *
  • The state address windows will contain just the window itself it it has never been merged + * but has state. + *
  • It is possible none of the state address windows correspond to the window itself. For + * example, two windows W1 and W2 with state may be merged to form W12. From then on additional + * state can be added to just W1 or W2. Thus the state address windows for W12 do not need to + * include W12. + *
  • If W1 is in the set for W2 then W1 is not a state address window of any other active + * window. Furthermore W1 will map to W2 in {@link #windowToActiveWindow}. + *
*/ - private final Map> mergeTree; + @Nullable + private Map> activeWindowToStateAddressWindows; /** - * Used to determine if writing the mergeTree (which is relatively stable) - * is necessary. + * As above, but only for EPHEMERAL windows. Does not need to be persisted. */ - private final Map> originalMergeTree; + private final Map> activeWindowToEphemeralWindows; - private final ValueState>> mergeTreeValue; + /** + * A map from window to the ACTIVE window it has been merged into. + * + *

Does not need to be persisted. + * + *

    + *
  • Key window may be ACTIVE, MERGED or EPHEMERAL. + *
  • ACTIVE windows map to themselves. + *
  • If W1 maps to W2 then W2 is in {@link #activeWindowToStateAddressWindows}. + *
  • If W1 = W2 then W1 is ACTIVE. If W1 is in the state address window set for W2 then W1 is + * MERGED. Otherwise W1 is EPHEMERAL. + *
+ */ + @Nullable + private Map windowToActiveWindow; + + /** + * Deep clone of {@link #activeWindowToStateAddressWindows} as of last commit. + * + *

Used to avoid writing to state if no changes have been made during the work unit. + */ + @Nullable + private Map> originalActiveWindowToStateAddressWindows; + + /** + * Handle representing our state in the backend. + */ + private final ValueState>> valueState; public MergingActiveWindowSet(WindowFn windowFn, StateInternals state) { this.windowFn = windowFn; - StateTag>>> mergeTreeAddr = StateTags.makeSystemTagInternal( - StateTags.value("tree", - MapCoder.of(windowFn.windowCoder(), SetCoder.of(windowFn.windowCoder())))); - this.mergeTreeValue = state.state(StateNamespaces.global(), mergeTreeAddr); - this.mergeTree = emptyIfNull(mergeTreeValue.get().read()); + StateTag>>> mergeTreeAddr = + StateTags.makeSystemTagInternal(StateTags.value( + "tree", MapCoder.of(windowFn.windowCoder(), SetCoder.of(windowFn.windowCoder())))); + valueState = state.state(StateNamespaces.global(), mergeTreeAddr); + // Little use trying to prefetch this state since the ReduceFnRunner is stymied until it is + // available. + activeWindowToStateAddressWindows = emptyIfNull(valueState.get().read()); + activeWindowToEphemeralWindows = new HashMap<>(); + originalActiveWindowToStateAddressWindows = deepCopy(activeWindowToStateAddressWindows); + windowToActiveWindow = invert(activeWindowToStateAddressWindows); + } - originalMergeTree = deepCopy(mergeTree); + @Override + public void removeEphemeralWindows() { + for (Map.Entry> entry : activeWindowToEphemeralWindows.entrySet()) { + for (W ephemeral : entry.getValue()) { + windowToActiveWindow.remove(ephemeral); + } + } + activeWindowToEphemeralWindows.clear(); } @Override public void persist() { - if (!mergeTree.equals(originalMergeTree)) { - mergeTreeValue.set(mergeTree); + if (activeWindowToStateAddressWindows.equals(originalActiveWindowToStateAddressWindows)) { + // No change. + return; + } + // All NEW windows must have been accounted for. + for (Map.Entry> entry : activeWindowToStateAddressWindows.entrySet()) { + Preconditions.checkState( + !entry.getValue().isEmpty(), "Cannot persist NEW window %s", entry.getKey()); } + // Should be no EPHEMERAL windows. + Preconditions.checkState( + activeWindowToEphemeralWindows.isEmpty(), "Unexpected EPHEMERAL windows before persist"); + + valueState.set(activeWindowToStateAddressWindows); + // No need to update originalActiveWindowToStateAddressWindows since this object is about to + // become garbage. } @Override - public boolean contains(W window) { - return mergeTree.containsKey(window); + @Nullable + public W representative(W window) { + return windowToActiveWindow.get(window); } @Override - public boolean add(W window) { - if (mergeTree.containsKey(window)) { - return false; + public Set getActiveWindows() { + return activeWindowToStateAddressWindows.keySet(); + } + + @Override + public boolean isActive(W window) { + return activeWindowToStateAddressWindows.containsKey(window); + } + + @Override + public void addNew(W window) { + if (!windowToActiveWindow.containsKey(window)) { + activeWindowToStateAddressWindows.put(window, new HashSet()); } + } - mergeTree.put(window, new HashSet()); - return true; + @Override + public void addActive(W window) { + if (!windowToActiveWindow.containsKey(window)) { + Set stateAddressWindows = new HashSet<>(); + stateAddressWindows.add(window); + activeWindowToStateAddressWindows.put(window, stateAddressWindows); + windowToActiveWindow.put(window, window); + } } @Override public void remove(W window) { - mergeTree.remove(window); + for (W stateAddressWindow : activeWindowToStateAddressWindows.get(window)) { + windowToActiveWindow.remove(stateAddressWindow); + } + activeWindowToStateAddressWindows.remove(window); + Set ephemeralWindows = activeWindowToEphemeralWindows.get(window); + if (ephemeralWindows != null) { + for (W ephemeralWindow : ephemeralWindows) { + windowToActiveWindow.remove(ephemeralWindow); + } + activeWindowToEphemeralWindows.remove(window); + } + windowToActiveWindow.remove(window); } private class MergeContextImpl extends WindowFn.MergeContext { - private MergeCallback mergeCallback; public MergeContextImpl(MergeCallback mergeCallback) { @@ -113,81 +211,250 @@ public MergeContextImpl(MergeCallback mergeCallback) { @Override public Collection windows() { - return mergeTree.keySet(); + return activeWindowToStateAddressWindows.keySet(); } @Override public void merge(Collection toBeMerged, W mergeResult) throws Exception { - boolean isResultNew = !mergeTree.containsKey(mergeResult); - recordMerge(toBeMerged, mergeResult); - mergeCallback.onMerge(toBeMerged, mergeResult, isResultNew); + recordMerge(mergeCallback, toBeMerged, mergeResult); } } @Override public void merge(MergeCallback mergeCallback) throws Exception { + // See what the window function does with the NEW and already ACTIVE windows. windowFn.mergeWindows(new MergeContextImpl(mergeCallback)); + + for (Map.Entry> entry : activeWindowToStateAddressWindows.entrySet()) { + if (entry.getValue().isEmpty()) { + // This window was NEW but since it survived merging must now become ACTIVE. + W window = entry.getKey(); + entry.getValue().add(window); + windowToActiveWindow.put(window, window); + } + } } - @Override - public Iterable sourceWindows(W window) { - Set curWindows = new HashSet<>(); - curWindows.add(window); + /** + * A {@code WindowFn.mergeWindows} call has requested {@code toBeMerged} (which must + * all be ACTIVE} be considered equivalent to {@code activeWindow} (which is either a + * member of {@code toBeMerged} or is a new window). + */ + private void recordMerge(MergeCallback mergeCallback, Collection toBeMerged, W mergeResult) + throws Exception { + Set newStateAddressWindows = new HashSet<>(); + Set existingStateAddressWindows = activeWindowToStateAddressWindows.get(mergeResult); + if (existingStateAddressWindows != null) { + // Preserve all the existing state address windows for mergeResult. + newStateAddressWindows.addAll(existingStateAddressWindows); + } - Set sourceWindows = mergeTree.get(window); - if (sourceWindows != null) { - curWindows.addAll(sourceWindows); + Set newEphemeralWindows = new HashSet<>(); + Set existingEphemeralWindows = activeWindowToEphemeralWindows.get(mergeResult); + if (existingEphemeralWindows != null) { + // Preserve all the existing EPHEMERAL windows for meregResult. + newEphemeralWindows.addAll(existingEphemeralWindows); } - return curWindows; + + Collection activeToBeMerged = new ArrayList<>(); + + for (W other : toBeMerged) { + Set otherStateAddressWindows = activeWindowToStateAddressWindows.get(other); + Preconditions.checkState(otherStateAddressWindows != null, "Window %s is not ACTIVE", other); + + for (W otherStateAddressWindow : otherStateAddressWindows) { + // Since otherTarget equiv other AND other equiv mergeResult + // THEN otherTarget equiv mergeResult. + newStateAddressWindows.add(otherStateAddressWindow); + windowToActiveWindow.put(otherStateAddressWindow, mergeResult); + } + activeWindowToStateAddressWindows.remove(other); + + Set otherEphemeralWindows = activeWindowToEphemeralWindows.get(other); + if (otherEphemeralWindows != null) { + for (W otherEphemeral : otherEphemeralWindows) { + // Since otherEphemeral equiv other AND other equiv mergeResult + // THEN otherEphemeral equiv mergeResult. + newEphemeralWindows.add(otherEphemeral); + windowToActiveWindow.put(otherEphemeral, mergeResult); + } + } + activeWindowToEphemeralWindows.remove(other); + + // Now other equiv mergeResult. + if (otherStateAddressWindows.contains(other)) { + // Other was ACTIVE and is now known to be MERGED. + newStateAddressWindows.add(other); + activeToBeMerged.add(other); + } else if (otherStateAddressWindows.isEmpty()) { + // Other was NEW thus has no state. It is now EPHEMERAL. + newEphemeralWindows.add(other); + } else if (other.equals(mergeResult)) { + // Other was ACTIVE, was never used to store elements, but is still ACTIVE. + // Leave it as active. + activeToBeMerged.add(other); + } else { + // Other was ACTIVE, was never used to store element, as is no longer considered ACTIVE. + // It is now EPHEMERAL. + newEphemeralWindows.add(other); + // However, since it may have metadata state, include it in the ACTIVE to be merged set. + activeToBeMerged.add(other); + } + windowToActiveWindow.put(other, mergeResult); + } + + if (newStateAddressWindows.isEmpty()) { + // If stateAddressWindows is empty then toBeMerged must have only contained EPHEMERAL windows. + // Promote mergeResult to be active now. + newStateAddressWindows.add(mergeResult); + } + windowToActiveWindow.put(mergeResult, mergeResult); + + activeWindowToStateAddressWindows.put(mergeResult, newStateAddressWindows); + if (!newEphemeralWindows.isEmpty()) { + activeWindowToEphemeralWindows.put(mergeResult, newEphemeralWindows); + } + + mergeCallback.onMerge(toBeMerged, activeToBeMerged, mergeResult); + } + + /** + * Return the state address windows for ACTIVE {@code window} from which all state associated + * should + * be read and merged. + */ + @Override + public Set readStateAddresses(W window) { + Set stateAddressWindows = activeWindowToStateAddressWindows.get(window); + Preconditions.checkState(stateAddressWindows != null, "Window %s is not ACTIVE", window); + return stateAddressWindows; + } + + /** + * Return the state address window of ACTIVE {@code window} into which all new state should be + * written. + */ + @Override + public W writeStateAddress(W window) { + Set stateAddressWindows = activeWindowToStateAddressWindows.get(window); + Preconditions.checkState(stateAddressWindows != null, "Window %s is not ACTIVE", window); + W result = Iterables.getFirst(stateAddressWindows, null); + Preconditions.checkState(result != null, "Window %s is still NEW", window); + return result; } - private void recordMerge(Collection otherWindows, W newWindow) throws Exception { - Set subWindows = mergeTree.get(newWindow); - if (subWindows == null) { - subWindows = new HashSet<>(); + @VisibleForTesting + public void checkInvariants() { + Set knownStateAddressWindows = new HashSet<>(); + for (Map.Entry> entry : activeWindowToStateAddressWindows.entrySet()) { + W active = entry.getKey(); + Preconditions.checkState(!entry.getValue().isEmpty(), + "Unexpected empty state address window set for ACTIVE window %s", active); + for (W stateAddressWindow : entry.getValue()) { + Preconditions.checkState(knownStateAddressWindows.add(stateAddressWindow), + "%s is in more than one state address window set", stateAddressWindow); + Preconditions.checkState(active.equals(windowToActiveWindow.get(stateAddressWindow)), + "%s should have %s as its ACTIVE window", stateAddressWindow, active); + } } + for (Map.Entry> entry : activeWindowToEphemeralWindows.entrySet()) { + W active = entry.getKey(); + Preconditions.checkState(activeWindowToStateAddressWindows.containsKey(active), + "%s must be ACTIVE window", active); + Preconditions.checkState( + !entry.getValue().isEmpty(), "Unexpected empty EPHEMERAL set for %s", active); + for (W ephemeralWindow : entry.getValue()) { + Preconditions.checkState(knownStateAddressWindows.add(ephemeralWindow), + "%s is EPHEMERAL/state address of more than one ACTIVE window", ephemeralWindow); + Preconditions.checkState(active.equals(windowToActiveWindow.get(ephemeralWindow)), + "%s should have %s as its ACTIVE window", ephemeralWindow, active); + } + } + for (Map.Entry entry : windowToActiveWindow.entrySet()) { + Preconditions.checkState(activeWindowToStateAddressWindows.containsKey(entry.getValue()), + "%s should be ACTIVE since representative for %s", entry.getValue(), entry.getKey()); + } + } - for (W other : otherWindows) { - if (!mergeTree.containsKey(other)) { - throw new IllegalArgumentException("Tried to merge a non-existent window: " + other); + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("MergingActiveWindowSet {\n"); + for (Map.Entry> entry : activeWindowToStateAddressWindows.entrySet()) { + W active = entry.getKey(); + Set stateAddressWindows = entry.getValue(); + if (stateAddressWindows.isEmpty()) { + sb.append(" NEW "); + sb.append(active); + sb.append('\n'); + } else { + sb.append(" ACTIVE "); + sb.append(active); + sb.append(":\n"); + for (W stateAddressWindow : stateAddressWindows) { + if (stateAddressWindow.equals(active)) { + sb.append(" ACTIVE "); + } else { + sb.append(" MERGED "); + } + sb.append(stateAddressWindow); + sb.append("\n"); + W active2 = windowToActiveWindow.get(stateAddressWindow); + Preconditions.checkState(active2.equals(active)); + } + Set ephemeralWindows = activeWindowToEphemeralWindows.get(active); + if (ephemeralWindows != null) { + for (W ephemeralWindow : ephemeralWindows) { + sb.append(" EPHEMERAL "); + sb.append(ephemeralWindow); + sb.append('\n'); + } + } } - subWindows.addAll(mergeTree.get(other)); - subWindows.add(other); - mergeTree.remove(other); } - mergeTree.put(newWindow, subWindows); + sb.append("}"); + return sb.toString(); } + // ====================================================================== - private static Map> emptyIfNull(Map> input) { - if (input == null) { + /** + * Replace null {@code multimap} with empty map, and replace null entries in {@code multimap} with + * empty sets. + */ + private static Map> emptyIfNull(Map> multimap) { + if (multimap == null) { return new HashMap<>(); } else { - for (Map.Entry> entry : input.entrySet()) { + for (Map.Entry> entry : multimap.entrySet()) { if (entry.getValue() == null) { entry.setValue(new HashSet()); } } - return input; + return multimap; } } - private Map> deepCopy(Map> mergeTree) { - Map> newMergeTree = new HashMap<>(); - for (Map.Entry> entry : mergeTree.entrySet()) { - newMergeTree.put(entry.getKey(), new HashSet(entry.getValue())); + /** Return a deep copy of {@code multimap}. */ + private static Map> deepCopy(Map> multimap) { + Map> newMultimap = new HashMap<>(); + for (Map.Entry> entry : multimap.entrySet()) { + newMultimap.put(entry.getKey(), new HashSet(entry.getValue())); } - return newMergeTree; - } - - @Override - public int size() { - return mergeTree.size(); + return newMultimap; } - @Override - public Collection originalWindows(Collection windows) { - return Collections.unmodifiableCollection( - Collections2.filter(windows, Predicates.in(originalMergeTree.keySet()))); + /** Return inversion of {@code multimap}, which must be invertible. */ + private static Map invert(Map> multimap) { + Map result = new HashMap<>(); + for (Map.Entry> entry : multimap.entrySet()) { + W active = entry.getKey(); + for (W target : entry.getValue()) { + W previous = result.put(target, active); + Preconditions.checkState(previous == null, + "Window %s has both %s and %s as representatives", target, previous, active); + } + } + return result; } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NonEmptyPanes.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NonEmptyPanes.java index c4f39a9b23813..f567888260768 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NonEmptyPanes.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NonEmptyPanes.java @@ -97,7 +97,7 @@ private static class GeneralNonEmptyPanes extends NonEm @Override public void recordContent(ReduceFn.Context context) { - context.state().access(PANE_ADDITIONS_TAG).add(1L); + context.state().accessAcrossMergedWindows(PANE_ADDITIONS_TAG).add(1L); } @Override @@ -107,6 +107,8 @@ public void clearPane(ReduceFn.Context context) { @Override public StateContents isEmpty(ReduceFn.Context context) { + // Since we only check for empty element sets when a trigger fires it's unreasonable + // to require a prefetch. return context.state().accessAcrossMergedWindows(PANE_ADDITIONS_TAG).isEmpty(); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NonMergingActiveWindowSet.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NonMergingActiveWindowSet.java index 2c453ef95b934..10d7666f5dd53 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NonMergingActiveWindowSet.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/NonMergingActiveWindowSet.java @@ -17,9 +17,9 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; +import com.google.common.collect.ImmutableSet; -import java.util.Collection; -import java.util.Collections; +import java.util.Set; /** * Implementation of {@link ActiveWindowSet} used with {@link WindowFn WindowFns} that don't support @@ -27,48 +27,51 @@ * * @param the types of windows being managed */ -public class NonMergingActiveWindowSet - implements ActiveWindowSet { +public class NonMergingActiveWindowSet implements ActiveWindowSet { + @Override + public void removeEphemeralWindows() {} + + @Override + public void persist() {} @Override - public void persist() { - // Nothing to persist. + public W representative(W window) { + // Always represented by itself. + return window; } @Override - public boolean add(W window) { - // We don't track anything, so we cannot determine if the window is new or not. - return true; + public Set getActiveWindows() { + // Only supported when merging. + throw new java.lang.UnsupportedOperationException(); } @Override - public boolean contains(W window) { + public boolean isActive(W window) { // Windows should never disappear, since we don't support merging. return true; } @Override - public void remove(W window) {} + public void addNew(W window) {} @Override - public void merge(MergeCallback reduceFnRunner) throws Exception { - // We never merge, so there is nothing to do here. - } + public void addActive(W window) {} @Override - public Iterable sourceWindows(W window) { - // There is no merging, so the only source window is the window itself. - return Collections.singleton(window); - } + public void remove(W window) {} + + @Override + public void merge(MergeCallback mergeCallback) throws Exception {} + @Override - public int size() { - throw new UnsupportedOperationException("Cannot determine size of NonMergingActiveWindowSet"); + public Set readStateAddresses(W window) { + return ImmutableSet.of(window); } @Override - public Collection originalWindows(Collection windows) { - throw new UnsupportedOperationException( - "Cannot determine original windows of NonMergingActiveWindowSet"); + public W writeStateAddress(W window) { + return window; } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFn.java index b8a13f96e8a9b..e76868aa42f38 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFn.java @@ -39,17 +39,24 @@ */ public abstract class ReduceFn implements Serializable { - /** Interface for interacting with persistent state. */ public interface StateContext { - /** Access the storage for the given {@code address} in the current window. */ + /** + * Access the storage for the given {@code address} in the current window. + * + *

Never accounts for merged windows. When windows are merged, any state accessed via + * this method must be eagerly combined and written into the result window. + */ StateT access(StateTag address); /** * Access the storage for the given {@code address} in all of the windows that were - * merged into the current window including the current window. + * merged into the current window. * - *

If no windows were merged, this reads from just the current window. + *

If no windows were merged, this reads and writes to just the current window. + * Otherwise, when windows merge we do not eagerly combine state, but rather defer the + * combination to reading time. Thus reads will be from all 'merged windows' for the + * current window, and writes will be to the designated 'writing window' for the current window. */ > StateT accessAcrossMergedWindows( StateTag address); @@ -58,15 +65,16 @@ public interface StateContext { /** Interface for interacting with persistent state within {@link #onMerge}. */ public interface MergingStateContext extends StateContext { /** - * Access a merged view of the storage for the given {@code address} - * in all of the windows being merged. + * Analogous to {@link #access}, but across all windows which are about to be merged. */ - public abstract > StateT accessAcrossMergingWindows( - StateTag address); + > StateT mergingAccess(StateTag address); - /** Access a map from windows being merged to the associated {@code StateT}. */ - public abstract Map accessInEachMergingWindow( - StateTag address); + /** + * Analogous to {@link #access}, but returned as a map from each window which is + * about to be merged to the corresponding state. + */ + public abstract Map + mergingAccessInEachMergingWindow(StateTag address); } /** @@ -117,7 +125,6 @@ public abstract class Context { /** Information accessible within {@link #processValue}. */ public abstract class ProcessValueContext extends Context { - /** Return the actual value being processed. */ public abstract InputT value(); @@ -141,7 +148,6 @@ public abstract class OnMergeContext extends Context { /** Information accessible within {@link #onTrigger}. */ public abstract class OnTriggerContext extends Context { - /** Returns the {@link PaneInfo} for the trigger firing being processed. */ public abstract PaneInfo paneInfo(); @@ -185,7 +191,7 @@ public abstract class OnTriggerContext extends Context { * * @param c Context to use prefetch from. */ - public void prefetchOnTrigger(StateContext c) { } + public void prefetchOnTrigger(StateContext c) {} /** * Called to clear any persisted state that the {@link ReduceFn} may be holding. This will be diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnContextFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnContextFactory.java index df4a853599334..643919035f407 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnContextFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnContextFactory.java @@ -124,7 +124,7 @@ static class StateContextImpl private final ActiveWindowSet activeWindows; private final W window; - protected StateNamespace namespace; + protected StateNamespace windowNamespace; protected final Coder windowCoder; private final StateInternals stateInternals; @@ -137,7 +137,7 @@ public StateContextImpl( this.windowCoder = windowCoder; this.stateInternals = stateInternals; this.window = window; - this.namespace = namespaceFor(window); + this.windowNamespace = namespaceFor(window); } protected StateNamespace namespaceFor(W window) { @@ -149,23 +149,23 @@ W window() { } StateNamespace namespace() { - return namespace; + return windowNamespace; } @Override public StorageT access(StateTag address) { - return stateInternals.state(namespace, address); + return stateInternals.state(windowNamespace, address); } @Override public > StorageT accessAcrossMergedWindows( StateTag address) { - List sourceNamespaces = new ArrayList<>(); - for (W sourceWindow : activeWindows.sourceWindows(window)) { - sourceNamespaces.add(namespaceFor(sourceWindow)); + List readNamespaces = new ArrayList<>(); + for (W readWindow : activeWindows.readStateAddresses(window)) { + readNamespaces.add(namespaceFor(readWindow)); } - - return stateInternals.mergedState(sourceNamespaces, namespace, address, window); + StateNamespace writeNamespace = namespaceFor(activeWindows.writeStateAddress(window)); + return stateInternals.mergedState(readNamespaces, writeNamespace, address, window); } } @@ -180,10 +180,6 @@ public MergingStateContextImpl(StateContextImpl delegate, Collection mergi this.mergingWindows = mergingWindows; } - StateNamespace namespace() { - return delegate.namespace; - } - W window() { return delegate.window(); } @@ -204,19 +200,17 @@ public StorageT access(StateTag address) { } @Override - public > StateT accessAcrossMergingWindows( - StateTag address) { - List mergingNamespaces = new ArrayList<>(); + public > StateT mergingAccess(StateTag address) { + List readNamespaces = new ArrayList<>(); for (W mergingWindow : mergingWindows) { - mergingNamespaces.add(delegate.namespaceFor(mergingWindow)); + readNamespaces.add(delegate.namespaceFor(mergingWindow)); } - return delegate.stateInternals.mergedState( - mergingNamespaces, delegate.namespace, address, delegate.window()); + readNamespaces, delegate.windowNamespace, address, delegate.window); } @Override - public Map accessInEachMergingWindow( + public Map mergingAccessInEachMergingWindow( StateTag address) { ImmutableMap.Builder builder = ImmutableMap.builder(); for (W mergingWindow : mergingWindows) { @@ -236,7 +230,7 @@ private class ContextImpl extends ReduceFn.Context { private ContextImpl(StateContextImpl state) { reduceFn.super(); this.state = state; - this.timers = new TimersImpl(state.namespace); + this.timers = new TimersImpl(state.namespace()); } @Override @@ -278,7 +272,7 @@ private ProcessValueContextImpl(StateContextImpl state, InputT value, Instant this.state = state; this.value = value; this.timestamp = timestamp; - this.timers = new TimersImpl(state.namespace); + this.timers = new TimersImpl(state.namespace()); } @Override @@ -331,7 +325,7 @@ private OnTriggerContextImpl(StateContextImpl state, this.state = state; this.pane = pane; this.callbacks = callbacks; - this.timers = new TimersImpl(state.namespace); + this.timers = new TimersImpl(state.namespace()); } @Override @@ -379,7 +373,7 @@ private class OnMergeContextImpl private OnMergeContextImpl(MergingStateContextImpl state) { reduceFn.super(); this.state = state; - this.timers = new TimersImpl(state.delegate.namespace); + this.timers = new TimersImpl(state.delegate.namespace()); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java index 4d11e8a708110..d1276800f58bf 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReduceFnRunner.java @@ -18,10 +18,12 @@ import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.GroupByKey.GroupByKeyOnly; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo.Timing; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.TriggerResult; import com.google.cloud.dataflow.sdk.transforms.windowing.Window.ClosingBehavior; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.ActiveWindowSet.MergeCallback; import com.google.cloud.dataflow.sdk.util.ReduceFnContextFactory.OnTriggerCallbacks; import com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData; @@ -31,21 +33,17 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Functions; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; import com.google.common.base.Throwables; -import com.google.common.collect.FluentIterable; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import org.joda.time.Duration; import org.joda.time.Instant; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -79,33 +77,110 @@ public class ReduceFnRunner { public static final String DROPPED_DUE_TO_LATENESS_COUNTER = "DroppedDueToLateness"; private final WindowingStrategy windowingStrategy; - private final TimerInternals timerInternals; + private final WindowingInternals> windowingInternals; private final Aggregator droppedDueToClosedWindow; private final Aggregator droppedDueToLateness; - private final TriggerRunner triggerRunner; - private final K key; + + /** + * Track which windows are still active and which 'state address' windows contain state + * for a merged window. + * + *

In general, when windows are merged we prefer to defer merging their state until the + * overall state is needed. In other words, we prefer to merge state 'lazily' (on read) + * instead of 'eagerly' (on merge). + */ private final ActiveWindowSet activeWindows; + + /** + * User's reduce function (or {@link SystemReduceFn} for simple GroupByKey operations). + * May store its own state. + * + *

    + *
  • Merging: Uses {@link #activeWindows} to determine the 'state address' windows under which + * state is read and written. Merging may be done lazily, in which case state is merged + * only when a pane fires. + *
  • Lifetime: Possibly cleared when a pane fires. Always cleared when a window is + * garbage collected. + *
+ */ + private final ReduceFn reduceFn; + + /** + * Manage the setting and firing of timer events. + * + *
    + *
  • Merging: Timers are cancelled when windows are merged away. + *
  • Lifetime: Timers automatically disappear after they fire. + *
+ */ + private final TimerInternals timerInternals; + + /** + * Manage the execution and state for triggers. + * + *
    + *
  • Merging: All state is keyed by actual window, so does not depend on {@link #activeWindows}. + * Individual triggers know how to eagerly merge their state on merge. + *
  • Lifetime: Most trigger state is cleared when the final pane is emitted. However + * a tombstone is left behind which must be cleared when the window is garbage collected. + *
+ */ + private final TriggerRunner triggerRunner; + + /** + * Store the output watermark holds for each window. + * + *
    + *
  • Merging: Generally uses {@link #activeWindows} to maintain the 'state address' windows + * under which holds are stored, and holds are merged lazily only when a pane fires. + * However there are two special cases: + *
      + *
    • Depending on the window's {@link OutputTimeFn}, it is possible holds need to be read, + * recalculated, cleared, and added back on merging. + *
    • When a pane fires it may be necessary to add (back) an end-of-window or + * garbage collection hold. If the current window is no longer active these holds will + * be associated with the current window. + *
    + *
  • Lifetime: Cleared when a pane fires or when the window is garbage collected. + *
+ */ private final WatermarkHold watermarkHold; + private final ReduceFnContextFactory contextFactory; - private final ReduceFn reduceFn; - private final PaneInfoTracker paneInfo; + + /** + * Store the previously emitted pane (if any) for each window. + * + *
    + *
  • Merging: Always keyed by actual window, so does not depend on {@link #activeWindows}. + * Cleared when window is merged away. + *
  • Lifetime: Cleared when trigger is finished or window is garbage collected. + *
+ */ + private final PaneInfoTracker paneInfoTracker; + + /** + * Store whether we've seen any elements for a window since the last pane was emitted. + * + *
    + *
  • Merging: Uses {@link #activeWindows} determine the state address windows under which + * counts are stored. Merging is done lazily when checking if a pane needs to fire. + *
  • Lifetime: Cleared when pane fires or window is garbage collected. + *
+ */ private final NonEmptyPanes nonEmptyPanes; - public ReduceFnRunner( - K key, - WindowingStrategy windowingStrategy, - TimerInternals timerInternals, - WindowingInternals> windowingInternals, - Aggregator droppedDueToClosedWindow, - Aggregator droppedDueToLateness, + public ReduceFnRunner(K key, WindowingStrategy windowingStrategy, + TimerInternals timerInternals, WindowingInternals> windowingInternals, + Aggregator droppedDueToClosedWindow, Aggregator droppedDueToLateness, ReduceFn reduceFn) { this.key = key; this.timerInternals = timerInternals; - this.paneInfo = new PaneInfoTracker(timerInternals); + this.paneInfoTracker = new PaneInfoTracker(timerInternals); this.windowingInternals = windowingInternals; this.droppedDueToClosedWindow = droppedDueToClosedWindow; this.droppedDueToLateness = droppedDueToLateness; @@ -117,42 +192,67 @@ public ReduceFnRunner( this.windowingStrategy = objectWindowingStrategy; this.nonEmptyPanes = NonEmptyPanes.create(this.windowingStrategy, this.reduceFn); + // Note this may trigger a GetData request to load the existing window set. this.activeWindows = createActiveWindowSet(); - this.contextFactory = new ReduceFnContextFactory( - key, reduceFn, this.windowingStrategy, this.windowingInternals.stateInternals(), - this.activeWindows, timerInternals); + this.contextFactory = + new ReduceFnContextFactory(key, reduceFn, this.windowingStrategy, + this.windowingInternals.stateInternals(), this.activeWindows, timerInternals); this.watermarkHold = new WatermarkHold<>(timerInternals, windowingStrategy); this.triggerRunner = new TriggerRunner<>( windowingStrategy.getTrigger(), - new TriggerContextFactory<>(windowingStrategy, this.windowingInternals.stateInternals(), - activeWindows)); + new TriggerContextFactory<>( + windowingStrategy, this.windowingInternals.stateInternals(), activeWindows)); } private ActiveWindowSet createActiveWindowSet() { return windowingStrategy.getWindowFn().isNonMerging() - ? new NonMergingActiveWindowSet() - : new MergingActiveWindowSet( - windowingStrategy.getWindowFn(), windowingInternals.stateInternals()); + ? new NonMergingActiveWindowSet() : new MergingActiveWindowSet( + windowingStrategy.getWindowFn(), windowingInternals.stateInternals()); } - @VisibleForTesting boolean isFinished(W window) { + @VisibleForTesting + boolean isFinished(W window) { return triggerRunner.isClosed(contextFactory.base(window).state()); } + /** + * Incorporate {@code values} into the underlying reduce function, and manage holds, timers, + * triggers, and window merging. + * + *

The general strategy is: + *

    + *
  1. Use {@link WindowedValue#getWindows} (itself determined using + * {@link WindowFn#assignWindows}) to determine which windows each element belongs to. Some of + * those windows will already have state associated with them. The rest are considered NEW. + *
  2. Use {@link WindowFn#mergeWindows} to attempt to merge currently ACTIVE and NEW windows. + * Each NEW window will become either ACTIVE, MERGED, or EPHEMERAL. (See {@link ActiveWindowSet} + * for definitions of these terms.) + *
  3. If at all possible, eagerly substitute EPHEMERAL windows with their ACTIVE state address + * windows before any state is associated with the EPHEMERAL window. In the common case that + * windows for new elements are merged into existing ACTIVE windows then no additional storage + * or merging overhead will be incurred. + *
  4. Otherwise, keep track of the state address windows for ACTIVE windows so that their + * states can be merged on-demand when a pane fires. + *
  5. Process the element for each of the windows it's windows have been merged into according + * to {@link ActiveWindowSet}. Processing may require running triggers, setting timers, setting + * holds, and invoking {@link ReduceFn#onTrigger}. + *
+ */ public void processElements(Iterable> values) { - Function windowMapping = Functions.identity(); - + // Map from element window to the result of running its trigger. final Map results = Maps.newHashMap(); - // If windows might merge, extract the windows from all the values, and pre-merge them. if (!windowingStrategy.getWindowFn().isNonMerging()) { - windowMapping = premergeForValues(values, results); + // If an incoming element introduces a new window, attempt to merge it into an existing + // window eagerly. Otherwise track which state address windows are used to store the state + // for each merged, active window. + collectAndMergeWindows(values, results); } - // Process the elements + // Process each element, using the updated activeWindows determined by collectAndMergeWindows. for (WindowedValue value : values) { - processElement(windowMapping, results, value); + processElement(results, value); } // Trigger output from any window that was triggered by merging or processing elements. @@ -160,6 +260,20 @@ public void processElements(Iterable> values) { handleTriggerResult( contextFactory.base(result.getKey()), false/*isEndOfWindow*/, result.getValue()); } + + // We're all done with merging and emitting elements so can compress the activeWindow state. + activeWindows.removeEphemeralWindows(); + } + + public void persist() { + activeWindows.persist(); + } + + /** Is {@code window} expired w.r.t. the garbage collection watermark? */ + private boolean canDropDueToExpiredWindow(W window) { + Instant inputWM = timerInternals.currentInputWatermarkTime(); + return inputWM != null + && window.maxTimestamp().plus(windowingStrategy.getAllowedLateness()).isBefore(inputWM); } /** @@ -168,34 +282,43 @@ public void processElements(Iterable> values) { * @param results an output parameter that accumulates all of the windows that have had the * trigger return FIRE or FIRE_AND_FINISH. Once present in this map, it is no longer * necessary to evaluate triggers for the given window. - * @return A function which maps the initial windows of the values to the intermediate windows - * they should be processed in. */ - private Function premergeForValues( + private void collectAndMergeWindows( Iterable> values, final Map results) { - // Add the windows from the values to the active window set, and keep track of which ones - // were not previously in the active window set. - Set newWindows = addToActiveWindows(values); + Set currentlyActiveWindows = Sets.newHashSet(activeWindows.getActiveWindows()); - // Merge all of the active windows and retain a mapping from source windows to result windows. - final Map sourceWindowsToResultWindows = mergeActiveWindows(results); + // Collect the windows from all elements (except those which are too late) and + // make sure they are already in the active window set or are added as NEW windows. + for (WindowedValue value : values) { + for (BoundedWindow untypedWindow : value.getWindows()) { + @SuppressWarnings("unchecked") + W window = (W) untypedWindow; - // For any new windows that survived merging, make sure we've scheduled cleanup - for (W window : newWindows) { - if (activeWindows.contains(window)) { - scheduleEndOfWindowOrGarbageCollectionTimer(contextFactory.base(window)); + if (canDropDueToExpiredWindow(window)) { + // This element is too late to contribute to this window. + // We will update the counter for this in the corresponding processElement call. + continue; + } + + ReduceFn.Context context = contextFactory.base(window); + if (triggerRunner.isClosed(context.state())) { + // This window has already been closed. + // We will update the counter for this in the corresponding processElement call. + continue; + } + // Add this window as NEW if we've not yet seen it. + activeWindows.addNew(window); } } - // Update our window mapping function. - return new Function() { - @Override - public W apply(W input) { - W result = sourceWindowsToResultWindows.get(input); - // If null, the initial window wasn't subject to any merging. - return result == null ? input : result; - } - }; + // Merge all of the active windows and retain a mapping from source windows to result windows. + mergeActiveWindows(results); + + // Make sure we've scheduled timers for any ACTIVE windows we just introduced. + // (Timers for ACTIVE windows which are now MERGED will have been discarded above.) + for (W window : Sets.difference(activeWindows.getActiveWindows(), currentlyActiveWindows)) { + scheduleEndOfWindowOrGarbageCollectionTimer(contextFactory.base(window)); + } } /** @@ -204,41 +327,62 @@ public W apply(W input) { * @param results an output parameter that accumulates all of the windows that have had the * trigger return FIRE or FIRE_AND_FINISH. Once present in this map, it is no longer * necessary to evaluate triggers for the given window. - * @return A map from initial windows of the values to the intermediate windows they should be - * processed in. The domain will be the windows that were merged into intermediate windows - * and the range is the intermediate windows that exist in the active window set. */ - private Map mergeActiveWindows(final Map results) { - final Map sourceWindowsToResultWindows = - Maps.newHashMapWithExpectedSize(activeWindows.size()); - + private void mergeActiveWindows(final Map results) { try { activeWindows.merge(new MergeCallback() { @Override - public void onMerge(Collection mergedWindows, W resultWindow, boolean isResultNew) + public void onMerge(Collection toBeMerged, Collection activeToBeMerged, W mergeResult) throws Exception { - // We only need to call onMerge with windows that were previously persisted. - Collection originalWindows = activeWindows.originalWindows(mergedWindows); - if (!originalWindows.isEmpty()) { - TriggerResult result = - ReduceFnRunner.this.onMerge(originalWindows, resultWindow, isResultNew); - if (result.isFire()) { - results.put(resultWindow, result); - } - } else { - // If there were no windows, then merging didn't rearrange the cleanup timers. Make - // sure that we have one properly scheduled - scheduleEndOfWindowOrGarbageCollectionTimer(contextFactory.base(resultWindow)); + // At this point activeWindows has already incorporated the results of the merge. + ReduceFn.OnMergeContext mergeResultContext = + contextFactory.forMerge(toBeMerged, mergeResult); + + // Prefetch various state. + triggerRunner.prefetchForMerge(mergeResultContext.state()); + + // Run the reduceFn to perform any needed merging. + try { + reduceFn.onMerge(mergeResultContext); + } catch (Exception e) { + throw wrapMaybeUserException(e); } - for (W mergedWindow : mergedWindows) { - sourceWindowsToResultWindows.put(mergedWindow, resultWindow); + // Merge the watermark holds if the output time function is not just MIN. + // Otherwise, leave all the merging window watermark holds where they are. + watermarkHold.onMerge(mergeResultContext); + + // Have the trigger merge state as needed, and handle the result. + TriggerResult result; + try { + result = triggerRunner.onMerge(mergeResultContext); + } catch (Exception e) { + Throwables.propagateIfPossible(e); + throw new RuntimeException("Failed to merge the triggers", e); + } - // If the window wasn't in the persisted original set, then we scheduled cleanup above - // but didn't pass it to merge to have the cleanup canceled. Do so here - if (!originalWindows.contains(mergedWindow)) { - cancelEndOfWindowAndGarbageCollectionTimers(contextFactory.base(mergedWindow)); + if (result.isFire()) { + results.put(mergeResult, result); + } + + for (W active : activeToBeMerged) { + if (active.equals(mergeResult)) { + // Not merged away. + continue; } + WindowTracing.debug("ReduceFnRunner.mergeActiveWindows/onMerge: Merging {} into {}", + active, mergeResult); + // Currently ACTIVE window is about to become MERGED. + ReduceFn.Context clearContext = contextFactory.base(active); + // We are going to take care of any cleanup now, so cancel timers. + cancelEndOfWindowAndGarbageCollectionTimers(clearContext); + // All the trigger state has been merged. Clear any tombstones. + triggerRunner.clearEverything(clearContext); + // We no longer care about any previous panes of merged away windows. The + // merge result window gets to start fresh if it is new. + paneInfoTracker.clear(clearContext.state()); + // Any reduceFn state, watermark holds and non-empty pane state have either been + // merged away or will be lazily merged when the next pane fires. } } }); @@ -246,78 +390,36 @@ public void onMerge(Collection mergedWindows, W resultWindow, boolean isResul Throwables.propagateIfPossible(e); throw new RuntimeException("Exception while merging windows", e); } - return sourceWindowsToResultWindows; - } - - /** Is the {@code window} expired w.r.t. the garbage collection watermark? */ - private Predicate canDropDueToExpiredWindow = new Predicate() { - @Override - public boolean apply(W window) { - Instant inputWM = timerInternals.currentInputWatermarkTime(); - return inputWM != null - && window.maxTimestamp().plus(windowingStrategy.getAllowedLateness()).isBefore(inputWM); - } - }; - - /** - * Add the initial windows from each of the values to the active window set. Returns the set of - * new windows. - */ - private Set addToActiveWindows(Iterable> values) { - Set newWindows = new HashSet<>(); - for (WindowedValue value : values) { - - for (BoundedWindow untypedWindow : value.getWindows()) { - @SuppressWarnings("unchecked") - W window = (W) untypedWindow; - - if (canDropDueToExpiredWindow.apply(window)) { - // This value will be dropped (and reported in a counter) by processElement. - // Hence it won't contribute to any new window. - continue; - } - - ReduceFn.Context context = contextFactory.base(window); - if (!triggerRunner.isClosed(context.state())) { - if (activeWindows.add(window)) { - newWindows.add(window); - } - } - } - } - return newWindows; } /** - * @param windowMapping a function which maps windows associated with the value to the window that - * it was merged into, and in which we should actually process the element * @param results a record of all of the windows that have had the trigger return FIRE or * FIRE_AND_FINISH. Once present in this map, it is no longer necessary to evaluate triggers * for the given result. * @param value the value being processed */ - private void processElement( - Function windowMapping, Map results, WindowedValue value) { - - // Only consider representative windows from among all windows in equivalence classes - // induced by window merging. + private void processElement(Map results, WindowedValue value) { + // Redirect element windows to the ACTIVE windows they have been merged into. + // It is possible two of the element's windows have been merged into the same window. + // In that case we'll process the same element for the same window twice. @SuppressWarnings("unchecked") - FluentIterable mappedWindows = - FluentIterable.from((Collection) value.getWindows()) - .transform(windowMapping); - - // Some windows may be expired - Iterable windows = mappedWindows.filter(Predicates.not(canDropDueToExpiredWindow)); - - // Count the number of elements that are dropped - for (W expiredWindow : mappedWindows.filter(canDropDueToExpiredWindow)) { + Collection windows = new ArrayList<>(); + for (BoundedWindow untypedWindow : value.getWindows()) { + @SuppressWarnings("unchecked") + W window = (W) untypedWindow; + if (canDropDueToExpiredWindow(window)) { + // The element is too late for this window. droppedDueToLateness.addValue(1L); WindowTracing.debug( - "processElement: Dropping element at {} for key:{} and window:{} since window is " - + "too far behind inputWatermark:{}; outputWatermark:{}", - value.getTimestamp(), key, expiredWindow, - timerInternals.currentInputWatermarkTime(), - timerInternals.currentOutputWatermarkTime()); + "ReduceFnRunner.processElement: Dropping element at {} for key:{}; window:{} " + + "since too far behind inputWatermark:{}; outputWatermark:{}", + value.getTimestamp(), key, window, timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + } else { + W active = activeWindows.representative(window); + Preconditions.checkState(active != null, "Window %s should have been added", window); + windows.add(active); + } } // Prefetch in each of the windows if we're going to need to process triggers @@ -329,15 +431,20 @@ private void processElement( } } - // And process each of the windows + // Process the element for each (representative) window it belongs to. for (W window : windows) { ReduceFn.ProcessValueContext context = contextFactory.forValue(window, value.getValue(), value.getTimestamp()); // Check to see if the triggerRunner thinks the window is closed. If so, drop that window. if (!results.containsKey(window) && triggerRunner.isClosed(context.state())) { - droppedDueToClosedWindow.addValue(1L); - continue; + droppedDueToClosedWindow.addValue(1L); + WindowTracing.debug( + "ReduceFnRunner.processElement: Dropping element at {} for key:{}; window:{} " + + "since window is no longer active at inputWatermark:{}; outputWatermark:{}", + value.getTimestamp(), key, window, timerInternals.currentInputWatermarkTime(), + timerInternals.currentOutputWatermarkTime()); + continue; } nonEmptyPanes.recordContent(context); @@ -376,67 +483,6 @@ private void processElement( } } - /** - * Make sure that all the state built up in this runner has been persisted. - */ - public void persist() { - activeWindows.persist(); - } - - /** - * Called when windows merge. - */ - public TriggerResult onMerge( - Collection mergedWindows, W resultWindow, boolean isResultWindowNew) { - ReduceFn.OnMergeContext resultContext = - contextFactory.forMerge(mergedWindows, resultWindow); - - // Schedule state reads for trigger execution. - triggerRunner.prefetchForMerge(resultContext.state()); - - // Run the reduceFn to perform any needed merging. - try { - reduceFn.onMerge(resultContext); - } catch (Exception e) { - throw wrapMaybeUserException(e); - } - - // Merge the watermark hold - watermarkHold.mergeHolds(resultContext); - - // Have the trigger merge state as needed, and handle the result. - TriggerResult triggerResult; - try { - triggerResult = triggerRunner.onMerge(resultContext); - } catch (Exception e) { - Throwables.propagateIfPossible(e); - throw new RuntimeException("Failed to merge the triggers", e); - } - - // Cleanup the trigger state in the old windows. - for (W mergedWindow : mergedWindows) { - if (!mergedWindow.equals(resultWindow)) { - try { - ReduceFn.Context mergedContext = contextFactory.base(mergedWindow); - cancelEndOfWindowAndGarbageCollectionTimers(mergedContext); - triggerRunner.clearEverything(mergedContext); - paneInfo.clear(mergedContext.state()); - } catch (Exception e) { - Throwables.propagateIfPossible(e); - throw new RuntimeException("Exception while clearing trigger state", e); - } - } - } - - // Schedule cleanup if the window is new. Do this after cleaning up the old state in case one - // of them had a timer at the same point. - if (isResultWindowNew) { - scheduleEndOfWindowOrGarbageCollectionTimer(resultContext); - } - - return triggerResult; - } - /** * Called when an end-of-window, garbage collection, or trigger-specific timer fires. */ @@ -447,10 +493,11 @@ public void onTimer(TimerData timer) { @SuppressWarnings("unchecked") WindowNamespace windowNamespace = (WindowNamespace) timer.getNamespace(); W window = windowNamespace.getWindow(); - // If the window is subject to merging then all timers should have been cleared upon merge. - Preconditions.checkState( - !windowingStrategy.getWindowFn().isNonMerging() || activeWindows.contains(window), - "Received timer %s for inactive window %s", timer, window); + + if (!activeWindows.isActive(window)) { + WindowTracing.debug( + "ReduceFnRunner.onTimer: Note that timer {} is for non-ACTIVE window {}", timer, window); + } ReduceFn.Context context = contextFactory.base(window); @@ -469,19 +516,19 @@ public void onTimer(TimerData timer) { if (isGarbageCollection) { WindowTracing.debug( - "onTimer: Cleaning up for key:{}; window:{} at {} with " + "ReduceFnRunner.onTimer: Cleaning up for key:{}; window:{} at {} with " + "inputWatermark:{}; outputWatermark:{}", key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(), timerInternals.currentOutputWatermarkTime()); - if (activeWindows.contains(window) && !triggerRunner.isClosed(context.state())) { + if (activeWindows.isActive(window) && !triggerRunner.isClosed(context.state())) { // We need to call onTrigger to emit the final pane if required. // The final pane *may* be ON_TIME if: // - AllowedLateness = 0 (ie the timer is at end-of-window), and; // - The trigger fires on the end-of-window timer. boolean isWatermarkTrigger = isEndOfWindowTimer && runTriggersForTimer(context, timer).isFire(); - onTrigger(context, isWatermarkTrigger, true/*isFinish*/); + onTrigger(context, isWatermarkTrigger, true/*isFinish*/, false/*willStillBeActive*/); } // Clear all the state for this window since we'll never see elements for it again. @@ -494,26 +541,26 @@ public void onTimer(TimerData timer) { } } else { WindowTracing.debug( - "onTimer: Triggering for key:{}; window:{} at {} with " + "ReduceFnRunner.onTimer: Triggering for key:{}; window:{} at {} with " + "inputWatermark:{}; outputWatermark:{}", key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(), timerInternals.currentOutputWatermarkTime()); - boolean isFinish = false; - if (activeWindows.contains(window) && !triggerRunner.isClosed(context.state())) { + if (activeWindows.isActive(window) && !triggerRunner.isClosed(context.state())) { TriggerResult result = runTriggersForTimer(context, timer); handleTriggerResult(context, isEndOfWindowTimer, result); - isFinish = result.isFinish(); } - if (isEndOfWindowTimer && !isFinish) { + if (isEndOfWindowTimer) { // Since we are processing an on-time firing we should schedule the garbage collection // timer. (If getAllowedLateness is zero then the timer event will be considered a // cleanup event and handled by the above). + // Note we must do this even if the trigger is finished so that we are sure to cleanup + // any final trigger tombstones. Preconditions.checkState( windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO), "Unexpected zero getAllowedLateness"); WindowTracing.debug( - "onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with " + "ReduceFnRunner.onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with " + "inputWatermark:{}; outputWatermark:{}", key, context.window(), cleanupTime, timerInternals.currentInputWatermarkTime(), timerInternals.currentOutputWatermarkTime()); @@ -545,15 +592,20 @@ private TriggerResult runTriggersForTimer( * */ private void clearAllState(ReduceFn.Context context) throws Exception { - nonEmptyPanes.clearPane(context); - try { - reduceFn.clearState(context); - } catch (Exception e) { - throw wrapMaybeUserException(e); + boolean isActive = activeWindows.isActive(context.window()); + watermarkHold.clearHolds(context, isActive); + if (isActive) { + // The trigger never finished, so make sure we clear any remaining state. + try { + reduceFn.clearState(context); + } catch (Exception e) { + throw wrapMaybeUserException(e); + } + nonEmptyPanes.clearPane(context); + activeWindows.remove(context.window()); } triggerRunner.clearEverything(context); - paneInfo.clear(context.state()); - watermarkHold.clear(context); + paneInfoTracker.clear(context.state()); } /** Should the reduce function state be cleared? */ @@ -582,37 +634,51 @@ private void handleTriggerResult(ReduceFn.Context context // If the trigger fired due to an end-of-window timer, treat it as an AfterWatermark trigger. boolean isWatermarkTrigger = isEndOfWindow; + // Will be able to clear all element state after triggering? + boolean shouldDiscard = shouldDiscardAfterFiring(result); + // Run onTrigger to produce the actual pane contents. // As a side effect it will clear all element holds, but not necessarily any // end-of-window or garbage collection holds. - onTrigger(context, isWatermarkTrigger, result.isFinish()); + onTrigger(context, isWatermarkTrigger, result.isFinish(), !shouldDiscard); // Now that we've triggered, the pane is empty. nonEmptyPanes.clearPane(context); // Cleanup buffered data if appropriate - if (shouldDiscardAfterFiring(result)) { - // Clear the reduceFn state + if (shouldDiscard) { + // Clear the reduceFn state across all windows in the equivalence class for the current + // window. try { reduceFn.clearState(context); } catch (Exception e) { throw wrapMaybeUserException(e); } - // Remove the window from active set -- nothing is buffered. + // Remove the window from active set. + // This will forget the equivalence class for this window. + WindowTracing.debug("ReduceFnRunner.handleTriggerResult: removing {}", context.window()); activeWindows.remove(context.window()); + + if (!result.isFinish()) { + // We still need to consider this window active since we may have had to add an + // end-of-window or garbage collection hold above. + activeWindows.addActive(context.window()); + } } if (result.isFinish()) { - // If we're finishing, clear up the trigger tree as well. - // However, we'll leave behind a tombstone so we know the trigger is finished. + // If we're finishing, eagerly clear state to reduce pressure on the backend. + // Leave behind a tombstone in the trigger runner so we know the trigger is finished. try { triggerRunner.clearState(context); - paneInfo.clear(context.state()); + paneInfoTracker.clear(context.state()); } catch (Exception e) { Throwables.propagateIfPossible(e); throw new RuntimeException("Exception while clearing trigger state", e); } + // No more watermark holds will be placed (even for end-of-window or garbage + // collection holds). } } @@ -644,12 +710,12 @@ private boolean needToEmit( * @param isFinish true if this will be the last triggering processed */ private void onTrigger(final ReduceFn.Context context, - boolean isWatermarkTrigger, boolean isFinish) { + boolean isWatermarkTrigger, boolean isFinish, boolean willStillBeActive) { // Collect state. StateContents outputTimestampFuture = - watermarkHold.extractAndRelease(context, isFinish); + watermarkHold.extractAndRelease(context, isFinish, willStillBeActive); StateContents paneFuture = - paneInfo.getNextPaneInfo(context, isWatermarkTrigger, isFinish); + paneInfoTracker.getNextPaneInfo(context, isWatermarkTrigger, isFinish); StateContents isEmptyFuture = nonEmptyPanes.isEmpty(context); reduceFn.prefetchOnTrigger(context.state()); @@ -669,7 +735,7 @@ private void onTrigger(final ReduceFn.Context context, public void output(OutputT toOutput) { // We're going to output panes, so commit the (now used) PaneInfo. // TODO: Unnecessary if isFinal? - paneInfo.storeCurrentPaneInfo(context, pane); + paneInfoTracker.storeCurrentPaneInfo(context, pane); // Output the actual value. windowingInternals.outputWindowedValue( @@ -696,7 +762,7 @@ private void scheduleEndOfWindowOrGarbageCollectionTimer(ReduceFn.Co "Asking to set a timer at %s behind input watermark %s", fireTime, inputWM); } WindowTracing.trace( - "scheduleTimer: Scheduling {} timer at {} for " + "ReduceFnRunner.scheduleEndOfWindowOrGarbageCollectionTimer: Scheduling {} timer at {} for " + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}", which, fireTime, key, context.window(), inputWM, timerInternals.currentOutputWatermarkTime()); @@ -705,7 +771,7 @@ private void scheduleEndOfWindowOrGarbageCollectionTimer(ReduceFn.Co private void cancelEndOfWindowAndGarbageCollectionTimers(ReduceFn.Context context) { WindowTracing.debug( - "cancelTimer: Deleting timers for " + "ReduceFnRunner.cancelEndOfWindowAndGarbageCollectionTimers: Deleting timers for " + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}", key, context.window(), timerInternals.currentInputWatermarkTime(), timerInternals.currentOutputWatermarkTime()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SystemReduceFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SystemReduceFn.java index ab55d3f0b306c..d7d77f9a330b9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SystemReduceFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/SystemReduceFn.java @@ -98,7 +98,7 @@ public SystemReduceFn(StateTag> buffer @Override public void processValue(ProcessValueContext c) throws Exception { - c.state().access(bufferTag).add(c.value()); + c.state().accessAcrossMergedWindows(bufferTag).add(c.value()); } @Override @@ -125,6 +125,8 @@ public void clearState(Context c) throws Exception { @Override public StateContents isEmpty(StateContext state) { + // Since we only check for empty element sets when a trigger fires it's unreasonable + // to require a prefetch. return state.accessAcrossMergedWindows(bufferTag).isEmpty(); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerContextFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerContextFactory.java index 64c6de39b36eb..4dfef33e1de16 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerContextFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerContextFactory.java @@ -252,7 +252,7 @@ public TriggerStateContextImpl(ActiveWindowSet activeWindows, // Annoyingly, since we hadn't set the triggerIndex yet (we can't do it before super) // This will would otherwise have incorporated 0 as the trigger index. - this.namespace = namespaceFor(window); + this.windowNamespace = namespaceFor(window); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerRunner.java index 34116057dc333..7397189ae94f3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/TriggerRunner.java @@ -45,8 +45,8 @@ * @param The kind of windows being processed. */ public class TriggerRunner { - - @VisibleForTesting static final StateTag> FINISHED_BITS_TAG = + @VisibleForTesting + static final StateTag> FINISHED_BITS_TAG = StateTags.makeSystemTagInternal(StateTags.value("closed", BitSetCoder.of())); private final ExecutableTrigger rootTrigger; @@ -75,6 +75,13 @@ public boolean isClosed(ReduceFn.StateContext state) { return readFinishedBits(state.access(FINISHED_BITS_TAG)).get(0); } + public void prefetchForValue(ReduceFn.StateContext state) { + if (isFinishedSetNeeded()) { + state.access(FINISHED_BITS_TAG).get(); + } + rootTrigger.getSpec().prefetchOnElement(state); + } + /** * Run the trigger logic to deal with a new value. */ @@ -87,6 +94,16 @@ public TriggerResult processValue(ReduceFn.ProcessValueContext c) th return result; } + public void prefetchForMerge(ReduceFn.MergingStateContext state) { + if (isFinishedSetNeeded()) { + for (ValueState value : + state.mergingAccessInEachMergingWindow(FINISHED_BITS_TAG).values()) { + value.get(); + } + } + rootTrigger.getSpec().prefetchOnMerge(state); + } + /** * Run the trigger merging logic as part of executing the specified merge. */ @@ -97,11 +114,10 @@ public TriggerResult onMerge(ReduceFn.OnMergeContext c) throws Excep // And read the finished bits in each merging window. ImmutableMap.Builder mergingFinishedSets = ImmutableMap.builder(); Map> mergingFinishedSetState = - c.state().accessInEachMergingWindow(FINISHED_BITS_TAG); + c.state().mergingAccessInEachMergingWindow(FINISHED_BITS_TAG); for (W window : c.mergingWindows()) { // Don't need to clone these, since the trigger context doesn't allow modification - mergingFinishedSets.put(window, - readFinishedBits(mergingFinishedSetState.get(window))); + mergingFinishedSets.put(window, readFinishedBits(mergingFinishedSetState.get(window))); } Trigger.OnMergeContext mergeContext = @@ -117,6 +133,13 @@ public TriggerResult onMerge(ReduceFn.OnMergeContext c) throws Excep return result.getTriggerResult(); } + public void prefetchForTimer(ReduceFn.StateContext state) { + if (isFinishedSetNeeded()) { + state.access(FINISHED_BITS_TAG).get(); + } + rootTrigger.getSpec().prefetchOnElement(state); + } + /** * Run the trigger logic appropriate for receiving a timer with the specified destination ID. */ @@ -166,29 +189,6 @@ public void clearEverything(ReduceFn.Context c) throws Exception { } } - public void prefetchForValue(ReduceFn.StateContext state) { - if (isFinishedSetNeeded()) { - state.access(FINISHED_BITS_TAG).get(); - } - rootTrigger.getSpec().prefetchOnElement(state); - } - - public void prefetchForMerge(ReduceFn.MergingStateContext state) { - if (isFinishedSetNeeded()) { - for (ValueState value : state.accessInEachMergingWindow(FINISHED_BITS_TAG).values()) { - value.get(); - } - } - rootTrigger.getSpec().prefetchOnMerge(state); - } - - public void prefetchForTimer(ReduceFn.StateContext state) { - if (isFinishedSetNeeded()) { - state.access(FINISHED_BITS_TAG).get(); - } - rootTrigger.getSpec().prefetchOnElement(state); - } - private boolean isFinishedSetNeeded() { // TODO: If we know that no trigger in the tree will ever finish, we don't need to do the // lookup. Right now, we special case this for the DefaultTrigger. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java index bac3af11f7b58..0c6413eacc7d4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/WatermarkHold.java @@ -178,7 +178,7 @@ public WatermarkHold(TimerInternals timerInternals, WindowingStrategy wind */ public void addHolds(ReduceFn.ProcessValueContext context) { if (!addElementHold(context)) { - addEndOfWindowOrGarbageCollectionHolds(context); + addEndOfWindowOrGarbageCollectionHolds(context, true); } } @@ -220,7 +220,7 @@ private boolean addElementHold(ReduceFn.ProcessValueContext context) tooLate = true; } else { tooLate = false; - context.state().access(elementHoldTag).add(elementHold); + context.state().accessAcrossMergedWindows(elementHoldTag).add(elementHold); } WindowTracing.trace( "WatermarkHold.addHolds: element hold at {} is {} for " @@ -237,9 +237,10 @@ private boolean addElementHold(ReduceFn.ProcessValueContext context) *

The end-of-window hold guarantees that an empty {@code ON_TIME} pane can be given * a timestamp which will not be considered beyond allowed lateness by any downstream computation. */ - private void addEndOfWindowOrGarbageCollectionHolds(ReduceFn.Context context) { - if (!addEndOfWindowHold(context)) { - addGarbageCollectionHold(context); + private void addEndOfWindowOrGarbageCollectionHolds( + ReduceFn.Context context, boolean isActive) { + if (!addEndOfWindowHold(context, isActive)) { + addGarbageCollectionHold(context, isActive); } } @@ -249,7 +250,7 @@ private void addEndOfWindowOrGarbageCollectionHolds(ReduceFn.Context *

The end-of-window hold guarantees that any empty {@code ON_TIME} pane can be given * a timestamp which will not be considered beyond allowed lateness by any downstream computation. */ - private boolean addEndOfWindowHold(ReduceFn.Context context) { + private boolean addEndOfWindowHold(ReduceFn.Context context, boolean isActive) { // Only add an end-of-window hold if we can be sure the end-of-window timer // has not yet fired. Otherwise we risk holding up the output watermark until // the garbage collection timer fires, which may be a very long time in the future. @@ -263,13 +264,21 @@ private boolean addEndOfWindowHold(ReduceFn.Context context) { tooLate = false; Preconditions.checkState(outputWM == null || !eowHold.isBefore(outputWM), "End-of-window hold %s cannot be before output watermark %s", eowHold, outputWM); - context.state().access(EXTRA_HOLD_TAG).add(eowHold); + if (isActive) { + context.state().accessAcrossMergedWindows(EXTRA_HOLD_TAG).add(eowHold); + } else { + // The window is not currently ACTIVE, so we can't use accessAcrossMergedWindows + // to collect its state. Instead, store the holds under the window itself. The + // caller will be responsible for ensuring the active window set now considers this + // window ACTIVE. + context.state().access(EXTRA_HOLD_TAG).add(eowHold); + } } WindowTracing.trace( - "WatermarkHold.addEndOfWindowHold: end-of-window hold at {} is {} for " + "WatermarkHold.addEndOfWindowHold: end-of-window hold for %s at {} is {} for " + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}", - eowHold, tooLate ? "too late" : "on-time", context.key(), context.window(), inputWM, - outputWM); + isActive ? "active" : "inactive", eowHold, tooLate ? "too late" : "on-time", context.key(), + context.window(), inputWM, outputWM); return !tooLate; } @@ -281,7 +290,7 @@ private boolean addEndOfWindowHold(ReduceFn.Context context) { * computation. If we are sure no empty final panes can be emitted then there's no need * for an additional hold. */ - private void addGarbageCollectionHold(ReduceFn.Context context) { + private void addGarbageCollectionHold(ReduceFn.Context context, boolean isActive) { // Only add a garbage collection hold if we are sure we need an empty final pane and // the window will be garbage collected after the end-of-window trigger. if (context.windowingStrategy().getClosingBehavior() == ClosingBehavior.FIRE_ALWAYS @@ -290,21 +299,30 @@ private void addGarbageCollectionHold(ReduceFn.Context context) { Instant outputWM = timerInternals.currentOutputWatermarkTime(); Instant inputWM = timerInternals.currentInputWatermarkTime(); WindowTracing.trace( - "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} for " + "WatermarkHold.addGarbageCollectionHold: garbage collection hold for %s at {} for " + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}", - gcHold, context.key(), context.window(), inputWM, outputWM); + isActive ? "active" : "inactive", gcHold, context.key(), context.window(), inputWM, + outputWM); Preconditions.checkState(inputWM == null || !gcHold.isBefore(inputWM), "Garbage collection hold %s cannot be before input watermark %s", gcHold, inputWM); - context.state().access(EXTRA_HOLD_TAG).add(gcHold); + if (isActive) { + context.state().accessAcrossMergedWindows(EXTRA_HOLD_TAG).add(gcHold); + } else { + // See comment above for addEndOfWindowHold. + context.state().access(EXTRA_HOLD_TAG).add(gcHold); + } } } /** - * Updates the watermark hold when windows merge. For example, if the new window implies - * a later watermark hold, then earlier holds may be released. + * Updates the watermark hold when windows merge if it is possible the merged value does + * not equal all of the existing holds. For example, if the new window implies a later + * watermark hold, then earlier holds may be released. + * + *

Note that state may be left behind in merged windows. */ - public void mergeHolds(final ReduceFn.OnMergeContext context) { - WindowTracing.debug("mergeHolds: for key:{}; window:{}; inputWatermark:{}; outputWatermark:{}", + public void onMerge(final ReduceFn.OnMergeContext context) { + WindowTracing.debug("onMerge: for key:{}; window:{}; inputWatermark:{}; outputWatermark:{}", context.key(), context.window(), timerInternals.currentInputWatermarkTime(), timerInternals.currentOutputWatermarkTime()); // If the output hold depends only on the window, then there may not be a hold in place @@ -312,7 +330,7 @@ public void mergeHolds(final ReduceFn.OnMergeContext context) { if (windowingStrategy.getOutputTimeFn().dependsOnlyOnWindow()) { Instant arbitraryTimestamp = new Instant(0); context.state() - .access(elementHoldTag) + .accessAcrossMergedWindows(elementHoldTag) .add(windowingStrategy.getOutputTimeFn().assignOutputTime( arbitraryTimestamp, context.window())); } @@ -322,21 +340,28 @@ public void mergeHolds(final ReduceFn.OnMergeContext context) { } /** - * Return (a future for) the earliest data hold for {@code context}. Clear the data hold after - * reading. If {@code isFinal}, also clear any end-of-window or garbage collection hold. + * Return (a future for) the earliest hold for {@code context}. Clear all the holds after + * reading, but add/restore an end-of-window or garbage collection hold if required. * *

The returned timestamp is the output timestamp according to the {@link OutputTimeFn} * from the windowing strategy of this {@link WatermarkHold}, combined across all the non-late - * elements in the current pane. + * elements in the current pane. If there is no such value the timestamp is the end + * of the window. + * + *

If {@code willStillBeActive} then any end-of-window or garbage collection holds will + * be reestablished in one of the target windows alread in use for this window. Otherwise, + * the holds will be placed in this window itself. */ - public StateContents extractAndRelease( - final ReduceFn.Context context, final boolean isFinal) { + public StateContents extractAndRelease(final ReduceFn.Context context, + final boolean isFinal, final boolean willStillBeActive) { WindowTracing.debug( "extractAndRelease: for key:{}; window:{}; inputWatermark:{}; outputWatermark:{}", context.key(), context.window(), timerInternals.currentInputWatermarkTime(), timerInternals.currentOutputWatermarkTime()); final WatermarkStateInternal elementHoldState = context.state().accessAcrossMergedWindows(elementHoldTag); + // Since we only extract holds when a trigger fires it is unreasonable to expect + // the state to be prefetched. final StateContents elementHoldFuture = elementHoldState.get(); final WatermarkStateInternal extraHoldState = context.state().accessAcrossMergedWindows(EXTRA_HOLD_TAG); @@ -374,9 +399,8 @@ public Instant read() { elementHoldState.clear(); extraHoldState.clear(); - // Reinstate the end-of-window and garbage collection holds if still required. if (!isFinal) { - addEndOfWindowOrGarbageCollectionHolds(context); + addEndOfWindowOrGarbageCollectionHolds(context, willStillBeActive); } return hold; @@ -384,13 +408,22 @@ public Instant read() { }; } - /** Clear any remaining holds. */ - public void clear(ReduceFn.Context context) { + /** + * Clear any remaining holds. If {@code isActive} then we assume holds could be placed in any + * of the target windows for this window. Otherwise we assume only this window has any + * end-of-window or garbage collection holds. + */ + public void clearHolds(ReduceFn.Context context, boolean isActive) { WindowTracing.debug( - "WatermarkHold.clear: for key:{}; window:{}; inputWatermark:{}; outputWatermark:{}", - context.key(), context.window(), timerInternals.currentInputWatermarkTime(), - timerInternals.currentOutputWatermarkTime()); - context.state().accessAcrossMergedWindows(elementHoldTag).clear(); - context.state().accessAcrossMergedWindows(EXTRA_HOLD_TAG).clear(); + "WatermarkHold.clearHolds: For key:{}; %s window:{}; " + + "inputWatermark:{}; outputWatermark:{}", + context.key(), isActive ? "active" : "inactive", context.window(), + timerInternals.currentInputWatermarkTime(), timerInternals.currentOutputWatermarkTime()); + if (isActive) { + context.state().accessAcrossMergedWindows(elementHoldTag).clear(); + context.state().accessAcrossMergedWindows(EXTRA_HOLD_TAG).clear(); + } else { + context.state().access(EXTRA_HOLD_TAG).clear(); + } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergingStateInternals.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergingStateInternals.java index 11a73ad7d2ea9..f47c9ddbcf5d2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergingStateInternals.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/state/MergingStateInternals.java @@ -50,14 +50,10 @@ public ValueState bindValue(StateTag> address, Coder cod public BagState bindBag(StateTag> address, Coder elemCoder) { List> sources = new ArrayList<>(); for (StateNamespace sourceNamespace : sourceNamespaces) { - // Skip adding the result namespace for now. - if (!sourceNamespace.equals(resultNamespace)) { - sources.add(state(sourceNamespace, address)); - } + sources.add(state(sourceNamespace, address)); } BagState results = state(resultNamespace, address); - sources.add(results); return new MergedBag<>(sources, results); } @@ -68,14 +64,10 @@ CombiningValueStateInternal bindCombiningValue( Coder accumCoder, CombineFn combineFn) { List> sources = new ArrayList<>(); for (StateNamespace sourceNamespace : sourceNamespaces) { - // Skip adding the result namespace for now. - if (!sourceNamespace.equals(resultNamespace)) { - sources.add(state(sourceNamespace, address)); - } + sources.add(state(sourceNamespace, address)); } CombiningValueStateInternal result = state(resultNamespace, address); - sources.add(result); return new MergedCombiningValue<>(sources, result, combineFn); } @@ -85,13 +77,9 @@ public WatermarkStateInternal bindWatermark( OutputTimeFn outputTimeFn) { List sources = new ArrayList<>(); for (StateNamespace sourceNamespace : sourceNamespaces) { - // Skip adding the result namespace for now. - if (!sourceNamespace.equals(resultNamespace)) { - sources.add(state(sourceNamespace, address)); - } + sources.add(state(sourceNamespace, address)); } WatermarkStateInternal result = state(resultNamespace, address); - sources.add(result); // It is the responsibility of the SDK to only pass allowed result windows. @SuppressWarnings("unchecked") diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java index db041290b9e37..05aba6008fed5 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java @@ -123,7 +123,7 @@ public void testAfterProcessingTimeWithMergingWindow() throws Exception { TimestampedValue.of(2, new Instant(2))); // in [2, 12), timer for 16 tester.advanceProcessingTime(new Instant(16)); - // This fires, because the earliest element in [1, 12) arrived at time 10 + // This fires, because the earliest element in [1, 11) arrived at time 10 assertThat(tester.extractOutput(), Matchers.contains( WindowMatchers.isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 1, 12))); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/MergingActiveWindowSetTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/MergingActiveWindowSetTest.java new file mode 100644 index 0000000000000..0b1a7521f40b0 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/MergingActiveWindowSetTest.java @@ -0,0 +1,175 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.util; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; +import com.google.cloud.dataflow.sdk.util.state.InMemoryStateInternals; +import com.google.cloud.dataflow.sdk.util.state.StateInternals; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Collection; + +/** + * Test NonMergingActiveWindowSet. + */ +@RunWith(JUnit4.class) +public class MergingActiveWindowSetTest { + private Sessions windowFn; + private StateInternals state; + private MergingActiveWindowSet set; + + @Before + public void before() { + windowFn = Sessions.withGapDuration(Duration.millis(10)); + state = new InMemoryStateInternals(); + set = new MergingActiveWindowSet<>(windowFn, state); + } + + @After + public void after() { + set = null; + state = null; + windowFn = null; + } + + private void add(final long instant) { + System.out.println("ADD " + instant); + final Object element = new Long(instant); + Sessions.AssignContext context = windowFn.new AssignContext() { + @Override + public Object element() { + return element; + } + + @Override + public Instant timestamp() { + return new Instant(instant); + } + + @Override + public Collection windows() { + return ImmutableList.of(); + } + }; + + for (IntervalWindow window : windowFn.assignWindows(context)) { + set.addNew(window); + } + } + + private void merge(ActiveWindowSet.MergeCallback callback) throws Exception { + System.out.println("MERGE"); + set.merge(callback); + set.checkInvariants(); + System.out.println(set); + } + + private void pruneAndPersist() { + System.out.println("PRUNE"); + set.removeEphemeralWindows(); + set.checkInvariants(); + System.out.println(set); + set.persist(); + } + + private IntervalWindow window(long start, long size) { + return new IntervalWindow(new Instant(start), new Duration(size)); + } + + @Test + public void test() throws Exception { + @SuppressWarnings("unchecked") + ActiveWindowSet.MergeCallback callback = + mock(ActiveWindowSet.MergeCallback.class); + + // NEW 1+10 + // NEW 2+10 + // NEW 15+10 + // => + // ACTIVE 1+11 (target 1+11) + // EPHEMERAL 1+10 -> 1+11 + // EPHEMERAL 2+10 -> 1+11 + // ACTIVE 15+10 (target 15+10) + add(1); + add(2); + add(15); + merge(callback); + verify(callback).onMerge(ImmutableList.of(window(1, 10), window(2, 10)), + ImmutableList.of(), window(1, 11)); + assertEquals(ImmutableSet.of(window(1, 11), window(15, 10)), set.getActiveWindows()); + assertEquals(window(1, 11), set.representative(window(1, 10))); + assertEquals(window(1, 11), set.representative(window(2, 10))); + assertEquals(window(1, 11), set.representative(window(1, 11))); + assertEquals(window(15, 10), set.representative(window(15, 10))); + assertEquals( + ImmutableSet.of(window(1, 11)), set.readStateAddresses(window(1, 11))); + assertEquals( + ImmutableSet.of(window(15, 10)), set.readStateAddresses(window(15, 10))); + + // NEW 3+10 + // => + // ACTIVE 1+12 (target 1+11) + // EPHEMERAL 3+10 -> 1+12 + // ACTIVE 15+10 (target 15+10) + add(3); + merge(callback); + verify(callback).onMerge(ImmutableList.of(window(1, 11), window(3, 10)), + ImmutableList.of(window(1, 11)), window(1, 12)); + assertEquals(ImmutableSet.of(window(1, 12), window(15, 10)), set.getActiveWindows()); + assertEquals(window(1, 12), set.representative(window(3, 10))); + + // NEW 8+10 + // => + // ACTIVE 1+24 (target 1+11, 15+10) + // MERGED 1+11 -> 1+24 + // MERGED 15+10 -> 1+24 + // EPHEMERAL 1+12 -> 1+24 + add(8); + merge(callback); + verify(callback).onMerge(ImmutableList.of(window(1, 12), window(8, 10), window(15, 10)), + ImmutableList.of(window(1, 12), window(15, 10)), window(1, 24)); + assertEquals(ImmutableSet.of(window(1, 24)), set.getActiveWindows()); + assertEquals(window(1, 24), set.representative(window(1, 12))); + assertEquals(window(1, 24), set.representative(window(1, 11))); + assertEquals(window(1, 24), set.representative(window(15, 10))); + + // NEW 9+10 + // => + // ACTIVE 1+24 (target 1+11, 15+10) + add(9); + merge(callback); + verify(callback).onMerge(ImmutableList.of(window(1, 24), window(9, 10)), + ImmutableList.of(window(1, 24)), window(1, 24)); + + pruneAndPersist(); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReduceFnTester.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReduceFnTester.java index b789d6bd33613..8327359277b44 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReduceFnTester.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReduceFnTester.java @@ -66,6 +66,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -218,7 +219,7 @@ private void assertHasOnlyGlobalAndAllowedTags( for (W expectedWindow : expectedWindows) { expectedWindowsSet.add(windowNamespace(expectedWindow)); } - Set actualWindows = new HashSet<>(); + Map>> actualWindows = new HashMap<>(); for (StateNamespace namespace : stateInternals.getNamespacesInUse()) { if (namespace instanceof StateNamespaces.GlobalNamespace) { @@ -228,7 +229,7 @@ private void assertHasOnlyGlobalAndAllowedTags( if (tagsInUse.isEmpty()) { continue; } - actualWindows.add(namespace); + actualWindows.put(namespace, tagsInUse); Set> unexpected = Sets.difference(tagsInUse, allowedTags); if (unexpected.isEmpty()) { continue; @@ -243,7 +244,8 @@ private void assertHasOnlyGlobalAndAllowedTags( } } - assertEquals(expectedWindowsSet, actualWindows); + assertEquals("Still in use: " + actualWindows.toString(), expectedWindowsSet, + actualWindows.keySet()); } private StateNamespace windowNamespace(W window) { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternalsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternalsTest.java index ee09e08c37ebd..bf9c6f7bb8dce 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternalsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/state/InMemoryStateInternalsTest.java @@ -154,7 +154,8 @@ public void testMergeBagIntoNewNamespace() throws Exception { bag1.add("!"); BagState merged = underTest.mergedState( - Arrays.asList(NAMESPACE_1, NAMESPACE_2), NAMESPACE_3, STRING_BAG_ADDR, WINDOW_3); + Arrays.asList(NAMESPACE_1, NAMESPACE_2, NAMESPACE_3), + NAMESPACE_3, STRING_BAG_ADDR, WINDOW_3); // Reading the merged bag gets both the contents assertThat(merged.get().read(), Matchers.containsInAnyOrder("Hello", "World", "!")); @@ -245,7 +246,8 @@ public void testMergeCombiningValueIntoNewNamespace() throws Exception { assertThat(value2.get().read(), Matchers.equalTo(10)); CombiningValueState merged = underTest.mergedState( - Arrays.asList(NAMESPACE_1, NAMESPACE_2), NAMESPACE_3, SUM_INTEGER_ADDR, WINDOW_3); + Arrays.asList(NAMESPACE_1, NAMESPACE_2, NAMESPACE_3), + NAMESPACE_3, SUM_INTEGER_ADDR, WINDOW_3); assertThat(value1.get().read(), Matchers.equalTo(11)); assertThat(value2.get().read(), Matchers.equalTo(10)); From c98f0438350f2fe4f00e1276605349594c60d5f2 Mon Sep 17 00:00:00 2001 From: lcwik Date: Tue, 5 Jan 2016 15:17:48 -0800 Subject: [PATCH 1268/1541] Native reader/sink for sorted key/value data ----Release Notes---- [] ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=111456179 --- .../sdk/runners/worker/IsmFormat.java | 276 +++++++++++++++ .../sdk/runners/worker/IsmReader.java | 316 ++++++++++++++++++ .../dataflow/sdk/runners/worker/IsmSink.java | 188 +++++++++++ .../dataflow/sdk/util/RandomAccessData.java | 266 +++++++++++++++ .../sdk/runners/worker/IsmFormatTest.java | 86 +++++ .../sdk/runners/worker/IsmReaderTest.java | 266 +++++++++++++++ .../sdk/runners/worker/IsmSinkTest.java | 90 +++++ .../sdk/util/RandomAccessDataTest.java | 158 +++++++++ 8 files changed, 1646 insertions(+) create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/IsmFormat.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/IsmReader.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/IsmSink.java create mode 100644 sdk/src/main/java/com/google/cloud/dataflow/sdk/util/RandomAccessData.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/IsmFormatTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/IsmReaderTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/worker/IsmSinkTest.java create mode 100644 sdk/src/test/java/com/google/cloud/dataflow/sdk/util/RandomAccessDataTest.java diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/IsmFormat.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/IsmFormat.java new file mode 100644 index 0000000000000..517b0e1a115b8 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/worker/IsmFormat.java @@ -0,0 +1,276 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.runners.worker; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.util.ScalableBloomFilter; +import com.google.cloud.dataflow.sdk.util.VarInt; +import com.google.common.base.MoreObjects; +import com.google.common.base.Preconditions; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Objects; + +/** + * An Ism file is a prefix encoded key value file with a bloom filter and an index to + * enable lookups. + * + *

An Ism file is composed of these high level sections (in order): + *

    + *
  • data block
  • + *
  • bloom filter (See {@link ScalableBloomFilter} for details on encoding format)
  • + *
  • index
  • + *
  • footer (See {@link Footer} for details on encoding format)
  • + *
+ * + *

The data block is composed of multiple copies of the following: + *

    + *
  • key prefix (See {@link KeyPrefix} for details on encoding format)
  • + *
  • unshared key bytes
  • + *
  • value bytes
  • + *
+ * + *

The index is composed of {@code N} copies of the following: + *

    + *
  • key prefix (See {@link KeyPrefix} for details on encoding format)
  • + *
  • unshared key bytes
  • + *
  • byte offset to key prefix in data block (variable length long coding)
  • + *
+ */ +class IsmFormat { + /** + * The prefix used before each key which contains the number of shared and unshared + * bytes from the previous key that was read. The key prefix along with the previous key + * and the unshared key bytes allows one to construct the current key by doing the following + * {@code currentKey = previousKey[0 : sharedBytes] + read(unsharedBytes)}. + * + *

The key prefix is encoded as: + *

    + *
  • number of shared key bytes (variable length integer coding)
  • + *
  • number of unshared key bytes (variable length integer coding)
  • + *
+ */ + static class KeyPrefix { + private final int sharedKeySize; + private final int unsharedKeySize; + + KeyPrefix(int sharedBytes, int unsharedBytes) { + this.sharedKeySize = sharedBytes; + this.unsharedKeySize = unsharedBytes; + } + + public int getSharedKeySize() { + return sharedKeySize; + } + + public int getUnsharedKeySize() { + return unsharedKeySize; + } + + @Override + public int hashCode() { + return Objects.hash(sharedKeySize, unsharedKeySize); + } + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + if (!(other instanceof KeyPrefix)) { + return false; + } + KeyPrefix keyPrefix = (KeyPrefix) other; + return sharedKeySize == keyPrefix.sharedKeySize + && unsharedKeySize == keyPrefix.unsharedKeySize; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("sharedKeySize", sharedKeySize) + .add("unsharedKeySize", unsharedKeySize) + .toString(); + } + } + + /** A {@link Coder} for {@link KeyPrefix}. */ + static final class KeyPrefixCoder extends AtomicCoder { + private static final KeyPrefixCoder INSTANCE = new KeyPrefixCoder(); + + @JsonCreator + public static KeyPrefixCoder of() { + return INSTANCE; + } + + @Override + public void encode(KeyPrefix value, OutputStream outStream, Coder.Context context) + throws CoderException, IOException { + VarInt.encode(value.sharedKeySize, outStream); + VarInt.encode(value.unsharedKeySize, outStream); + } + + @Override + public KeyPrefix decode(InputStream inStream, Coder.Context context) + throws CoderException, IOException { + return new KeyPrefix(VarInt.decodeInt(inStream), VarInt.decodeInt(inStream)); + } + + @Override + public boolean consistentWithEquals() { + return true; + } + + @Override + public boolean isRegisterByteSizeObserverCheap(KeyPrefix value, Coder.Context context) { + return true; + } + + @Override + protected long getEncodedElementByteSize(KeyPrefix value, Coder.Context context) + throws Exception { + Preconditions.checkNotNull(value); + return VarInt.getLength(value.sharedKeySize) + VarInt.getLength(value.unsharedKeySize); + } + } + + /** + * The footer stores the relevant information required to locate the index and bloom filter. + * It also stores a version byte and the number of keys stored. + * + *

The footer is encoded as the value containing: + *

    + *
  • start of bloom filter offset (big endian long coding)
  • + *
  • start of index position offset (big endian long coding)
  • + *
  • number of keys in file (big endian long coding)
  • + *
  • 0x01 (version key as a single byte)
  • + *
+ */ + static class Footer { + static final int LONG_BYTES = 8; + static final long FIXED_LENGTH = 3 * LONG_BYTES + 1; + static final byte VERSION = 1; + + private final long indexPosition; + private final long bloomFilterPosition; + private final long numberOfKeys; + + Footer(long indexPosition, long bloomFilterPosition, long numberOfKeys) { + this.indexPosition = indexPosition; + this.bloomFilterPosition = bloomFilterPosition; + this.numberOfKeys = numberOfKeys; + } + + public long getIndexPosition() { + return indexPosition; + } + + public long getBloomFilterPosition() { + return bloomFilterPosition; + } + + public long getNumberOfKeys() { + return numberOfKeys; + } + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + if (!(other instanceof Footer)) { + return false; + } + Footer footer = (Footer) other; + return indexPosition == footer.indexPosition + && bloomFilterPosition == footer.bloomFilterPosition + && numberOfKeys == footer.numberOfKeys; + } + + @Override + public int hashCode() { + return Objects.hash(indexPosition, bloomFilterPosition, numberOfKeys); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("version", 1) + .add("indexPosition", indexPosition) + .add("bloomFilterPosition", bloomFilterPosition) + .add("numberOfKeys", numberOfKeys) + .toString(); + } + } + + /** A {@link Coder} for {@link Footer}. */ + static final class FooterCoder extends AtomicCoder